|
@@ -16,6 +16,7 @@
|
|
|
* limitations under the License.
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
import org.apache.commons.io.IOUtils;
|
|
|
import org.apache.hadoop.fs.ChecksumException;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
@@ -26,6 +27,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
|
|
+import org.apache.hadoop.io.nativeio.NativeIO;
|
|
|
import org.apache.hadoop.net.unix.DomainSocket;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.util.NativeCodeLoader;
|
|
@@ -39,13 +41,20 @@ import org.junit.rules.ExpectedException;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
+import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
import static org.apache.hadoop.fs.StorageType.DEFAULT;
|
|
|
import static org.apache.hadoop.fs.StorageType.RAM_DISK;
|
|
|
import static org.hamcrest.CoreMatchers.equalTo;
|
|
|
+import static org.hamcrest.core.Is.is;
|
|
|
import static org.junit.Assert.assertThat;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
+/**
|
|
|
+ * Test Lazy persist behavior with short-circuit reads. These tests
|
|
|
+ * will be run on Linux only with Native IO enabled. The tests fake
|
|
|
+ * RAM_DISK storage using local disk.
|
|
|
+ */
|
|
|
public class TestScrLazyPersistFiles extends LazyPersistTestCase {
|
|
|
|
|
|
@BeforeClass
|
|
@@ -58,6 +67,10 @@ public class TestScrLazyPersistFiles extends LazyPersistTestCase {
|
|
|
Assume.assumeThat(NativeCodeLoader.isNativeCodeLoaded() && !Path.WINDOWS,
|
|
|
equalTo(true));
|
|
|
Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
|
|
|
+
|
|
|
+ final long osPageSize = NativeIO.POSIX.getCacheManipulator().getOperatingSystemPageSize();
|
|
|
+ Preconditions.checkState(BLOCK_SIZE >= osPageSize);
|
|
|
+ Preconditions.checkState(BLOCK_SIZE % osPageSize == 0);
|
|
|
}
|
|
|
|
|
|
@Rule
|
|
@@ -69,35 +82,27 @@ public class TestScrLazyPersistFiles extends LazyPersistTestCase {
|
|
|
*/
|
|
|
@Test
|
|
|
public void testRamDiskShortCircuitRead()
|
|
|
- throws IOException, InterruptedException {
|
|
|
- getClusterBuilder().setNumDatanodes(REPL_FACTOR)
|
|
|
- .setStorageTypes(new StorageType[]{RAM_DISK, DEFAULT})
|
|
|
- .setRamDiskStorageLimit(2 * BLOCK_SIZE - 1)
|
|
|
- .setUseScr(true)
|
|
|
- .build();
|
|
|
+ throws IOException, InterruptedException, TimeoutException {
|
|
|
+ getClusterBuilder().setUseScr(true).build();
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
final int SEED = 0xFADED;
|
|
|
Path path = new Path("/" + METHOD_NAME + ".dat");
|
|
|
|
|
|
+ // Create a file and wait till it is persisted.
|
|
|
makeRandomTestFile(path, BLOCK_SIZE, true, SEED);
|
|
|
ensureFileReplicasOnStorageType(path, RAM_DISK);
|
|
|
+ waitForMetric("RamDiskBlocksLazyPersisted", 1);
|
|
|
|
|
|
- // Sleep for a short time to allow the lazy writer thread to do its job
|
|
|
- Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
|
|
|
-
|
|
|
- //assertThat(verifyReadRandomFile(path, BLOCK_SIZE, SEED), is(true));
|
|
|
- FSDataInputStream fis = fs.open(path);
|
|
|
+ HdfsDataInputStream fis = (HdfsDataInputStream) fs.open(path);
|
|
|
|
|
|
// Verify SCR read counters
|
|
|
try {
|
|
|
- fis = fs.open(path);
|
|
|
byte[] buf = new byte[BUFFER_LENGTH];
|
|
|
fis.read(0, buf, 0, BUFFER_LENGTH);
|
|
|
- HdfsDataInputStream dfsis = (HdfsDataInputStream) fis;
|
|
|
Assert.assertEquals(BUFFER_LENGTH,
|
|
|
- dfsis.getReadStatistics().getTotalBytesRead());
|
|
|
+ fis.getReadStatistics().getTotalBytesRead());
|
|
|
Assert.assertEquals(BUFFER_LENGTH,
|
|
|
- dfsis.getReadStatistics().getTotalShortCircuitBytesRead());
|
|
|
+ fis.getReadStatistics().getTotalShortCircuitBytesRead());
|
|
|
} finally {
|
|
|
fis.close();
|
|
|
fis = null;
|
|
@@ -111,106 +116,77 @@ public class TestScrLazyPersistFiles extends LazyPersistTestCase {
|
|
|
* @throws InterruptedException
|
|
|
*/
|
|
|
@Test
|
|
|
- public void testRamDiskEvictionWithShortCircuitReadHandle()
|
|
|
- throws IOException, InterruptedException {
|
|
|
- // 5 replica + delta, SCR.
|
|
|
- getClusterBuilder().setNumDatanodes(REPL_FACTOR)
|
|
|
- .setStorageTypes(new StorageType[]{RAM_DISK, DEFAULT})
|
|
|
- .setRamDiskStorageLimit(6 * BLOCK_SIZE - 1)
|
|
|
- .setEvictionLowWatermarkReplicas(3)
|
|
|
- .setUseScr(true)
|
|
|
- .build();
|
|
|
-
|
|
|
+ public void tesScrDuringEviction()
|
|
|
+ throws Exception {
|
|
|
+ getClusterBuilder().setUseScr(true).build();
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
|
|
|
- Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
|
|
|
- final int SEED = 0xFADED;
|
|
|
|
|
|
- makeRandomTestFile(path1, BLOCK_SIZE, true, SEED);
|
|
|
+ // Create a file and wait till it is persisted.
|
|
|
+ makeTestFile(path1, BLOCK_SIZE, true);
|
|
|
ensureFileReplicasOnStorageType(path1, RAM_DISK);
|
|
|
+ waitForMetric("RamDiskBlocksLazyPersisted", 1);
|
|
|
|
|
|
- // Sleep for a short time to allow the lazy writer thread to do its job.
|
|
|
- // However the block replica should not be evicted from RAM_DISK yet.
|
|
|
- Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
|
|
|
-
|
|
|
- // No eviction should happen as the free ratio is below the threshold
|
|
|
- FSDataInputStream fis = fs.open(path1);
|
|
|
+ HdfsDataInputStream fis = (HdfsDataInputStream) fs.open(path1);
|
|
|
try {
|
|
|
// Keep and open read handle to path1 while creating path2
|
|
|
byte[] buf = new byte[BUFFER_LENGTH];
|
|
|
fis.read(0, buf, 0, BUFFER_LENGTH);
|
|
|
-
|
|
|
- // Create the 2nd file that will trigger RAM_DISK eviction.
|
|
|
- makeTestFile(path2, BLOCK_SIZE * 2, true);
|
|
|
- ensureFileReplicasOnStorageType(path2, RAM_DISK);
|
|
|
+ triggerEviction(cluster.getDataNodes().get(0));
|
|
|
|
|
|
// Ensure path1 is still readable from the open SCR handle.
|
|
|
- fis.read(fis.getPos(), buf, 0, BUFFER_LENGTH);
|
|
|
- HdfsDataInputStream dfsis = (HdfsDataInputStream) fis;
|
|
|
- Assert.assertEquals(2 * BUFFER_LENGTH,
|
|
|
- dfsis.getReadStatistics().getTotalBytesRead());
|
|
|
- Assert.assertEquals(2 * BUFFER_LENGTH,
|
|
|
- dfsis.getReadStatistics().getTotalShortCircuitBytesRead());
|
|
|
+ fis.read(0, buf, 0, BUFFER_LENGTH);
|
|
|
+ assertThat(fis.getReadStatistics().getTotalBytesRead(),
|
|
|
+ is((long) 2 * BUFFER_LENGTH));
|
|
|
+ assertThat(fis.getReadStatistics().getTotalShortCircuitBytesRead(),
|
|
|
+ is((long) 2 * BUFFER_LENGTH));
|
|
|
} finally {
|
|
|
IOUtils.closeQuietly(fis);
|
|
|
}
|
|
|
-
|
|
|
- // After the open handle is closed, path1 should be evicted to DISK.
|
|
|
- triggerBlockReport();
|
|
|
- ensureFileReplicasOnStorageType(path1, DEFAULT);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testShortCircuitReadAfterEviction()
|
|
|
- throws IOException, InterruptedException {
|
|
|
- Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
|
|
|
- getClusterBuilder().setRamDiskReplicaCapacity(1 + EVICTION_LOW_WATERMARK)
|
|
|
- .setUseScr(true)
|
|
|
+ public void testScrAfterEviction()
|
|
|
+ throws IOException, InterruptedException, TimeoutException {
|
|
|
+ getClusterBuilder().setUseScr(true)
|
|
|
.setUseLegacyBlockReaderLocal(false)
|
|
|
.build();
|
|
|
doShortCircuitReadAfterEvictionTest();
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testLegacyShortCircuitReadAfterEviction()
|
|
|
- throws IOException, InterruptedException {
|
|
|
- getClusterBuilder().setRamDiskReplicaCapacity(1 + EVICTION_LOW_WATERMARK)
|
|
|
- .setUseScr(true)
|
|
|
+ public void testLegacyScrAfterEviction()
|
|
|
+ throws IOException, InterruptedException, TimeoutException {
|
|
|
+ getClusterBuilder().setUseScr(true)
|
|
|
.setUseLegacyBlockReaderLocal(true)
|
|
|
.build();
|
|
|
doShortCircuitReadAfterEvictionTest();
|
|
|
+
|
|
|
+ // In the implementation of legacy short-circuit reads, any failure is
|
|
|
+ // trapped silently, reverts back to a remote read, and also disables all
|
|
|
+ // subsequent legacy short-circuit reads in the ClientContext.
|
|
|
+ // Assert that it didn't get disabled.
|
|
|
+ ClientContext clientContext = client.getClientContext();
|
|
|
+ Assert.assertFalse(clientContext.getDisableLegacyBlockReaderLocal());
|
|
|
}
|
|
|
|
|
|
private void doShortCircuitReadAfterEvictionTest() throws IOException,
|
|
|
- InterruptedException {
|
|
|
+ InterruptedException, TimeoutException {
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
|
|
|
- Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
|
|
|
|
|
|
final int SEED = 0xFADED;
|
|
|
makeRandomTestFile(path1, BLOCK_SIZE, true, SEED);
|
|
|
+ ensureFileReplicasOnStorageType(path1, RAM_DISK);
|
|
|
+ waitForMetric("RamDiskBlocksLazyPersisted", 1);
|
|
|
|
|
|
// Verify short-circuit read from RAM_DISK.
|
|
|
- ensureFileReplicasOnStorageType(path1, RAM_DISK);
|
|
|
File metaFile = cluster.getBlockMetadataFile(0,
|
|
|
DFSTestUtil.getFirstBlock(fs, path1));
|
|
|
assertTrue(metaFile.length() <= BlockMetadataHeader.getHeaderSize());
|
|
|
assertTrue(verifyReadRandomFile(path1, BLOCK_SIZE, SEED));
|
|
|
|
|
|
- // Sleep for a short time to allow the lazy writer thread to do its job.
|
|
|
- Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
|
|
|
-
|
|
|
- // Verify short-circuit read from RAM_DISK once again.
|
|
|
- ensureFileReplicasOnStorageType(path1, RAM_DISK);
|
|
|
- metaFile = cluster.getBlockMetadataFile(0,
|
|
|
- DFSTestUtil.getFirstBlock(fs, path1));
|
|
|
- assertTrue(metaFile.length() <= BlockMetadataHeader.getHeaderSize());
|
|
|
- assertTrue(verifyReadRandomFile(path1, BLOCK_SIZE, SEED));
|
|
|
-
|
|
|
- // Create another file with a replica on RAM_DISK, which evicts the first.
|
|
|
- makeRandomTestFile(path2, BLOCK_SIZE, true, SEED);
|
|
|
- Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
|
|
|
- triggerBlockReport();
|
|
|
+ triggerEviction(cluster.getDataNodes().get(0));
|
|
|
|
|
|
// Verify short-circuit read still works from DEFAULT storage. This time,
|
|
|
// we'll have a checksum written during lazy persistence.
|
|
@@ -219,54 +195,35 @@ public class TestScrLazyPersistFiles extends LazyPersistTestCase {
|
|
|
DFSTestUtil.getFirstBlock(fs, path1));
|
|
|
assertTrue(metaFile.length() > BlockMetadataHeader.getHeaderSize());
|
|
|
assertTrue(verifyReadRandomFile(path1, BLOCK_SIZE, SEED));
|
|
|
-
|
|
|
- // In the implementation of legacy short-circuit reads, any failure is
|
|
|
- // trapped silently, reverts back to a remote read, and also disables all
|
|
|
- // subsequent legacy short-circuit reads in the ClientContext. If the test
|
|
|
- // uses legacy, then assert that it didn't get disabled.
|
|
|
- ClientContext clientContext = client.getClientContext();
|
|
|
- if (clientContext.getUseLegacyBlockReaderLocal()) {
|
|
|
- Assert.assertFalse(clientContext.getDisableLegacyBlockReaderLocal());
|
|
|
- }
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testShortCircuitReadBlockFileCorruption() throws IOException,
|
|
|
- InterruptedException {
|
|
|
- Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
|
|
|
- getClusterBuilder().setRamDiskReplicaCapacity(1 + EVICTION_LOW_WATERMARK)
|
|
|
- .setUseScr(true)
|
|
|
+ public void testScrBlockFileCorruption() throws IOException,
|
|
|
+ InterruptedException, TimeoutException {
|
|
|
+ getClusterBuilder().setUseScr(true)
|
|
|
.setUseLegacyBlockReaderLocal(false)
|
|
|
.build();
|
|
|
doShortCircuitReadBlockFileCorruptionTest();
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testLegacyShortCircuitReadBlockFileCorruption() throws IOException,
|
|
|
- InterruptedException {
|
|
|
- getClusterBuilder().setRamDiskReplicaCapacity(1 + EVICTION_LOW_WATERMARK)
|
|
|
- .setUseScr(true)
|
|
|
+ public void testLegacyScrBlockFileCorruption() throws IOException,
|
|
|
+ InterruptedException, TimeoutException {
|
|
|
+ getClusterBuilder().setUseScr(true)
|
|
|
.setUseLegacyBlockReaderLocal(true)
|
|
|
.build();
|
|
|
doShortCircuitReadBlockFileCorruptionTest();
|
|
|
}
|
|
|
|
|
|
public void doShortCircuitReadBlockFileCorruptionTest() throws IOException,
|
|
|
- InterruptedException {
|
|
|
+ InterruptedException, TimeoutException {
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
|
|
|
- Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
|
|
|
|
|
|
- final int SEED = 0xFADED;
|
|
|
- makeRandomTestFile(path1, BLOCK_SIZE, true, SEED);
|
|
|
+ makeTestFile(path1, BLOCK_SIZE, true);
|
|
|
ensureFileReplicasOnStorageType(path1, RAM_DISK);
|
|
|
-
|
|
|
- // Create another file with a replica on RAM_DISK, which evicts the first.
|
|
|
- makeRandomTestFile(path2, BLOCK_SIZE, true, SEED);
|
|
|
-
|
|
|
- // Sleep for a short time to allow the lazy writer thread to do its job.
|
|
|
- Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
|
|
|
- triggerBlockReport();
|
|
|
+ waitForMetric("RamDiskBlocksLazyPersisted", 1);
|
|
|
+ triggerEviction(cluster.getDataNodes().get(0));
|
|
|
|
|
|
// Corrupt the lazy-persisted block file, and verify that checksum
|
|
|
// verification catches it.
|
|
@@ -277,42 +234,32 @@ public class TestScrLazyPersistFiles extends LazyPersistTestCase {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testShortCircuitReadMetaFileCorruption() throws IOException,
|
|
|
- InterruptedException {
|
|
|
- Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
|
|
|
- getClusterBuilder().setRamDiskReplicaCapacity(1 + EVICTION_LOW_WATERMARK)
|
|
|
- .setUseScr(true)
|
|
|
+ public void testScrMetaFileCorruption() throws IOException,
|
|
|
+ InterruptedException, TimeoutException {
|
|
|
+ getClusterBuilder().setUseScr(true)
|
|
|
.setUseLegacyBlockReaderLocal(false)
|
|
|
.build();
|
|
|
doShortCircuitReadMetaFileCorruptionTest();
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testLegacyShortCircuitReadMetaFileCorruption() throws IOException,
|
|
|
- InterruptedException {
|
|
|
- getClusterBuilder().setRamDiskReplicaCapacity(1 + EVICTION_LOW_WATERMARK)
|
|
|
- .setUseScr(true)
|
|
|
+ public void testLegacyScrMetaFileCorruption() throws IOException,
|
|
|
+ InterruptedException, TimeoutException {
|
|
|
+ getClusterBuilder().setUseScr(true)
|
|
|
.setUseLegacyBlockReaderLocal(true)
|
|
|
.build();
|
|
|
doShortCircuitReadMetaFileCorruptionTest();
|
|
|
}
|
|
|
|
|
|
public void doShortCircuitReadMetaFileCorruptionTest() throws IOException,
|
|
|
- InterruptedException {
|
|
|
+ InterruptedException, TimeoutException {
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
|
|
|
- Path path2 = new Path("/" + METHOD_NAME + ".02.dat");
|
|
|
|
|
|
- final int SEED = 0xFADED;
|
|
|
- makeRandomTestFile(path1, BLOCK_SIZE, true, SEED);
|
|
|
+ makeTestFile(path1, BLOCK_SIZE, true);
|
|
|
ensureFileReplicasOnStorageType(path1, RAM_DISK);
|
|
|
-
|
|
|
- // Create another file with a replica on RAM_DISK, which evicts the first.
|
|
|
- makeRandomTestFile(path2, BLOCK_SIZE, true, SEED);
|
|
|
-
|
|
|
- // Sleep for a short time to allow the lazy writer thread to do its job.
|
|
|
- Thread.sleep(3 * LAZY_WRITER_INTERVAL_SEC * 1000);
|
|
|
- triggerBlockReport();
|
|
|
+ waitForMetric("RamDiskBlocksLazyPersisted", 1);
|
|
|
+ triggerEviction(cluster.getDataNodes().get(0));
|
|
|
|
|
|
// Corrupt the lazy-persisted checksum file, and verify that checksum
|
|
|
// verification catches it.
|