|
@@ -21,10 +21,7 @@ import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
-import org.apache.hadoop.fs.StorageType;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
@@ -34,10 +31,9 @@ import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
|
|
|
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
|
|
import org.junit.After;
|
|
|
-import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
|
|
@@ -54,17 +50,18 @@ public class TestReadStripedFile {
|
|
|
private DistributedFileSystem fs;
|
|
|
private final Path dirPath = new Path("/striped");
|
|
|
private Path filePath = new Path(dirPath, "file");
|
|
|
- private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS;
|
|
|
- private final short TOTAL_SIZE = HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS;
|
|
|
+ private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS;
|
|
|
+ private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS;
|
|
|
+ private final short BLK_GROUP_SIZE = DATA_BLK_NUM + PARITY_BLK_NUM;
|
|
|
private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
|
|
|
private final int NUM_STRIPE_PER_BLOCK = 2;
|
|
|
- private final int BLOCKSIZE = 2 * GROUP_SIZE * CELLSIZE;
|
|
|
+ private final int BLOCKSIZE = NUM_STRIPE_PER_BLOCK * DATA_BLK_NUM * CELLSIZE;
|
|
|
|
|
|
@Before
|
|
|
public void setup() throws IOException {
|
|
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
|
|
|
SimulatedFSDataset.setFactory(conf);
|
|
|
- cluster = new MiniDFSCluster.Builder(conf).numDataNodes(TOTAL_SIZE)
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(BLK_GROUP_SIZE)
|
|
|
.build();
|
|
|
cluster.waitActive();
|
|
|
fs = cluster.getFileSystem();
|
|
@@ -77,72 +74,14 @@ public class TestReadStripedFile {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private LocatedStripedBlock createDummyLocatedBlock() {
|
|
|
- final long blockGroupID = -1048576;
|
|
|
- DatanodeInfo[] locs = new DatanodeInfo[TOTAL_SIZE];
|
|
|
- String[] storageIDs = new String[TOTAL_SIZE];
|
|
|
- StorageType[] storageTypes = new StorageType[TOTAL_SIZE];
|
|
|
- int[] indices = new int[TOTAL_SIZE];
|
|
|
- for (int i = 0; i < TOTAL_SIZE; i++) {
|
|
|
- locs[i] = new DatanodeInfo(cluster.getDataNodes().get(i).getDatanodeId());
|
|
|
- storageIDs[i] = cluster.getDataNodes().get(i).getDatanodeUuid();
|
|
|
- storageTypes[i] = StorageType.DISK;
|
|
|
- indices[i] = (i + 2) % GROUP_SIZE;
|
|
|
- }
|
|
|
- return new LocatedStripedBlock(new ExtendedBlock("pool", blockGroupID),
|
|
|
- locs, storageIDs, storageTypes, indices, 0, false, null);
|
|
|
- }
|
|
|
-
|
|
|
- @Test
|
|
|
- public void testParseDummyStripedBlock() {
|
|
|
- LocatedStripedBlock lsb = createDummyLocatedBlock();
|
|
|
- LocatedBlock[] blocks = DFSStripedInputStream.parseStripedBlockGroup(
|
|
|
- lsb, GROUP_SIZE, CELLSIZE);
|
|
|
- assertEquals(GROUP_SIZE, blocks.length);
|
|
|
- for (int j = 0; j < GROUP_SIZE; j++) {
|
|
|
- assertFalse(blocks[j].isStriped());
|
|
|
- assertEquals(j,
|
|
|
- BlockIdManager.getBlockIndex(blocks[j].getBlock().getLocalBlock()));
|
|
|
- assertEquals(j * CELLSIZE, blocks[j].getStartOffset());
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- @Test
|
|
|
- public void testParseStripedBlock() throws Exception {
|
|
|
- final int numBlocks = 4;
|
|
|
- DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks,
|
|
|
- NUM_STRIPE_PER_BLOCK);
|
|
|
- LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
|
|
|
- filePath.toString(), 0, BLOCKSIZE * numBlocks);
|
|
|
-
|
|
|
- assertEquals(4, lbs.locatedBlockCount());
|
|
|
- List<LocatedBlock> lbList = lbs.getLocatedBlocks();
|
|
|
- for (LocatedBlock lb : lbList) {
|
|
|
- assertTrue(lb.isStriped());
|
|
|
- }
|
|
|
-
|
|
|
- for (int i = 0; i < numBlocks; i++) {
|
|
|
- LocatedStripedBlock lsb = (LocatedStripedBlock) (lbs.get(i));
|
|
|
- LocatedBlock[] blks = DFSStripedInputStream.parseStripedBlockGroup(lsb,
|
|
|
- GROUP_SIZE, CELLSIZE);
|
|
|
- assertEquals(GROUP_SIZE, blks.length);
|
|
|
- for (int j = 0; j < GROUP_SIZE; j++) {
|
|
|
- assertFalse(blks[j].isStriped());
|
|
|
- assertEquals(j,
|
|
|
- BlockIdManager.getBlockIndex(blks[j].getBlock().getLocalBlock()));
|
|
|
- assertEquals(i * BLOCKSIZE + j * CELLSIZE, blks[j].getStartOffset());
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Test {@link DFSStripedInputStream#getBlockAt(long)}
|
|
|
*/
|
|
|
@Test
|
|
|
public void testGetBlock() throws Exception {
|
|
|
final int numBlocks = 4;
|
|
|
- DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks,
|
|
|
- NUM_STRIPE_PER_BLOCK);
|
|
|
+ DFSTestUtil.createStripedFile(cluster, filePath, dirPath, numBlocks,
|
|
|
+ NUM_STRIPE_PER_BLOCK, true);
|
|
|
LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
|
|
|
filePath.toString(), 0, BLOCKSIZE * numBlocks);
|
|
|
final DFSStripedInputStream in =
|
|
@@ -151,9 +90,9 @@ public class TestReadStripedFile {
|
|
|
List<LocatedBlock> lbList = lbs.getLocatedBlocks();
|
|
|
for (LocatedBlock aLbList : lbList) {
|
|
|
LocatedStripedBlock lsb = (LocatedStripedBlock) aLbList;
|
|
|
- LocatedBlock[] blks = DFSStripedInputStream.parseStripedBlockGroup(lsb,
|
|
|
- GROUP_SIZE, CELLSIZE);
|
|
|
- for (int j = 0; j < GROUP_SIZE; j++) {
|
|
|
+ LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(lsb,
|
|
|
+ CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM);
|
|
|
+ for (int j = 0; j < DATA_BLK_NUM; j++) {
|
|
|
LocatedBlock refreshed = in.getBlockAt(blks[j].getStartOffset());
|
|
|
assertEquals(blks[j].getBlock(), refreshed.getBlock());
|
|
|
assertEquals(blks[j].getStartOffset(), refreshed.getStartOffset());
|
|
@@ -165,15 +104,16 @@ public class TestReadStripedFile {
|
|
|
@Test
|
|
|
public void testPread() throws Exception {
|
|
|
final int numBlocks = 4;
|
|
|
- DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks,
|
|
|
- NUM_STRIPE_PER_BLOCK);
|
|
|
+ DFSTestUtil.createStripedFile(cluster, filePath, dirPath, numBlocks,
|
|
|
+ NUM_STRIPE_PER_BLOCK, true);
|
|
|
LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
|
|
|
filePath.toString(), 0, BLOCKSIZE);
|
|
|
|
|
|
assert lbs.get(0) instanceof LocatedStripedBlock;
|
|
|
LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0));
|
|
|
- for (int i = 0; i < GROUP_SIZE; i++) {
|
|
|
- Block blk = new Block(bg.getBlock().getBlockId() + i, BLOCKSIZE,
|
|
|
+ for (int i = 0; i < DATA_BLK_NUM; i++) {
|
|
|
+ Block blk = new Block(bg.getBlock().getBlockId() + i,
|
|
|
+ NUM_STRIPE_PER_BLOCK * CELLSIZE,
|
|
|
bg.getBlock().getGenerationStamp());
|
|
|
blk.setGenerationStamp(bg.getBlock().getGenerationStamp());
|
|
|
cluster.injectBlocks(i, Arrays.asList(blk),
|