|
@@ -15,7 +15,7 @@
|
|
|
* See the License for the specific language governing permissions and
|
|
|
* limitations under the License.
|
|
|
*/
|
|
|
-package org.apache.hadoop.hdfs;
|
|
|
+package org.apache.hadoop.hdfs.client.impl;
|
|
|
|
|
|
import static org.hamcrest.CoreMatchers.equalTo;
|
|
|
|
|
@@ -32,9 +32,15 @@ import org.apache.hadoop.fs.ChecksumException;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.hdfs.ClientContext;
|
|
|
+import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
+import org.apache.hadoop.hdfs.DFSInputStream;
|
|
|
+import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
+import org.apache.hadoop.hdfs.ExtendedBlockId;
|
|
|
+import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
+import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
|
|
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
|
|
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache;
|
|
@@ -55,24 +61,24 @@ import org.junit.Test;
|
|
|
|
|
|
public class TestBlockReaderLocal {
|
|
|
private static TemporarySocketDirectory sockDir;
|
|
|
-
|
|
|
+
|
|
|
@BeforeClass
|
|
|
public static void init() {
|
|
|
sockDir = new TemporarySocketDirectory();
|
|
|
DomainSocket.disableBindPathValidation();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@AfterClass
|
|
|
public static void shutdown() throws IOException {
|
|
|
sockDir.close();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
public static void assertArrayRegionsEqual(byte []buf1, int off1, byte []buf2,
|
|
|
int off2, int len) {
|
|
|
for (int i = 0; i < len; i++) {
|
|
|
if (buf1[off1 + i] != buf2[off2 + i]) {
|
|
|
- Assert.fail("arrays differ at byte " + i + ". " +
|
|
|
- "The first array has " + (int)buf1[off1 + i] +
|
|
|
+ Assert.fail("arrays differ at byte " + i + ". " +
|
|
|
+ "The first array has " + (int)buf1[off1 + i] +
|
|
|
", but the second array has " + (int)buf2[off2 + i]);
|
|
|
}
|
|
|
}
|
|
@@ -85,7 +91,7 @@ public class TestBlockReaderLocal {
|
|
|
* @param buf The ByteBuffer to read into
|
|
|
* @param off The offset in the buffer to read into
|
|
|
* @param len The number of bytes to read.
|
|
|
- *
|
|
|
+ *
|
|
|
* @throws IOException If it could not read the requested number of bytes
|
|
|
*/
|
|
|
private static void readFully(BlockReaderLocal reader,
|
|
@@ -120,7 +126,7 @@ public class TestBlockReaderLocal {
|
|
|
// default: no-op
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
public void runBlockReaderLocalTest(BlockReaderLocalTest test,
|
|
|
boolean checksum, long readahead) throws IOException {
|
|
|
Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
|
|
@@ -139,7 +145,7 @@ public class TestBlockReaderLocal {
|
|
|
BlockReaderLocal blockReaderLocal = null;
|
|
|
FSDataInputStream fsIn = null;
|
|
|
byte original[] = new byte[BlockReaderLocalTest.TEST_LENGTH];
|
|
|
-
|
|
|
+
|
|
|
FileSystem fs = null;
|
|
|
ShortCircuitShm shm = null;
|
|
|
RandomAccessFile raf = null;
|
|
@@ -186,7 +192,7 @@ public class TestBlockReaderLocal {
|
|
|
raf.setLength(8192);
|
|
|
FileInputStream shmStream = new FileInputStream(raf.getFD());
|
|
|
shm = new ShortCircuitShm(ShmId.createRandom(), shmStream);
|
|
|
- ShortCircuitReplica replica =
|
|
|
+ ShortCircuitReplica replica =
|
|
|
new ShortCircuitReplica(key, dataIn, metaIn, shortCircuitCache,
|
|
|
Time.now(), shm.allocAndRegisterSlot(
|
|
|
ExtendedBlockId.fromExtendedBlock(block)));
|
|
@@ -216,21 +222,21 @@ public class TestBlockReaderLocal {
|
|
|
if (raf != null) raf.close();
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- private static class TestBlockReaderLocalImmediateClose
|
|
|
+
|
|
|
+ private static class TestBlockReaderLocalImmediateClose
|
|
|
extends BlockReaderLocalTest {
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Test
|
|
|
public void testBlockReaderLocalImmediateClose() throws IOException {
|
|
|
runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), true, 0);
|
|
|
runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), false, 0);
|
|
|
}
|
|
|
-
|
|
|
- private static class TestBlockReaderSimpleReads
|
|
|
+
|
|
|
+ private static class TestBlockReaderSimpleReads
|
|
|
extends BlockReaderLocalTest {
|
|
|
@Override
|
|
|
- public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
+ public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
throws IOException {
|
|
|
byte buf[] = new byte[TEST_LENGTH];
|
|
|
reader.readFully(buf, 0, 512);
|
|
@@ -246,7 +252,7 @@ public class TestBlockReaderLocal {
|
|
|
BlockReaderLocalTest.BYTES_PER_CHECKSUM);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Test
|
|
|
public void testBlockReaderSimpleReads() throws IOException {
|
|
|
runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true,
|
|
@@ -275,11 +281,11 @@ public class TestBlockReaderLocal {
|
|
|
throws IOException {
|
|
|
runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), false, 0);
|
|
|
}
|
|
|
-
|
|
|
- private static class TestBlockReaderLocalArrayReads2
|
|
|
+
|
|
|
+ private static class TestBlockReaderLocalArrayReads2
|
|
|
extends BlockReaderLocalTest {
|
|
|
@Override
|
|
|
- public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
+ public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
throws IOException {
|
|
|
byte buf[] = new byte[TEST_LENGTH];
|
|
|
reader.readFully(buf, 0, 10);
|
|
@@ -296,7 +302,7 @@ public class TestBlockReaderLocal {
|
|
|
assertArrayRegionsEqual(original, 1716, buf, 1716, 5);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Test
|
|
|
public void testBlockReaderLocalArrayReads2() throws IOException {
|
|
|
runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(),
|
|
@@ -322,10 +328,10 @@ public class TestBlockReaderLocal {
|
|
|
runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(), false, 0);
|
|
|
}
|
|
|
|
|
|
- private static class TestBlockReaderLocalByteBufferReads
|
|
|
+ private static class TestBlockReaderLocalByteBufferReads
|
|
|
extends BlockReaderLocalTest {
|
|
|
@Override
|
|
|
- public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
+ public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
throws IOException {
|
|
|
ByteBuffer buf = ByteBuffer.wrap(new byte[TEST_LENGTH]);
|
|
|
readFully(reader, buf, 0, 10);
|
|
@@ -339,7 +345,7 @@ public class TestBlockReaderLocal {
|
|
|
assertArrayRegionsEqual(original, 811, buf.array(), 811, 5);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Test
|
|
|
public void testBlockReaderLocalByteBufferReads()
|
|
|
throws IOException {
|
|
@@ -354,7 +360,7 @@ public class TestBlockReaderLocal {
|
|
|
new TestBlockReaderLocalByteBufferReads(),
|
|
|
false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Test
|
|
|
public void testBlockReaderLocalByteBufferReadsNoReadahead()
|
|
|
throws IOException {
|
|
@@ -373,7 +379,7 @@ public class TestBlockReaderLocal {
|
|
|
* Test reads that bypass the bounce buffer (because they are aligned
|
|
|
* and bigger than the readahead).
|
|
|
*/
|
|
|
- private static class TestBlockReaderLocalByteBufferFastLaneReads
|
|
|
+ private static class TestBlockReaderLocalByteBufferFastLaneReads
|
|
|
extends BlockReaderLocalTest {
|
|
|
@Override
|
|
|
public void doTest(BlockReaderLocal reader, byte original[])
|
|
@@ -410,7 +416,7 @@ public class TestBlockReaderLocal {
|
|
|
50);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Test
|
|
|
public void testBlockReaderLocalByteBufferFastLaneReads()
|
|
|
throws IOException {
|
|
@@ -456,7 +462,7 @@ public class TestBlockReaderLocal {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
+ public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
throws IOException {
|
|
|
byte buf[] = new byte[TEST_LENGTH];
|
|
|
if (usingChecksums) {
|
|
@@ -471,19 +477,19 @@ public class TestBlockReaderLocal {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Test
|
|
|
public void testBlockReaderLocalReadCorruptStart()
|
|
|
throws IOException {
|
|
|
runBlockReaderLocalTest(new TestBlockReaderLocalReadCorruptStart(), true,
|
|
|
HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private static class TestBlockReaderLocalReadCorrupt
|
|
|
extends BlockReaderLocalTest {
|
|
|
boolean usingChecksums = false;
|
|
|
@Override
|
|
|
- public void setup(File blockFile, boolean usingChecksums)
|
|
|
+ public void setup(File blockFile, boolean usingChecksums)
|
|
|
throws IOException {
|
|
|
RandomAccessFile bf = null;
|
|
|
this.usingChecksums = usingChecksums;
|
|
@@ -496,7 +502,7 @@ public class TestBlockReaderLocal {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
+ public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
throws IOException {
|
|
|
byte buf[] = new byte[TEST_LENGTH];
|
|
|
try {
|
|
@@ -522,7 +528,7 @@ public class TestBlockReaderLocal {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Test
|
|
|
public void testBlockReaderLocalReadCorrupt()
|
|
|
throws IOException {
|
|
@@ -555,7 +561,7 @@ public class TestBlockReaderLocal {
|
|
|
public void setup(File blockFile, boolean usingChecksums)
|
|
|
throws IOException {
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
throws IOException {
|
|
@@ -675,7 +681,7 @@ public class TestBlockReaderLocal {
|
|
|
runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
|
|
|
false, 0);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Test
|
|
|
public void testBlockReaderLocalReadZeroBytes()
|
|
|
throws IOException {
|
|
@@ -703,7 +709,7 @@ public class TestBlockReaderLocal {
|
|
|
runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(),
|
|
|
false, 0);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
|
|
|
@Test(timeout=60000)
|
|
|
public void TestStatisticsForShortCircuitLocalRead() throws Exception {
|
|
@@ -714,7 +720,7 @@ public class TestBlockReaderLocal {
|
|
|
public void TestStatisticsForLocalRead() throws Exception {
|
|
|
testStatistics(false);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private void testStatistics(boolean isShortCircuit) throws Exception {
|
|
|
Assume.assumeTrue(DomainSocket.getLoadingFailureReason() == null);
|
|
|
HdfsConfiguration conf = new HdfsConfiguration();
|
|
@@ -756,12 +762,12 @@ public class TestBlockReaderLocal {
|
|
|
IOUtils.readFully(fsIn, original, 0,
|
|
|
BlockReaderLocalTest.TEST_LENGTH);
|
|
|
HdfsDataInputStream dfsIn = (HdfsDataInputStream)fsIn;
|
|
|
- Assert.assertEquals(BlockReaderLocalTest.TEST_LENGTH,
|
|
|
+ Assert.assertEquals(BlockReaderLocalTest.TEST_LENGTH,
|
|
|
dfsIn.getReadStatistics().getTotalBytesRead());
|
|
|
- Assert.assertEquals(BlockReaderLocalTest.TEST_LENGTH,
|
|
|
+ Assert.assertEquals(BlockReaderLocalTest.TEST_LENGTH,
|
|
|
dfsIn.getReadStatistics().getTotalLocalBytesRead());
|
|
|
if (isShortCircuit) {
|
|
|
- Assert.assertEquals(BlockReaderLocalTest.TEST_LENGTH,
|
|
|
+ Assert.assertEquals(BlockReaderLocalTest.TEST_LENGTH,
|
|
|
dfsIn.getReadStatistics().getTotalShortCircuitBytesRead());
|
|
|
} else {
|
|
|
Assert.assertEquals(0,
|