|
@@ -36,11 +36,16 @@ import org.apache.hadoop.hdfs.ClientContext;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSInputStream;
|
|
import org.apache.hadoop.hdfs.DFSInputStream;
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
|
+import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
import org.apache.hadoop.hdfs.ExtendedBlockId;
|
|
import org.apache.hadoop.hdfs.ExtendedBlockId;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
|
+import org.apache.hadoop.hdfs.ReadStatistics;
|
|
|
|
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
|
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.BlockType;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
|
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
|
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache;
|
|
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache;
|
|
@@ -783,4 +788,59 @@ public class TestBlockReaderLocal {
|
|
if (sockDir != null) sockDir.close();
|
|
if (sockDir != null) sockDir.close();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ @Test(timeout = 60000)
|
|
|
|
+ public void testStatisticsForErasureCodingRead() throws IOException {
|
|
|
|
+ HdfsConfiguration conf = new HdfsConfiguration();
|
|
|
|
+
|
|
|
|
+ final ErasureCodingPolicy ecPolicy =
|
|
|
|
+ StripedFileTestUtil.getDefaultECPolicy();
|
|
|
|
+ final int numDataNodes =
|
|
|
|
+ ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits();
|
|
|
|
+ // The length of test file is one full strip + one partial stripe. And
|
|
|
|
+ // it is not bound to the stripe cell size.
|
|
|
|
+ final int length = ecPolicy.getCellSize() * (numDataNodes + 1) + 123;
|
|
|
|
+ final long randomSeed = 4567L;
|
|
|
|
+ final short repl = 1;
|
|
|
|
+ try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
|
+ .numDataNodes(numDataNodes).build()) {
|
|
|
|
+ cluster.waitActive();
|
|
|
|
+ DistributedFileSystem fs = cluster.getFileSystem();
|
|
|
|
+ fs.enableErasureCodingPolicy(ecPolicy.getName());
|
|
|
|
+
|
|
|
|
+ Path ecDir = new Path("/ec");
|
|
|
|
+ fs.mkdirs(ecDir);
|
|
|
|
+ fs.setErasureCodingPolicy(ecDir, ecPolicy.getName());
|
|
|
|
+ Path nonEcDir = new Path("/noEc");
|
|
|
|
+ fs.mkdirs(nonEcDir);
|
|
|
|
+
|
|
|
|
+ byte[] buf = new byte[length];
|
|
|
|
+
|
|
|
|
+ Path nonEcFile = new Path(nonEcDir, "file1");
|
|
|
|
+ DFSTestUtil.createFile(fs, nonEcFile, length, repl, randomSeed);
|
|
|
|
+ try (HdfsDataInputStream in = (HdfsDataInputStream) fs.open(nonEcFile)) {
|
|
|
|
+ IOUtils.readFully(in, buf, 0, length);
|
|
|
|
+
|
|
|
|
+ ReadStatistics stats = in.getReadStatistics();
|
|
|
|
+ Assert.assertEquals(BlockType.CONTIGUOUS, stats.getBlockType());
|
|
|
|
+ Assert.assertEquals(length, stats.getTotalBytesRead());
|
|
|
|
+ Assert.assertEquals(length, stats.getTotalLocalBytesRead());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ Path ecFile = new Path(ecDir, "file2");
|
|
|
|
+ DFSTestUtil.createFile(fs, ecFile, length, repl, randomSeed);
|
|
|
|
+ // Shutdown one DataNode so that erasure coding decoding process can kick
|
|
|
|
+ // in.
|
|
|
|
+ cluster.shutdownDataNode(0);
|
|
|
|
+ try (HdfsDataInputStream in = (HdfsDataInputStream) fs.open(ecFile)) {
|
|
|
|
+ IOUtils.readFully(in, buf, 0, length);
|
|
|
|
+
|
|
|
|
+ ReadStatistics stats = in.getReadStatistics();
|
|
|
|
+ Assert.assertEquals(BlockType.STRIPED, stats.getBlockType());
|
|
|
|
+ Assert.assertEquals(length, stats.getTotalLocalBytesRead());
|
|
|
|
+ Assert.assertEquals(length, stats.getTotalBytesRead());
|
|
|
|
+ Assert.assertTrue(stats.getTotalEcDecodingTimeMillis() > 0);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|