HDFS-8882. Erasure Coding: Use datablocks, parityblocks and cell size from ErasureCodingPolicy. Contributed by Vinayakumar B.

Change-Id: Ic56da0b426f47c63dac440aef6f5fc8554f6cf13
This commit is contained in:
Zhe Zhang 2015-09-23 13:34:09 -07:00
parent 7bff8ca1c8
commit c09dc258a8
38 changed files with 177 additions and 156 deletions

View File

@ -86,17 +86,6 @@ public final class HdfsConstants {
public static final String CLIENT_NAMENODE_PROTOCOL_NAME =
"org.apache.hadoop.hdfs.protocol.ClientProtocol";
/*
* These values correspond to the values used by the system default erasure
* coding policy.
* TODO: get these values from ec policy of the associated INodeFile
*/
public static final byte NUM_DATA_BLOCKS = 6;
public static final byte NUM_PARITY_BLOCKS = 3;
// The chunk size for striped block which is used by erasure coding
public static final int BLOCK_STRIPED_CELL_SIZE = 64 * 1024;
// Timeouts for communicating with DataNode for streaming writes/reads
public static final int READ_TIMEOUT = 60 * 1000;
public static final int READ_TIMEOUT_EXTENSION = 5 * 1000;

View File

@ -536,6 +536,7 @@ message BlockWithLocationsProto {
optional bytes indices = 5;
optional uint32 dataBlockNum = 6;
optional uint32 cellSize = 7;
}
/**

View File

@ -446,4 +446,7 @@
getAllErasureCodingPolicies. (Rakesh R via zhz)
HDFS-8920. Erasure Coding: when recovering lost blocks, logs can be too
verbose and hurt performance. (Rui Li via Kai Zheng)
verbose and hurt performance. (Rui Li via Kai Zheng)
HDFS-8882. Erasure Coding: Use datablocks, parityblocks and cell size from
ErasureCodingPolicy (Vinayakumar B via zhz)

View File

@ -18,25 +18,20 @@
package org.apache.hadoop.hdfs;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator;
import org.apache.hadoop.hdfs.DFSStripedOutputStream.MultipleBlockingQueue;
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.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.util.ByteArrayManager;
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
import org.apache.hadoop.io.erasurecode.ECSchema;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.Progressable;
@ -109,6 +104,8 @@ public class StripedDataStreamer extends DataStreamer {
private final Coordinator coordinator;
private final int index;
private volatile boolean failed;
private final ECSchema schema;
private final int cellSize;
StripedDataStreamer(HdfsFileStatus stat,
DFSClient dfsClient, String src,
@ -120,6 +117,8 @@ public class StripedDataStreamer extends DataStreamer {
byteArrayManage, favoredNodes);
this.index = index;
this.coordinator = coordinator;
this.schema = stat.getErasureCodingPolicy().getSchema();
this.cellSize = stat.getErasureCodingPolicy().getCellSize();
}
int getIndex() {
@ -135,7 +134,7 @@ public class StripedDataStreamer extends DataStreamer {
}
private boolean isParityStreamer() {
return index >= NUM_DATA_BLOCKS;
return index >= schema.getNumDataUnits();
}
@Override
@ -168,7 +167,7 @@ public class StripedDataStreamer extends DataStreamer {
if (block != null) {
// set numByte for the previous block group
long bytes = 0;
for (int i = 0; i < NUM_DATA_BLOCKS; i++) {
for (int i = 0; i < schema.getNumDataUnits(); i++) {
final ExtendedBlock b = coordinator.takeEndBlock(i);
StripedBlockUtil.checkBlocks(index, block, i, b);
bytes += b.getNumBytes();
@ -183,15 +182,15 @@ public class StripedDataStreamer extends DataStreamer {
final LocatedBlock lb = StripedDataStreamer.super.locateFollowingBlock(
excludedNodes);
if (lb.getLocations().length < HdfsConstants.NUM_DATA_BLOCKS) {
if (lb.getLocations().length < schema.getNumDataUnits()) {
throw new IOException(
"Failed to get datablocks number of nodes from namenode: blockGroupSize= "
+ (HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS)
+ (schema.getNumDataUnits() + schema.getNumParityUnits())
+ ", blocks.length= " + lb.getLocations().length);
}
final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
(LocatedStripedBlock)lb,
BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
final LocatedBlock[] blocks =
StripedBlockUtil.parseStripedBlockGroup((LocatedStripedBlock) lb,
cellSize, schema.getNumDataUnits(), schema.getNumParityUnits());
for (int i = 0; i < blocks.length; i++) {
StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
@ -233,9 +232,10 @@ public class StripedDataStreamer extends DataStreamer {
final LocatedBlock updated = callUpdateBlockForPipeline(bg);
final long newGS = updated.getBlock().getGenerationStamp();
final LocatedBlock[] updatedBlks = StripedBlockUtil
.parseStripedBlockGroup((LocatedStripedBlock) updated,
BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) {
.parseStripedBlockGroup((LocatedStripedBlock) updated, cellSize,
schema.getNumDataUnits(), schema.getNumParityUnits());
for (int i = 0; i < schema.getNumDataUnits()
+ schema.getNumParityUnits(); i++) {
StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
if (si.isFailed()) {
continue; // skipping failed data streamer
@ -280,7 +280,7 @@ public class StripedDataStreamer extends DataStreamer {
final ExtendedBlock bg = coordinator.getBlockGroup();
final ExtendedBlock newBG = newBlock(bg, newGS);
final int n = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
final int n = schema.getNumDataUnits() + schema.getNumParityUnits();
final DatanodeInfo[] newNodes = new DatanodeInfo[n];
final String[] newStorageIDs = new String[n];
for (int i = 0; i < n; i++) {

View File

@ -192,6 +192,7 @@ public class PBHelper {
StripedBlockWithLocations sblk = (StripedBlockWithLocations) blk;
builder.setIndices(PBHelperClient.getByteString(sblk.getIndices()));
builder.setDataBlockNum(sblk.getDataBlockNum());
builder.setCellSize(sblk.getCellSize());
}
return builder.build();
}
@ -207,7 +208,7 @@ public class PBHelper {
PBHelperClient.convertStorageTypes(storageTypes, storageUuids.size()));
if (b.hasIndices()) {
blk = new StripedBlockWithLocations(blk, b.getIndices().toByteArray(),
(short) b.getDataBlockNum());
(short) b.getDataBlockNum(), b.getCellSize());
}
return blk;
}

View File

@ -415,11 +415,14 @@ public class Dispatcher {
final byte[] indices;
final short dataBlockNum;
final int cellSize;
public DBlockStriped(Block block, byte[] indices, short dataBlockNum) {
public DBlockStriped(Block block, byte[] indices, short dataBlockNum,
int cellSize) {
super(block);
this.indices = indices;
this.dataBlockNum = dataBlockNum;
this.cellSize = cellSize;
}
public DBlock getInternalBlock(StorageGroup storage) {
@ -429,8 +432,8 @@ public class Dispatcher {
}
byte idxInGroup = indices[idxInLocs];
long blkId = getBlock().getBlockId() + idxInGroup;
long numBytes = getInternalBlockLength(getNumBytes(),
HdfsConstants.BLOCK_STRIPED_CELL_SIZE, dataBlockNum, idxInGroup);
long numBytes = getInternalBlockLength(getNumBytes(), cellSize,
dataBlockNum, idxInGroup);
Block blk = new Block(getBlock());
blk.setBlockId(blkId);
blk.setNumBytes(numBytes);
@ -717,8 +720,8 @@ public class Dispatcher {
bytesReceived += sblkLocs.getBlock().getNumBytes() /
sblkLocs.getDataBlockNum();
block = new DBlockStriped(sblkLocs.getBlock(), sblkLocs.getIndices(),
sblkLocs.getDataBlockNum());
} else{
sblkLocs.getDataBlockNum(), sblkLocs.getCellSize());
} else {
bytesReceived += blkLocs.getBlock().getNumBytes();
block = new DBlock(blkLocs.getBlock());
}

View File

@ -22,8 +22,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
/**
* Subclass of {@link BlockInfo}, presenting a block group in erasure coding.
*
@ -65,6 +63,10 @@ public class BlockInfoStriped extends BlockInfo {
return (short) ecPolicy.getNumParityUnits();
}
public int getCellSize() {
return ecPolicy.getCellSize();
}
/**
* If the block is committed/completed and its length is less than a full
* stripe, it returns the the number of actual data blocks.
@ -73,7 +75,7 @@ public class BlockInfoStriped extends BlockInfo {
public short getRealDataBlockNum() {
if (isComplete() || getBlockUCState() == BlockUCState.COMMITTED) {
return (short) Math.min(getDataBlockNum(),
(getNumBytes() - 1) / BLOCK_STRIPED_CELL_SIZE + 1);
(getNumBytes() - 1) / ecPolicy.getCellSize() + 1);
} else {
return getDataBlockNum();
}
@ -200,7 +202,7 @@ public class BlockInfoStriped extends BlockInfo {
// `getNumBytes` is the total of actual data block size.
return StripedBlockUtil.spaceConsumedByStripedBlock(getNumBytes(),
ecPolicy.getNumDataUnits(), ecPolicy.getNumParityUnits(),
BLOCK_STRIPED_CELL_SIZE);
ecPolicy.getCellSize());
}
@Override

View File

@ -92,7 +92,6 @@ import org.apache.hadoop.hdfs.util.LightWeightHashSet;
import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
import org.apache.hadoop.metrics2.util.MBeans;
@ -2554,10 +2553,9 @@ public class BlockManager implements BlockStatsMXBean {
BlockIdManager.convertToStripedID(reported.getBlockId());
BlockInfoStriped stripedBlock = (BlockInfoStriped) storedBlock;
int reportedBlkIdx = BlockIdManager.getBlockIndex(reported);
wrongSize = reported.getNumBytes() !=
getInternalBlockLength(stripedBlock.getNumBytes(),
BLOCK_STRIPED_CELL_SIZE,
stripedBlock.getDataBlockNum(), reportedBlkIdx);
wrongSize = reported.getNumBytes() != getInternalBlockLength(
stripedBlock.getNumBytes(), stripedBlock.getCellSize(),
stripedBlock.getDataBlockNum(), reportedBlkIdx);
} else {
wrongSize = storedBlock.getNumBytes() != reported.getNumBytes();
}
@ -3413,7 +3411,7 @@ public class BlockManager implements BlockStatsMXBean {
(byte) blockStriped.getStorageBlockIndex(locations.get(i));
}
results.add(new StripedBlockWithLocations(blkWithLocs, indices,
blockStriped.getDataBlockNum()));
blockStriped.getDataBlockNum(), blockStriped.getCellSize()));
// approximate size
return block.getNumBytes() / blockStriped.getDataBlockNum();
}else{

View File

@ -185,7 +185,8 @@ public class Mover {
for (int i = 0; i < indices.length; i++) {
indices[i] = (byte) lsb.getBlockIndices()[i];
}
db = new DBlockStriped(blk, indices, (short) ecPolicy.getNumDataUnits());
db = new DBlockStriped(blk, indices, (short) ecPolicy.getNumDataUnits(),
ecPolicy.getCellSize());
} else {
db = new DBlock(blk);
}

View File

@ -204,9 +204,14 @@ class FSDirWriteFileOp {
clientMachine = pendingFile.getFileUnderConstructionFeature()
.getClientMachine();
isStriped = pendingFile.isStriped();
numTargets = isStriped ?
HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS :
pendingFile.getFileReplication();
ErasureCodingPolicy ecPolicy = null;
if (isStriped) {
ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(fsn, src);
numTargets = (short) (ecPolicy.getSchema().getNumDataUnits()
+ ecPolicy.getSchema().getNumParityUnits());
} else {
numTargets = pendingFile.getFileReplication();
}
storagePolicyID = pendingFile.getStoragePolicyID();
return new ValidateAddBlockResult(blockSize, numTargets, storagePolicyID,
clientMachine, isStriped);

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
@ -388,8 +388,13 @@ public class INodeFile extends INodeWithAdditionalFields
}
max = maxInSnapshot > max ? maxInSnapshot : max;
}
return isStriped() ?
HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS : max;
if(!isStriped()){
return max;
}
// TODO support more policies based on policyId
ErasureCodingPolicy ecPolicy =
ErasureCodingPolicyManager.getSystemDefaultPolicy();
return (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits());
}
/** Set the replication factor of this file. */

View File

@ -95,16 +95,17 @@ public class BlocksWithLocations {
public static class StripedBlockWithLocations extends BlockWithLocations {
final byte[] indices;
final short dataBlockNum;
final int cellSize;
public StripedBlockWithLocations(BlockWithLocations blk, byte[] indices,
short dataBlockNum) {
short dataBlockNum, int cellSize) {
super(blk.getBlock(), blk.getDatanodeUuids(), blk.getStorageIDs(),
blk.getStorageTypes());
Preconditions.checkArgument(
blk.getDatanodeUuids().length == indices.length);
this.indices = indices;
this.dataBlockNum = dataBlockNum;
this.cellSize = cellSize;
}
public byte[] getIndices() {
@ -114,6 +115,10 @@ public class BlocksWithLocations {
public short getDataBlockNum() {
return dataBlockNum;
}
public int getCellSize() {
return cellSize;
}
}
private final BlockWithLocations[] blocks;

View File

@ -166,8 +166,8 @@ import org.junit.Assume;
import org.mockito.internal.util.reflection.Whitebox;
import com.google.common.annotations.VisibleForTesting;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS;
/** Utilities for HDFS tests */
public class DFSTestUtil {

View File

@ -25,7 +25,6 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.web.ByteRangeInputStream;
@ -38,18 +37,19 @@ import java.util.Random;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
public class StripedFileTestUtil {
public static final Log LOG = LogFactory.getLog(StripedFileTestUtil.class);
/*
* These values correspond to the values used by the system default erasure
* coding policy.
*/
public static final short NUM_DATA_BLOCKS = (short) 6;
public static final short NUM_PARITY_BLOCKS = (short) 3;
public static final int BLOCK_STRIPED_CELL_SIZE = 64 * 1024;
static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
static final int stripesPerBlock = 4;
static final int blockSize = cellSize * stripesPerBlock;
static final int numDNs = dataBlocks + parityBlocks + 2;
static final int blockSize = BLOCK_STRIPED_CELL_SIZE * stripesPerBlock;
static final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS + 2;
static final Random random = new Random();
@ -85,9 +85,9 @@ public class StripedFileTestUtil {
static void verifyPread(FileSystem fs, Path srcPath, int fileLength,
byte[] expected, byte[] buf) throws IOException {
try (FSDataInputStream in = fs.open(srcPath)) {
int[] startOffsets = {0, 1, cellSize - 102, cellSize, cellSize + 102,
cellSize * (dataBlocks - 1), cellSize * (dataBlocks - 1) + 102,
cellSize * dataBlocks, fileLength - 102, fileLength - 1};
int[] startOffsets = {0, 1, BLOCK_STRIPED_CELL_SIZE - 102, BLOCK_STRIPED_CELL_SIZE, BLOCK_STRIPED_CELL_SIZE + 102,
BLOCK_STRIPED_CELL_SIZE * (NUM_DATA_BLOCKS - 1), BLOCK_STRIPED_CELL_SIZE * (NUM_DATA_BLOCKS - 1) + 102,
BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS, fileLength - 102, fileLength - 1};
for (int startOffset : startOffsets) {
startOffset = Math.max(0, Math.min(startOffset, fileLength - 1));
int remaining = fileLength - startOffset;
@ -158,21 +158,21 @@ public class StripedFileTestUtil {
pos = 0;
assertSeekAndRead(in, pos, fileLength);
if (fileLength > cellSize) {
if (fileLength > BLOCK_STRIPED_CELL_SIZE) {
// seek to cellSize boundary
pos = cellSize - 1;
pos = BLOCK_STRIPED_CELL_SIZE - 1;
assertSeekAndRead(in, pos, fileLength);
}
if (fileLength > cellSize * dataBlocks) {
if (fileLength > BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS) {
// seek to striped cell group boundary
pos = cellSize * dataBlocks - 1;
pos = BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS - 1;
assertSeekAndRead(in, pos, fileLength);
}
if (fileLength > blockSize * dataBlocks) {
if (fileLength > blockSize * NUM_DATA_BLOCKS) {
// seek to striped block group boundary
pos = blockSize * dataBlocks - 1;
pos = blockSize * NUM_DATA_BLOCKS - 1;
assertSeekAndRead(in, pos, fileLength);
}
@ -235,12 +235,12 @@ public class StripedFileTestUtil {
* number of actual data internal blocks. Otherwise returns NUM_DATA_BLOCKS.
*/
public static short getRealDataBlockNum(int numBytes) {
return (short) Math.min(dataBlocks,
return (short) Math.min(NUM_DATA_BLOCKS,
(numBytes - 1) / BLOCK_STRIPED_CELL_SIZE + 1);
}
public static short getRealTotalBlockNum(int numBytes) {
return (short) (getRealDataBlockNum(numBytes) + parityBlocks);
return (short) (getRealDataBlockNum(numBytes) + NUM_PARITY_BLOCKS);
}
/**

View File

@ -24,7 +24,6 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
@ -60,9 +59,9 @@ public class TestDFSStripedInputStream {
private final Path dirPath = new Path("/striped");
private Path filePath = new Path(dirPath, "file");
private final ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS;
private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS;
private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
private final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS;
private final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS;
private final int CELLSIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
private final int NUM_STRIPE_PER_BLOCK = 2;
private final int INTERNAL_BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE;
private final int BLOCK_GROUP_SIZE = DATA_BLK_NUM * INTERNAL_BLOCK_SIZE;

View File

@ -28,7 +28,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
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;
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
@ -51,13 +50,13 @@ public class TestDFSStripedOutputStream {
GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL);
}
private int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
private int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
private int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
private int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
private MiniDFSCluster cluster;
private DistributedFileSystem fs;
private Configuration conf;
private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
private final int stripesPerBlock = 4;
private final int blockSize = cellSize * stripesPerBlock;

View File

@ -65,9 +65,9 @@ public class TestDFSStripedOutputStreamWithFailure {
GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL);
}
private static final int NUM_DATA_BLOCKS = HdfsConstants.NUM_DATA_BLOCKS;
private static final int NUM_PARITY_BLOCKS = HdfsConstants.NUM_PARITY_BLOCKS;
private static final int CELL_SIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
private static final int NUM_DATA_BLOCKS = StripedFileTestUtil.NUM_DATA_BLOCKS;
private static final int NUM_PARITY_BLOCKS = StripedFileTestUtil.NUM_PARITY_BLOCKS;
private static final int CELL_SIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
private static final int STRIPES_PER_BLOCK = 4;
private static final int BLOCK_SIZE = CELL_SIZE * STRIPES_PER_BLOCK;
private static final int BLOCK_GROUP_SIZE = BLOCK_SIZE * NUM_DATA_BLOCKS;
@ -255,7 +255,7 @@ public class TestDFSStripedOutputStreamWithFailure {
DatanodeReportType.LIVE);
assertEquals("Mismatches number of live Dns ", numDatanodes, info.length);
Path srcPath = new Path(dir, "testAddBlockWhenNoSufficientParityNodes");
int fileLength = HdfsConstants.BLOCK_STRIPED_CELL_SIZE - 1000;
int fileLength = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE - 1000;
final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
DFSTestUtil.writeFile(dfs, srcPath, new String(expected));
StripedFileTestUtil.verifySeek(dfs, srcPath, fileLength);

View File

@ -46,16 +46,16 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.parityBlocks;
public class TestReadStripedFileWithDecoding {
static final Log LOG = LogFactory.getLog(TestReadStripedFileWithDecoding.class);
private MiniDFSCluster cluster;
private DistributedFileSystem fs;
private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
private final short parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
private final int smallFileLength = blockSize * dataBlocks - 123;
private final int largeFileLength = blockSize * dataBlocks + 123;
private final int[] fileLengths = {smallFileLength, largeFileLength};

View File

@ -32,8 +32,6 @@ import org.junit.Test;
import java.io.IOException;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
/**
@ -46,6 +44,8 @@ public class TestReadStripedFileWithMissingBlocks {
private static MiniDFSCluster cluster;
private static DistributedFileSystem fs;
private static Configuration conf = new HdfsConfiguration();
private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
private final int fileLength = blockSize * dataBlocks + 123;
@Before
@ -112,7 +112,7 @@ public class TestReadStripedFileWithMissingBlocks {
}
for (int i = 0; i < missingParityNum; i++) {
missingDataNodes[i + missingDataNum] = i +
Math.min(StripedFileTestUtil.dataBlocks, dataBlocks);
Math.min(StripedFileTestUtil.NUM_DATA_BLOCKS, dataBlocks);
}
stopDataNodes(locs, missingDataNodes);

View File

@ -58,9 +58,9 @@ import org.junit.Test;
public class TestRecoverStripedFile {
public static final Log LOG = LogFactory.getLog(TestRecoverStripedFile.class);
private static final int dataBlkNum = HdfsConstants.NUM_DATA_BLOCKS;
private static final int parityBlkNum = HdfsConstants.NUM_PARITY_BLOCKS;
private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
private static final int dataBlkNum = StripedFileTestUtil.NUM_DATA_BLOCKS;
private static final int parityBlkNum = StripedFileTestUtil.NUM_PARITY_BLOCKS;
private static final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
private static final int blockSize = cellSize * 3;
private static final int groupSize = dataBlkNum + parityBlkNum;
private static final int dnNum = groupSize + parityBlkNum;

View File

@ -40,10 +40,10 @@ import static org.junit.Assert.assertTrue;
public class TestSafeModeWithStripedFile {
static final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS;
static final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS;
static final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS;
static final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS;
static final int numDNs = DATA_BLK_NUM + PARITY_BLK_NUM;
static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
static final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
static final int blockSize = cellSize * 2;
static MiniDFSCluster cluster;

View File

@ -37,8 +37,6 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.stripesPerBlock;
@ -46,6 +44,8 @@ public class TestWriteReadStripedFile {
public static final Log LOG = LogFactory.getLog(TestWriteReadStripedFile.class);
private static MiniDFSCluster cluster;
private static DistributedFileSystem fs;
private static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
private static short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
private static Configuration conf = new HdfsConfiguration();
static {

View File

@ -31,9 +31,7 @@ import java.io.IOException;
import java.util.concurrent.atomic.AtomicInteger;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.parityBlocks;
public class TestWriteStripedFileWithFailure {
public static final Log LOG = LogFactory
@ -41,6 +39,8 @@ public class TestWriteStripedFileWithFailure {
private static MiniDFSCluster cluster;
private static FileSystem fs;
private static Configuration conf = new HdfsConfiguration();
private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
private final short parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
private final int smallFileLength = blockSize * dataBlocks - 123;
private final int largeFileLength = blockSize * dataBlocks + 123;
private final int[] fileLengths = {smallFileLength, largeFileLength};

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.StripedFileTestUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -201,7 +202,8 @@ public class TestPBHelper {
BlockWithLocations blkLocs = new BlockWithLocations(new Block(bid, 0, 1),
datanodeUuids, storageIDs, storageTypes);
if (isStriped) {
blkLocs = new StripedBlockWithLocations(blkLocs, indices, dataBlkNum);
blkLocs = new StripedBlockWithLocations(blkLocs, indices, dataBlkNum,
StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE);
}
return blkLocs;
}

View File

@ -67,6 +67,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.StripedFileTestUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -149,10 +150,10 @@ public class TestBalancer {
conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L);
}
int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
int groupSize = dataBlocks + parityBlocks;
private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
private final static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
private final static int stripesPerBlock = 4;
static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock;

View File

@ -31,8 +31,8 @@ import java.io.DataOutputStream;
import java.io.ByteArrayOutputStream;
import java.nio.ByteBuffer;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_PARITY_BLOCKS;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.StripedFileTestUtil;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
@ -31,9 +31,9 @@ import java.io.IOException;
public class TestBlockTokenWithDFSStriped extends TestBlockTokenWithDFS {
private final static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
private final static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
private final static int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
private final static int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
private final static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
private final static int stripesPerBlock = 4;
private final static int numDNs = dataBlocks + parityBlocks + 2;
private static MiniDFSCluster cluster;

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.StripedFileTestUtil;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.test.GenericTestUtils;
@ -58,9 +58,9 @@ public class TestSequentialBlockGroupId {
private final short REPLICATION = 1;
private final long SEED = 0;
private final int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
private final int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
private final int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
private final int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
private final int stripesPerBlock = 2;
private final int blockSize = cellSize * stripesPerBlock;

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.StripedFileTestUtil;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@ -415,9 +416,9 @@ public class TestMover {
}
}
int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
private final static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
private final static int stripesPerBlock = 4;
static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock;

View File

@ -23,6 +23,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.StripedFileTestUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@ -49,10 +50,10 @@ public class TestAddOverReplicatedStripedBlocks {
private DistributedFileSystem fs;
private final Path dirPath = new Path("/striped");
private Path filePath = new Path(dirPath, "file");
private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS;
private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS;
private final short GROUP_SIZE = DATA_BLK_NUM + PARITY_BLK_NUM;
private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
private final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS;
private final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS;
private final short GROUP_SIZE = (short) (DATA_BLK_NUM + PARITY_BLK_NUM);
private final int CELLSIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
private final int NUM_STRIPE_PER_BLOCK = 4;
private final int BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE;
private final int numDNs = GROUP_SIZE + 3;

View File

@ -24,6 +24,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.StripedFileTestUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
@ -57,13 +58,13 @@ import java.util.List;
import java.util.UUID;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS;
import static org.junit.Assert.assertEquals;
public class TestAddStripedBlocks {
private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS +
HdfsConstants.NUM_PARITY_BLOCKS;
private final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS +
StripedFileTestUtil.NUM_PARITY_BLOCKS);
private MiniDFSCluster cluster;
private DistributedFileSystem dfs;
@ -163,8 +164,8 @@ public class TestAddStripedBlocks {
boolean checkReplica) {
assertEquals(0, block.numNodes());
Assert.assertFalse(block.isComplete());
Assert.assertEquals(HdfsConstants.NUM_DATA_BLOCKS, block.getDataBlockNum());
Assert.assertEquals(HdfsConstants.NUM_PARITY_BLOCKS,
Assert.assertEquals(StripedFileTestUtil.NUM_DATA_BLOCKS, block.getDataBlockNum());
Assert.assertEquals(StripedFileTestUtil.NUM_PARITY_BLOCKS,
block.getParityBlockNum());
Assert.assertEquals(0,
block.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK);

View File

@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.StripedFileTestUtil;
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@ -450,8 +451,8 @@ public class TestFSEditLogLoader {
long blkId = 1;
long blkNumBytes = 1024;
long timestamp = 1426222918;
short blockNum = HdfsConstants.NUM_DATA_BLOCKS;
short parityNum = HdfsConstants.NUM_PARITY_BLOCKS;
short blockNum = StripedFileTestUtil.NUM_DATA_BLOCKS;
short parityNum = StripedFileTestUtil.NUM_PARITY_BLOCKS;
//set the storage policy of the directory
fs.mkdir(new Path(testDir), new FsPermission("755"));
@ -522,8 +523,8 @@ public class TestFSEditLogLoader {
long blkId = 1;
long blkNumBytes = 1024;
long timestamp = 1426222918;
short blockNum = HdfsConstants.NUM_DATA_BLOCKS;
short parityNum = HdfsConstants.NUM_PARITY_BLOCKS;
short blockNum = StripedFileTestUtil.NUM_DATA_BLOCKS;
short parityNum = StripedFileTestUtil.NUM_PARITY_BLOCKS;
//set the storage policy of the directory
fs.mkdir(new Path(testDir), new FsPermission("755"));

View File

@ -52,6 +52,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.StripedFileTestUtil;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
@ -414,8 +415,8 @@ public class TestFSImage {
*/
@Test
public void testSupportBlockGroup() throws IOException {
final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS +
HdfsConstants.NUM_PARITY_BLOCKS;
final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS
+ StripedFileTestUtil.NUM_PARITY_BLOCKS);
final int BLOCK_SIZE = 8 * 1024 * 1024;
Configuration conf = new HdfsConfiguration();
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
@ -447,8 +448,8 @@ public class TestFSImage {
BlockInfo[] blks = inode.getBlocks();
assertEquals(1, blks.length);
assertTrue(blks[0].isStriped());
assertEquals(HdfsConstants.NUM_DATA_BLOCKS, ((BlockInfoStriped)blks[0]).getDataBlockNum());
assertEquals(HdfsConstants.NUM_PARITY_BLOCKS, ((BlockInfoStriped)blks[0]).getParityBlockNum());
assertEquals(StripedFileTestUtil.NUM_DATA_BLOCKS, ((BlockInfoStriped)blks[0]).getDataBlockNum());
assertEquals(StripedFileTestUtil.NUM_PARITY_BLOCKS, ((BlockInfoStriped)blks[0]).getParityBlockNum());
} finally {
cluster.shutdown();
}

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.StripedFileTestUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -113,9 +114,9 @@ public class TestQuotaWithStripedBlocks {
final long actualDiskUsed = dirNode.getDirectoryWithQuotaFeature()
.getSpaceConsumed().getTypeSpaces().get(StorageType.DISK);
// In this case the file's real size is cell size * block group size.
Assert.assertEquals(HdfsConstants.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
Assert.assertEquals(StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
actualSpaceUsed);
Assert.assertEquals(HdfsConstants.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
Assert.assertEquals(StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
actualDiskUsed);
} finally {
IOUtils.cleanup(null, out);

View File

@ -35,16 +35,16 @@ import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECReco
import org.junit.Test;
import java.util.List;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS;
import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_PARITY_BLOCKS;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
public class TestRecoverStripedBlocks {
private final short GROUP_SIZE =
NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
(short) (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS);
private MiniDFSCluster cluster;
private final Path dirPath = new Path("/dir");
private Path filePath = new Path(dirPath, "file");

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.StripedFileTestUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
@ -222,8 +223,8 @@ public class TestStripedINodeFile {
final Path contiguousFile = new Path(parentDir, "someFile");
final DistributedFileSystem dfs;
final Configuration conf = new Configuration();
final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS
+ HdfsConstants.NUM_PARITY_BLOCKS;
final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS
+ StripedFileTestUtil.NUM_PARITY_BLOCKS);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY, 2);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE)

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.StripedFileTestUtil;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
@ -44,12 +44,12 @@ import org.junit.BeforeClass;
import org.junit.Test;
public class TestOfflineImageViewerWithStripedBlocks {
private static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
private static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
private static int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
private static int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
private static MiniDFSCluster cluster;
private static DistributedFileSystem fs;
private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
private static final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
private static final int stripesPerBlock = 3;
private static final int blockSize = cellSize * stripesPerBlock;

View File

@ -21,9 +21,9 @@ package org.apache.hadoop.hdfs.util;
import com.google.common.base.Preconditions;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.StripedFileTestUtil;
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.LocatedStripedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
@ -77,10 +77,10 @@ import static org.junit.Assert.assertFalse;
* TODO: test parity block logic
*/
public class TestStripedBlockUtil {
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_WIDTH = DATA_BLK_NUM + PARITY_BLK_NUM;
private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
private final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS;
private final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS;
private final short BLK_GROUP_WIDTH = (short) (DATA_BLK_NUM + PARITY_BLK_NUM);
private final int CELLSIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
private final int FULL_STRIPE_SIZE = DATA_BLK_NUM * CELLSIZE;
/** number of full stripes in a full block group */
private final int BLK_GROUP_STRIPE_NUM = 16;