HDFS-12818. Support multiple storages in DataNodeCluster / SimulatedFSDataset. Contributed by Erik Krogen.

This commit is contained in:
Erik Krogen 2017-12-18 11:36:22 -08:00 committed by Konstantin V Shvachko
parent 001008958d
commit 94576b17fb
3 changed files with 351 additions and 152 deletions

View File

@ -23,8 +23,8 @@
import java.io.OutputStream;
import java.net.URI;
import java.nio.channels.ClosedChannelException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
@ -37,11 +37,13 @@
import javax.management.ObjectName;
import javax.management.StandardMBean;
import com.google.common.math.LongMath;
import org.apache.commons.lang.ArrayUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.DF;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImplTestUtils;
import org.apache.hadoop.util.AutoCloseableLock;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.Block;
@ -88,6 +90,7 @@
*/
public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
public final static int BYTE_MASK = 0xff;
private final static int DEFAULT_NUM_SIMULATED_DATA_DIRS = 1;
static class Factory extends FsDatasetSpi.Factory<SimulatedFSDataset> {
@Override
public SimulatedFSDataset newInstance(DataNode datanode,
@ -100,10 +103,42 @@ public boolean isSimulated() {
return true;
}
}
/**
* Used to change the default number of data storages and to mark the
* FSDataset as simulated.
*/
static class TestUtilsFactory
extends FsDatasetTestUtils.Factory<FsDatasetTestUtils> {
@Override
public FsDatasetTestUtils newInstance(DataNode datanode) {
return new FsDatasetImplTestUtils(datanode) {
@Override
public int getDefaultNumOfDataDirs() {
return DEFAULT_NUM_SIMULATED_DATA_DIRS;
}
};
}
@Override
public boolean isSimulated() {
return true;
}
@Override
public int getDefaultNumOfDataDirs() {
return DEFAULT_NUM_SIMULATED_DATA_DIRS;
}
}
public static void setFactory(Configuration conf) {
conf.set(DFSConfigKeys.DFS_DATANODE_FSDATASET_FACTORY_KEY,
Factory.class.getName());
conf.setClass("org.apache.hadoop.hdfs.server.datanode." +
"SimulatedFSDatasetTestUtilsFactory",
TestUtilsFactory.class, FsDatasetTestUtils.Factory.class
);
}
public static byte simulatedByte(Block b, long offsetInBlk) {
@ -151,7 +186,7 @@ private class BInfo implements ReplicaInPipeline {
if (theBlock.getNumBytes() < 0) {
theBlock.setNumBytes(0);
}
if (!storage.alloc(bpid, theBlock.getNumBytes())) {
if (!getStorage(theBlock).alloc(bpid, theBlock.getNumBytes())) {
// expected length - actual length may
// be more - we find out at finalize
DataNode.LOG.warn("Lack of free storage on a block alloc");
@ -169,7 +204,7 @@ private class BInfo implements ReplicaInPipeline {
@Override
public String getStorageUuid() {
return storage.getStorageUuid();
return getStorage(theBlock).getStorageUuid();
}
@Override
@ -226,12 +261,12 @@ synchronized void finalizeBlock(String bpid, long finalSize)
// adjust if necessary
long extraLen = finalSize - theBlock.getNumBytes();
if (extraLen > 0) {
if (!storage.alloc(bpid,extraLen)) {
if (!getStorage(theBlock).alloc(bpid, extraLen)) {
DataNode.LOG.warn("Lack of free storage on a block alloc");
throw new IOException("Creating block, no free space available");
}
} else {
storage.free(bpid, -extraLen);
getStorage(theBlock).free(bpid, -extraLen);
}
theBlock.setNumBytes(finalSize);
@ -271,7 +306,7 @@ synchronized public ReplicaOutputStreams createStreams(boolean isCreate,
} else {
SimulatedOutputStream crcStream = new SimulatedOutputStream();
return new ReplicaOutputStreams(oStream, crcStream, requestedChecksum,
volume, fileIoProvider);
getStorage(theBlock).getVolume(), fileIoProvider);
}
}
@ -368,6 +403,7 @@ public void stopWriter(long xceiverStopTimeout) throws IOException {
*/
private static class SimulatedBPStorage {
private long used; // in bytes
private final Map<Block, BInfo> blockMap = new TreeMap<>();
long getUsed() {
return used;
@ -381,6 +417,10 @@ void free(long amount) {
used -= amount;
}
Map<Block, BInfo> getBlockMap() {
return blockMap;
}
SimulatedBPStorage() {
used = 0;
}
@ -392,10 +432,11 @@ void free(long amount) {
*/
private static class SimulatedStorage {
private final Map<String, SimulatedBPStorage> map =
new HashMap<String, SimulatedBPStorage>();
new ConcurrentHashMap<>();
private final long capacity; // in bytes
private final DatanodeStorage dnStorage;
private final SimulatedVolume volume;
synchronized long getFree() {
return capacity - getUsed();
@ -433,11 +474,15 @@ synchronized void free(String bpid, long amount) throws IOException {
getBPStorage(bpid).free(amount);
}
SimulatedStorage(long cap, DatanodeStorage.State state) {
SimulatedStorage(long cap, DatanodeStorage.State state,
FileIoProvider fileIoProvider, Configuration conf) {
capacity = cap;
dnStorage = new DatanodeStorage(
"SimulatedStorage-" + DatanodeStorage.generateUuid(),
state, StorageType.DEFAULT);
DataNodeVolumeMetrics volumeMetrics =
DataNodeVolumeMetrics.create(conf, dnStorage.getStorageID());
this.volume = new SimulatedVolume(this, fileIoProvider, volumeMetrics);
}
synchronized void addBlockPool(String bpid) {
@ -473,6 +518,18 @@ synchronized StorageReport getStorageReport(String bpid) {
false, getCapacity(), getUsed(), getFree(),
map.get(bpid).getUsed(), 0L);
}
SimulatedVolume getVolume() {
return volume;
}
Map<Block, BInfo> getBlockMap(String bpid) throws IOException {
SimulatedBPStorage bpStorage = map.get(bpid);
if (bpStorage == null) {
throw new IOException("Nonexistent block pool: " + bpid);
}
return bpStorage.getBlockMap();
}
}
static class SimulatedVolume implements FsVolumeSpi {
@ -601,10 +658,7 @@ public VolumeCheckResult check(VolumeCheckContext context)
}
}
private final Map<String, Map<Block, BInfo>> blockMap
= new ConcurrentHashMap<String, Map<Block,BInfo>>();
private final SimulatedStorage storage;
private final SimulatedVolume volume;
private final List<SimulatedStorage> storages;
private final String datanodeUuid;
private final DataNode datanode;
@ -615,27 +669,30 @@ public SimulatedFSDataset(DataStorage storage, Configuration conf) {
public SimulatedFSDataset(DataNode datanode, DataStorage storage, Configuration conf) {
this.datanode = datanode;
if (storage != null) {
int storageCount;
if (storage != null && storage.getNumStorageDirs() > 0) {
storageCount = storage.getNumStorageDirs();
for (int i = 0; i < storage.getNumStorageDirs(); ++i) {
DataStorage.createStorageID(storage.getStorageDir(i), false, conf);
}
this.datanodeUuid = storage.getDatanodeUuid();
} else {
storageCount = DataNode.getStorageLocations(conf).size();
this.datanodeUuid = "SimulatedDatanode-" + DataNode.generateUuid();
}
registerMBean(datanodeUuid);
this.fileIoProvider = new FileIoProvider(conf, datanode);
this.storage = new SimulatedStorage(
conf.getLong(CONFIG_PROPERTY_CAPACITY, DEFAULT_CAPACITY),
conf.getEnum(CONFIG_PROPERTY_STATE, DEFAULT_STATE));
// TODO: per volume id or path
DataNodeVolumeMetrics volumeMetrics = DataNodeVolumeMetrics.create(conf,
datanodeUuid);
this.volume = new SimulatedVolume(this.storage, this.fileIoProvider,
volumeMetrics);
this.datasetLock = new AutoCloseableLock();
this.storages = new ArrayList<>();
for (int i = 0; i < storageCount; i++) {
this.storages.add(new SimulatedStorage(
conf.getLong(CONFIG_PROPERTY_CAPACITY, DEFAULT_CAPACITY),
conf.getEnum(CONFIG_PROPERTY_STATE, DEFAULT_STATE),
fileIoProvider, conf));
}
}
public synchronized void injectBlocks(String bpid,
@ -651,33 +708,50 @@ public synchronized void injectBlocks(String bpid,
throw new IOException("Block already exists in block list");
}
}
Map<Block, BInfo> map = blockMap.get(bpid);
if (map == null) {
map = new TreeMap<>();
blockMap.put(bpid, map);
for (SimulatedStorage storage : storages) {
storage.addBlockPool(bpid);
}
for (Block b: injectBlocks) {
BInfo binfo = new BInfo(bpid, b, false);
map.put(binfo.theBlock, binfo);
getBlockMap(b, bpid).put(binfo.theBlock, binfo);
}
}
}
/** Get the storage that a given block lives within. */
private SimulatedStorage getStorage(Block b) {
return storages.get(LongMath.mod(b.getBlockId(), storages.size()));
}
/** Get a map for a given block pool Id */
private Map<Block, BInfo> getMap(String bpid) throws IOException {
final Map<Block, BInfo> map = blockMap.get(bpid);
if (map == null) {
throw new IOException("Non existent blockpool " + bpid);
}
return map;
/**
* Get the block map that a given block lives within, assuming it is within
* block pool bpid.
* @param b The block to look for
* @param bpid The block pool that contains b
* @return The block map (non-null)
* @throws IOException if bpid does not exist
*/
private Map<Block, BInfo> getBlockMap(Block b, String bpid)
throws IOException {
return getStorage(b).getBlockMap(bpid);
}
/**
* Get the block map that a given block lives within.
* @param b The extended block to look for
* @return The block map (non-null)
* @throws IOException if b is in a nonexistent block pool
*/
private Map<Block, BInfo> getBlockMap(ExtendedBlock b) throws IOException {
return getBlockMap(b.getLocalBlock(), b.getBlockPoolId());
}
@Override // FsDatasetSpi
public synchronized void finalizeBlock(ExtendedBlock b, boolean fsyncDir)
throws IOException {
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
BInfo binfo = map.get(b.getLocalBlock());
BInfo binfo = getBlockMap(b).get(b.getLocalBlock());
if (binfo == null) {
throw new IOException("Finalizing a non existing block " + b);
}
@ -687,20 +761,21 @@ public synchronized void finalizeBlock(ExtendedBlock b, boolean fsyncDir)
@Override // FsDatasetSpi
public synchronized void unfinalizeBlock(ExtendedBlock b) throws IOException{
if (isValidRbw(b)) {
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
map.remove(b.getLocalBlock());
getBlockMap(b).remove(b.getLocalBlock());
}
}
synchronized BlockListAsLongs getBlockReport(String bpid) {
synchronized BlockListAsLongs getBlockReport(String bpid,
SimulatedStorage storage) {
BlockListAsLongs.Builder report = BlockListAsLongs.builder();
final Map<Block, BInfo> map = blockMap.get(bpid);
if (map != null) {
for (BInfo b : map.values()) {
try {
for (BInfo b : storage.getBlockMap(bpid).values()) {
if (b.isFinalized()) {
report.add(b);
}
}
} catch (IOException ioe) {
DataNode.LOG.error("Exception while getting block reports", ioe);
}
return report.build();
}
@ -708,7 +783,11 @@ synchronized BlockListAsLongs getBlockReport(String bpid) {
@Override
public synchronized Map<DatanodeStorage, BlockListAsLongs> getBlockReports(
String bpid) {
return Collections.singletonMap(storage.getDnStorage(), getBlockReport(bpid));
Map<DatanodeStorage, BlockListAsLongs> blockReports = new HashMap<>();
for (SimulatedStorage storage : storages) {
blockReports.put(storage.getDnStorage(), getBlockReport(bpid, storage));
}
return blockReports;
}
@Override // FsDatasetSpi
@ -718,27 +797,49 @@ public List<Long> getCacheReport(String bpid) {
@Override // FSDatasetMBean
public long getCapacity() {
return storage.getCapacity();
long total = 0;
for (SimulatedStorage storage : storages) {
total += storage.getCapacity();
}
return total;
}
@Override // FSDatasetMBean
public long getDfsUsed() {
return storage.getUsed();
long total = 0;
for (SimulatedStorage storage : storages) {
total += storage.getUsed();
}
return total;
}
@Override // FSDatasetMBean
public long getBlockPoolUsed(String bpid) throws IOException {
return storage.getBlockPoolUsed(bpid);
long total = 0;
for (SimulatedStorage storage : storages) {
total += storage.getBlockPoolUsed(bpid);
}
return total;
}
@Override // FSDatasetMBean
public long getRemaining() {
return storage.getFree();
long total = 0;
for (SimulatedStorage storage : storages) {
total += storage.getFree();
}
return total;
}
@Override // FSDatasetMBean
public int getNumFailedVolumes() {
return storage.getNumFailedVolumes();
int total = 0;
for (SimulatedStorage storage : storages) {
total += storage.getNumFailedVolumes();
}
return total;
}
@Override // FSDatasetMBean
@ -803,8 +904,7 @@ public void getMetrics(MetricsCollector collector, boolean all) {
@Override // FsDatasetSpi
public synchronized long getLength(ExtendedBlock b) throws IOException {
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
BInfo binfo = map.get(b.getLocalBlock());
BInfo binfo = getBlockMap(b).get(b.getLocalBlock());
if (binfo == null) {
throw new IOException("Finalizing a non existing block " + b);
}
@ -814,34 +914,38 @@ public synchronized long getLength(ExtendedBlock b) throws IOException {
@Override
@Deprecated
public Replica getReplica(String bpid, long blockId) {
final Map<Block, BInfo> map = blockMap.get(bpid);
if (map != null) {
return map.get(new Block(blockId));
Block b = new Block(blockId);
try {
return getBlockMap(b, bpid).get(b);
} catch (IOException ioe) {
return null;
}
return null;
}
@Override
public synchronized String getReplicaString(String bpid, long blockId) {
Replica r = null;
final Map<Block, BInfo> map = blockMap.get(bpid);
if (map != null) {
r = map.get(new Block(blockId));
try {
Block b = new Block(blockId);
r = getBlockMap(b, bpid).get(b);
} catch (IOException ioe) {
// Ignore
}
return r == null? "null": r.toString();
}
@Override // FsDatasetSpi
public Block getStoredBlock(String bpid, long blkid) throws IOException {
final Map<Block, BInfo> map = blockMap.get(bpid);
if (map != null) {
BInfo binfo = map.get(new Block(blkid));
Block b = new Block(blkid);
try {
BInfo binfo = getBlockMap(b, bpid).get(b);
if (binfo == null) {
return null;
}
return new Block(blkid, binfo.getGenerationStamp(), binfo.getNumBytes());
} catch (IOException ioe) {
return null;
}
return null;
}
@Override // FsDatasetSpi
@ -851,18 +955,18 @@ public synchronized void invalidate(String bpid, Block[] invalidBlks)
if (invalidBlks == null) {
return;
}
final Map<Block, BInfo> map = getMap(bpid);
for (Block b: invalidBlks) {
if (b == null) {
continue;
}
Map<Block, BInfo> map = getBlockMap(b, bpid);
BInfo binfo = map.get(b);
if (binfo == null) {
error = true;
DataNode.LOG.warn("Invalidate: Missing block");
continue;
}
storage.free(bpid, binfo.getNumBytes());
getStorage(b).free(bpid, binfo.getNumBytes());
map.remove(b);
if (datanode != null) {
datanode.notifyNamenodeDeletedBlock(new ExtendedBlock(bpid, b),
@ -892,8 +996,11 @@ public boolean isCached(String bpid, long blockId) {
}
private BInfo getBInfo(final ExtendedBlock b) {
final Map<Block, BInfo> map = blockMap.get(b.getBlockPoolId());
return map == null? null: map.get(b.getLocalBlock());
try {
return getBlockMap(b).get(b.getLocalBlock());
} catch (IOException ioe) {
return null;
}
}
@Override // {@link FsDatasetSpi}
@ -957,8 +1064,7 @@ public String toString() {
@Override // FsDatasetSpi
public synchronized ReplicaHandler append(
ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException {
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
BInfo binfo = map.get(b.getLocalBlock());
BInfo binfo = getBlockMap(b).get(b.getLocalBlock());
if (binfo == null || !binfo.isFinalized()) {
throw new ReplicaNotFoundException("Block " + b
+ " is not valid, and cannot be appended to.");
@ -970,7 +1076,7 @@ public synchronized ReplicaHandler append(
@Override // FsDatasetSpi
public synchronized ReplicaHandler recoverAppend(
ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException {
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
final Map<Block, BInfo> map = getBlockMap(b);
BInfo binfo = map.get(b.getLocalBlock());
if (binfo == null) {
throw new ReplicaNotFoundException("Block " + b
@ -988,7 +1094,7 @@ public synchronized ReplicaHandler recoverAppend(
@Override // FsDatasetSpi
public Replica recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen)
throws IOException {
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
final Map<Block, BInfo> map = getBlockMap(b);
BInfo binfo = map.get(b.getLocalBlock());
if (binfo == null) {
throw new ReplicaNotFoundException("Block " + b
@ -1007,7 +1113,7 @@ public Replica recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen)
public synchronized ReplicaHandler recoverRbw(
ExtendedBlock b, long newGS, long minBytesRcvd, long maxBytesRcvd)
throws IOException {
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
final Map<Block, BInfo> map = getBlockMap(b);
BInfo binfo = map.get(b.getLocalBlock());
if ( binfo == null) {
throw new ReplicaNotFoundException("Block " + b
@ -1042,16 +1148,14 @@ public synchronized ReplicaHandler createTemporary(StorageType storageType,
throw new ReplicaAlreadyExistsException("Block " + b +
" is being written, and cannot be written to.");
}
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
BInfo binfo = new BInfo(b.getBlockPoolId(), b.getLocalBlock(), true);
map.put(binfo.theBlock, binfo);
getBlockMap(b).put(binfo.theBlock, binfo);
return new ReplicaHandler(binfo, null);
}
protected synchronized InputStream getBlockInputStream(ExtendedBlock b)
throws IOException {
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
BInfo binfo = map.get(b.getLocalBlock());
BInfo binfo = getBlockMap(b).get(b.getLocalBlock());
if (binfo == null) {
throw new IOException("No such Block " + b );
}
@ -1077,8 +1181,7 @@ public ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkoff,
@Override // FsDatasetSpi
public synchronized LengthInputStream getMetaDataInputStream(ExtendedBlock b
) throws IOException {
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
BInfo binfo = map.get(b.getLocalBlock());
BInfo binfo = getBlockMap(b).get(b.getLocalBlock());
if (binfo == null) {
throw new IOException("No such Block " + b );
}
@ -1266,8 +1369,7 @@ public boolean hasEnoughResource() {
public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
throws IOException {
ExtendedBlock b = rBlock.getBlock();
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
BInfo binfo = map.get(b.getLocalBlock());
BInfo binfo = getBlockMap(b).get(b.getLocalBlock());
if (binfo == null) {
throw new IOException("No such Block " + b );
}
@ -1282,7 +1384,7 @@ public Replica updateReplicaUnderRecovery(ExtendedBlock oldBlock,
long recoveryId,
long newBlockId,
long newlength) throws IOException {
return getMap(oldBlock.getBlockPoolId()).get(oldBlock.getLocalBlock());
return getBInfo(oldBlock);
}
@Override // FsDatasetSpi
@ -1292,15 +1394,16 @@ public long getReplicaVisibleLength(ExtendedBlock block) {
@Override // FsDatasetSpi
public void addBlockPool(String bpid, Configuration conf) {
Map<Block, BInfo> map = new TreeMap<>();
blockMap.put(bpid, map);
storage.addBlockPool(bpid);
for (SimulatedStorage storage : storages) {
storage.addBlockPool(bpid);
}
}
@Override // FsDatasetSpi
public void shutdownBlockPool(String bpid) {
blockMap.remove(bpid);
storage.removeBlockPool(bpid);
for (SimulatedStorage storage : storages) {
storage.removeBlockPool(bpid);
}
}
@Override // FsDatasetSpi
@ -1311,11 +1414,7 @@ public void deleteBlockPool(String bpid, boolean force) {
@Override
public ReplicaInPipeline convertTemporaryToRbw(ExtendedBlock temporary)
throws IOException {
final Map<Block, BInfo> map = blockMap.get(temporary.getBlockPoolId());
if (map == null) {
throw new IOException("Block pool not found, temporary=" + temporary);
}
final BInfo r = map.get(temporary.getLocalBlock());
final BInfo r = getBlockMap(temporary).get(temporary.getLocalBlock());
if (r == null) {
throw new IOException("Block not found, temporary=" + temporary);
} else if (r.isFinalized()) {
@ -1359,7 +1458,11 @@ public void checkAndUpdate(String bpid, ScanInfo info) throws IOException {
@Override
public FsVolumeReferences getFsVolumeReferences() {
return new FsVolumeReferences(Collections.singletonList(volume));
List<SimulatedVolume> volumes = new ArrayList<>();
for (SimulatedStorage storage : storages) {
volumes.add(storage.getVolume());
}
return new FsVolumeReferences(volumes);
}
@Override
@ -1371,14 +1474,21 @@ public void addVolume(
@Override
public DatanodeStorage getStorage(final String storageUuid) {
return storageUuid.equals(storage.getStorageUuid()) ?
storage.dnStorage :
null;
for (SimulatedStorage storage : storages) {
if (storageUuid.equals(storage.getStorageUuid())) {
return storage.getDnStorage();
}
}
return null;
}
@Override
public StorageReport[] getStorageReports(String bpid) {
return new StorageReport[] {storage.getStorageReport(bpid)};
List<StorageReport> reports = new ArrayList<>();
for (SimulatedStorage storage : storages) {
reports.add(storage.getStorageReport(bpid));
}
return reports.toArray(new StorageReport[0]);
}
@Override
@ -1393,7 +1503,7 @@ public Map<String, Object> getVolumeInfoMap() {
@Override
public FsVolumeSpi getVolume(ExtendedBlock b) {
return volume;
return getStorage(b.getLocalBlock()).getVolume();
}
@Override
@ -1428,12 +1538,12 @@ public ReplicaInfo moveBlockAcrossStorage(ExtendedBlock block,
@Override
public void setPinning(ExtendedBlock b) throws IOException {
blockMap.get(b.getBlockPoolId()).get(b.getLocalBlock()).pinned = true;
getBlockMap(b).get(b.getLocalBlock()).pinned = true;
}
@Override
public boolean getPinning(ExtendedBlock b) throws IOException {
return blockMap.get(b.getBlockPoolId()).get(b.getLocalBlock()).pinned;
return getBlockMap(b).get(b.getLocalBlock()).pinned;
}
@Override

View File

@ -26,20 +26,19 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.Map;
import org.apache.commons.lang.exception.ExceptionUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.SequentialBlockIdGenerator;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.util.DataChecksum;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@ -53,6 +52,16 @@ public class TestSimulatedFSDataset {
static final int BLOCK_LENGTH_MULTIPLIER = 79;
static final long FIRST_BLK_ID = 1;
private final int storageCount;
public TestSimulatedFSDataset() {
this(1);
}
protected TestSimulatedFSDataset(int storageCount) {
this.storageCount = storageCount;
}
@Before
public void setUp() throws Exception {
conf = new HdfsConfiguration();
@ -187,43 +196,28 @@ private void testWriteRead(boolean negativeBlkID) throws IOException {
@Test
public void testGetBlockReport() throws IOException {
SimulatedFSDataset fsdataset = getSimulatedFSDataset();
BlockListAsLongs blockReport = fsdataset.getBlockReport(bpid);
assertEquals(0, blockReport.getNumberOfBlocks());
final SimulatedFSDataset fsdataset = getSimulatedFSDataset();
assertBlockReportCountAndSize(fsdataset, 0);
addSomeBlocks(fsdataset);
blockReport = fsdataset.getBlockReport(bpid);
assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
for (Block b: blockReport) {
assertNotNull(b);
assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
}
assertBlockReportCountAndSize(fsdataset, NUMBLOCKS);
assertBlockLengthInBlockReports(fsdataset);
}
@Test
public void testInjectionEmpty() throws IOException {
SimulatedFSDataset fsdataset = getSimulatedFSDataset();
BlockListAsLongs blockReport = fsdataset.getBlockReport(bpid);
assertEquals(0, blockReport.getNumberOfBlocks());
assertBlockReportCountAndSize(fsdataset, 0);
int bytesAdded = addSomeBlocks(fsdataset);
blockReport = fsdataset.getBlockReport(bpid);
assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
for (Block b: blockReport) {
assertNotNull(b);
assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
}
assertBlockReportCountAndSize(fsdataset, NUMBLOCKS);
assertBlockLengthInBlockReports(fsdataset);
// Inject blocks into an empty fsdataset
// - injecting the blocks we got above.
SimulatedFSDataset sfsdataset = getSimulatedFSDataset();
sfsdataset.injectBlocks(bpid, blockReport);
blockReport = sfsdataset.getBlockReport(bpid);
assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
for (Block b: blockReport) {
assertNotNull(b);
assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
assertEquals(blockIdToLen(b.getBlockId()), sfsdataset
.getLength(new ExtendedBlock(bpid, b)));
}
injectBlocksFromBlockReport(fsdataset, sfsdataset);
assertBlockReportCountAndSize(fsdataset, NUMBLOCKS);
assertBlockLengthInBlockReports(fsdataset, sfsdataset);
assertEquals(bytesAdded, sfsdataset.getDfsUsed());
assertEquals(sfsdataset.getCapacity()-bytesAdded, sfsdataset.getRemaining());
}
@ -231,16 +225,10 @@ public void testInjectionEmpty() throws IOException {
@Test
public void testInjectionNonEmpty() throws IOException {
SimulatedFSDataset fsdataset = getSimulatedFSDataset();
BlockListAsLongs blockReport = fsdataset.getBlockReport(bpid);
assertEquals(0, blockReport.getNumberOfBlocks());
assertBlockReportCountAndSize(fsdataset, 0);
int bytesAdded = addSomeBlocks(fsdataset);
blockReport = fsdataset.getBlockReport(bpid);
assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
for (Block b: blockReport) {
assertNotNull(b);
assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
}
fsdataset = null;
assertBlockReportCountAndSize(fsdataset, NUMBLOCKS);
assertBlockLengthInBlockReports(fsdataset);
// Inject blocks into an non-empty fsdataset
// - injecting the blocks we got above.
@ -248,19 +236,10 @@ public void testInjectionNonEmpty() throws IOException {
// Add come blocks whose block ids do not conflict with
// the ones we are going to inject.
bytesAdded += addSomeBlocks(sfsdataset, NUMBLOCKS+1, false);
sfsdataset.getBlockReport(bpid);
assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
sfsdataset.getBlockReport(bpid);
assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
sfsdataset.injectBlocks(bpid, blockReport);
blockReport = sfsdataset.getBlockReport(bpid);
assertEquals(NUMBLOCKS*2, blockReport.getNumberOfBlocks());
for (Block b: blockReport) {
assertNotNull(b);
assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
assertEquals(blockIdToLen(b.getBlockId()), sfsdataset
.getLength(new ExtendedBlock(bpid, b)));
}
assertBlockReportCountAndSize(sfsdataset, NUMBLOCKS);
injectBlocksFromBlockReport(fsdataset, sfsdataset);
assertBlockReportCountAndSize(sfsdataset, NUMBLOCKS * 2);
assertBlockLengthInBlockReports(fsdataset, sfsdataset);
assertEquals(bytesAdded, sfsdataset.getDfsUsed());
assertEquals(sfsdataset.getCapacity()-bytesAdded, sfsdataset.getRemaining());
@ -270,7 +249,7 @@ public void testInjectionNonEmpty() throws IOException {
try {
sfsdataset = getSimulatedFSDataset();
sfsdataset.addBlockPool(bpid, conf);
sfsdataset.injectBlocks(bpid, blockReport);
injectBlocksFromBlockReport(fsdataset, sfsdataset);
assertTrue("Expected an IO exception", false);
} catch (IOException e) {
// ok - as expected
@ -334,8 +313,68 @@ public void testInvalidate() throws IOException {
assertTrue(fsdataset.isValidBlock(new ExtendedBlock(bpid, b)));
}
}
private SimulatedFSDataset getSimulatedFSDataset() {
/**
* Inject all of the blocks returned from sourceFSDataset's block reports
* into destinationFSDataset.
*/
private void injectBlocksFromBlockReport(SimulatedFSDataset sourceFSDataset,
SimulatedFSDataset destinationFSDataset) throws IOException {
for (Map.Entry<DatanodeStorage, BlockListAsLongs> ent :
sourceFSDataset.getBlockReports(bpid).entrySet()) {
destinationFSDataset.injectBlocks(bpid, ent.getValue());
}
}
/**
* Assert that the number of block reports returned from fsdataset matches
* {@code storageCount}, and that the total number of blocks is equal to
* expectedBlockCount.
*/
private void assertBlockReportCountAndSize(SimulatedFSDataset fsdataset,
int expectedBlockCount) {
Map<DatanodeStorage, BlockListAsLongs> blockReportMap =
fsdataset.getBlockReports(bpid);
assertEquals(storageCount, blockReportMap.size());
int totalCount = 0;
for (Map.Entry<DatanodeStorage, BlockListAsLongs> ent :
blockReportMap.entrySet()) {
totalCount += ent.getValue().getNumberOfBlocks();
}
assertEquals(expectedBlockCount, totalCount);
}
/**
* Convenience method to call {@link #assertBlockLengthInBlockReports(
* SimulatedFSDataset,SimulatedFSDataset)} with a null second parameter.
*/
private void assertBlockLengthInBlockReports(SimulatedFSDataset fsdataset)
throws IOException {
assertBlockLengthInBlockReports(fsdataset, null);
}
/**
* Assert that, for all of the blocks in the block report(s) returned from
* fsdataset, they are not null and their length matches the expectation.
* If otherFSDataset is non-null, additionally confirm that its idea of the
* length of the block matches as well.
*/
private void assertBlockLengthInBlockReports(SimulatedFSDataset fsdataset,
SimulatedFSDataset otherFSDataset) throws IOException {
for (Map.Entry<DatanodeStorage, BlockListAsLongs> ent :
fsdataset.getBlockReports(bpid).entrySet()) {
for (Block b : ent.getValue()) {
assertNotNull(b);
assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
if (otherFSDataset != null) {
assertEquals(blockIdToLen(b.getBlockId()), otherFSDataset
.getLength(new ExtendedBlock(bpid, b)));
}
}
}
}
protected SimulatedFSDataset getSimulatedFSDataset() {
SimulatedFSDataset fsdataset = new SimulatedFSDataset(null, conf);
fsdataset.addBlockPool(bpid, conf);
return fsdataset;

View File

@ -0,0 +1,50 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.datanode;
import org.junit.Before;
import org.junit.Test;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
import static org.junit.Assert.assertEquals;
/**
* Test that the {@link SimulatedFSDataset} works correctly when configured
* with multiple storages.
*/
public class TestSimulatedFSDatasetWithMultipleStorages
extends TestSimulatedFSDataset {
public TestSimulatedFSDatasetWithMultipleStorages() {
super(2);
}
@Before
public void setUp() throws Exception {
super.setUp();
conf.set(DFS_DATANODE_DATA_DIR_KEY, "data1,data2");
}
@Test
public void testMultipleStoragesConfigured() {
SimulatedFSDataset fsDataset = getSimulatedFSDataset();
assertEquals(2, fsDataset.getStorageReports(bpid).length);
}
}