|
@@ -19,8 +19,11 @@ package org.apache.hadoop.hdfs;
|
|
|
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
+import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
@@ -561,6 +564,50 @@ public class TestDFSStripedInputStream {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testReadWhenLastIncompleteCellComeInToDecodeAlignedStripe()
|
|
|
+ throws IOException {
|
|
|
+ DataNodeProperties stopDataNode = null;
|
|
|
+ try {
|
|
|
+ cluster.waitActive();
|
|
|
+ ErasureCodingPolicy policy = getEcPolicy();
|
|
|
+ DistributedFileSystem filesystem = cluster.getFileSystem();
|
|
|
+ filesystem.enableErasureCodingPolicy(policy.getName());
|
|
|
+ Path dir = new Path("/tmp");
|
|
|
+ filesystem.mkdirs(dir);
|
|
|
+ filesystem.getClient().setErasureCodingPolicy(dir.toString(),
|
|
|
+ policy.getName());
|
|
|
+ Path f = new Path(dir, "file");
|
|
|
+
|
|
|
+ //1. File with one stripe, last data cell should be half filed.
|
|
|
+ long fileLength = (policy.getCellSize() * policy.getNumDataUnits())
|
|
|
+ - (policy.getCellSize() / 2);
|
|
|
+ DFSTestUtil.createFile(filesystem, f, fileLength, (short) 1, 0);
|
|
|
+
|
|
|
+ //2. Stop first DN from stripe.
|
|
|
+ LocatedBlocks lbs = cluster.getNameNodeRpc().getBlockLocations(
|
|
|
+ f.toString(), 0, fileLength);
|
|
|
+ LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(0));
|
|
|
+ final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(bg,
|
|
|
+ cellSize, dataBlocks, parityBlocks);
|
|
|
+ cluster.stopDataNode(blocks[0].getLocations()[0].getName());
|
|
|
+
|
|
|
+ //3. Do pread for fist cell, reconstruction should happen
|
|
|
+ try (FSDataInputStream in = filesystem.open(f)) {
|
|
|
+ DFSStripedInputStream stripedIn = (DFSStripedInputStream) in
|
|
|
+ .getWrappedStream();
|
|
|
+ byte[] b = new byte[policy.getCellSize()];
|
|
|
+ stripedIn.read(0, b, 0, policy.getCellSize());
|
|
|
+ }
|
|
|
+ } catch (HadoopIllegalArgumentException e) {
|
|
|
+ fail(e.getMessage());
|
|
|
+ } finally {
|
|
|
+ if (stopDataNode != null) {
|
|
|
+ cluster.restartDataNode(stopDataNode, true);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Empties the pool for the specified buffer type, for the current ecPolicy.
|
|
|
* <p>
|