|
@@ -25,10 +25,11 @@ 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;
|
|
|
import java.util.Map;
|
|
|
+import java.util.Objects;
|
|
|
import java.util.Set;
|
|
|
import java.util.TreeMap;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
@@ -75,18 +76,18 @@ import org.apache.hadoop.util.DataChecksum;
|
|
|
|
|
|
/**
|
|
|
* This class implements a simulated FSDataset.
|
|
|
- *
|
|
|
+ *
|
|
|
* Blocks that are created are recorded but their data (plus their CRCs) are
|
|
|
* discarded.
|
|
|
* Fixed data is returned when blocks are read; a null CRC meta file is
|
|
|
* created for such data.
|
|
|
- *
|
|
|
+ *
|
|
|
* This FSDataset does not remember any block information across its
|
|
|
* restarts; it does however offer an operation to inject blocks
|
|
|
* (See the TestInectionForSImulatedStorage()
|
|
|
* for a usage example of injection.
|
|
|
- *
|
|
|
- * Note the synchronization is coarse grained - it is at each method.
|
|
|
+ *
|
|
|
+ * Note the synchronization is coarse grained - it is at each method.
|
|
|
*/
|
|
|
public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
public final static int BYTE_MASK = 0xff;
|
|
@@ -145,17 +146,17 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
byte firstByte = (byte) (b.getBlockId() & BYTE_MASK);
|
|
|
return (byte) ((firstByte + offsetInBlk % 29) & BYTE_MASK);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
public static final String CONFIG_PROPERTY_CAPACITY =
|
|
|
"dfs.datanode.simulateddatastorage.capacity";
|
|
|
-
|
|
|
+
|
|
|
public static final long DEFAULT_CAPACITY = 2L<<40; // 1 terabyte
|
|
|
-
|
|
|
+
|
|
|
public static final String CONFIG_PROPERTY_STATE =
|
|
|
"dfs.datanode.simulateddatastorage.state";
|
|
|
private static final DatanodeStorage.State DEFAULT_STATE =
|
|
|
DatanodeStorage.State.NORMAL;
|
|
|
-
|
|
|
+
|
|
|
static final byte[] nullCrcFileData;
|
|
|
|
|
|
private final AutoCloseableLock datasetLock;
|
|
@@ -183,8 +184,8 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
private boolean pinned = false;
|
|
|
BInfo(String bpid, Block b, boolean forWriting) throws IOException {
|
|
|
theBlock = new Block(b);
|
|
|
- if (theBlock.getNumBytes() < 0) {
|
|
|
- theBlock.setNumBytes(0);
|
|
|
+ if (theBlock.getNumBytes() < 0L) {
|
|
|
+ theBlock.setNumBytes(0L);
|
|
|
}
|
|
|
if (!getStorage(theBlock).alloc(bpid, theBlock.getNumBytes())) {
|
|
|
// expected length - actual length may
|
|
@@ -201,7 +202,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
oStream = null;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public String getStorageUuid() {
|
|
|
return getStorage(theBlock).getStorageUuid();
|
|
@@ -229,7 +230,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
theBlock.setNumBytes(length);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
synchronized SimulatedInputStream getIStream() {
|
|
|
if (!finalized) {
|
|
|
// throw new IOException("Trying to read an unfinalized block");
|
|
@@ -238,12 +239,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
return new SimulatedInputStream(theBlock.getNumBytes(), theBlock);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
synchronized void finalizeBlock(String bpid, long finalSize)
|
|
|
throws IOException {
|
|
|
if (finalized) {
|
|
|
throw new IOException(
|
|
|
- "Finalizing a block that has already been finalized" +
|
|
|
+ "Finalizing a block that has already been finalized" +
|
|
|
theBlock.getBlockId());
|
|
|
}
|
|
|
if (oStream == null) {
|
|
@@ -257,10 +258,10 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
throw new IOException(
|
|
|
"Size passed to finalize does not match the amount of data written");
|
|
|
}
|
|
|
- // We had allocated the expected length when block was created;
|
|
|
+ // We had allocated the expected length when block was created;
|
|
|
// adjust if necessary
|
|
|
long extraLen = finalSize - theBlock.getNumBytes();
|
|
|
- if (extraLen > 0) {
|
|
|
+ if (extraLen > 0L) {
|
|
|
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");
|
|
@@ -268,7 +269,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
} else {
|
|
|
getStorage(theBlock).free(bpid, -extraLen);
|
|
|
}
|
|
|
- theBlock.setNumBytes(finalSize);
|
|
|
+ theBlock.setNumBytes(finalSize);
|
|
|
|
|
|
finalized = true;
|
|
|
oStream = null;
|
|
@@ -289,7 +290,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
}
|
|
|
|
|
|
SimulatedInputStream getMetaIStream() {
|
|
|
- return new SimulatedInputStream(nullCrcFileData);
|
|
|
+ return new SimulatedInputStream(nullCrcFileData);
|
|
|
}
|
|
|
|
|
|
synchronized boolean isFinalized() {
|
|
@@ -297,7 +298,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- synchronized public ReplicaOutputStreams createStreams(boolean isCreate,
|
|
|
+ synchronized public ReplicaOutputStreams createStreams(boolean isCreate,
|
|
|
DataChecksum requestedChecksum)
|
|
|
throws IOException {
|
|
|
if (finalized) {
|
|
@@ -396,36 +397,37 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
public void stopWriter(long xceiverStopTimeout) throws IOException {
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Class is used for tracking block pool storage utilization similar
|
|
|
* to {@link BlockPoolSlice}
|
|
|
*/
|
|
|
private static class SimulatedBPStorage {
|
|
|
- private long used; // in bytes
|
|
|
+ // in bytes
|
|
|
+ private long used;
|
|
|
private final Map<Block, BInfo> blockMap = new TreeMap<>();
|
|
|
-
|
|
|
+
|
|
|
long getUsed() {
|
|
|
return used;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
void alloc(long amount) {
|
|
|
used += amount;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
void free(long amount) {
|
|
|
used -= amount;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
Map<Block, BInfo> getBlockMap() {
|
|
|
return blockMap;
|
|
|
}
|
|
|
|
|
|
SimulatedBPStorage() {
|
|
|
- used = 0;
|
|
|
+ used = 0L;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Class used for tracking datanode level storage utilization similar
|
|
|
* to {@link FSVolumeSet}
|
|
@@ -437,27 +439,27 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
private final long capacity; // in bytes
|
|
|
private final DatanodeStorage dnStorage;
|
|
|
private final SimulatedVolume volume;
|
|
|
-
|
|
|
+
|
|
|
synchronized long getFree() {
|
|
|
return capacity - getUsed();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
long getCapacity() {
|
|
|
return capacity;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
synchronized long getUsed() {
|
|
|
- long used = 0;
|
|
|
+ long used = 0L;
|
|
|
for (SimulatedBPStorage bpStorage : map.values()) {
|
|
|
used += bpStorage.getUsed();
|
|
|
}
|
|
|
return used;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
synchronized long getBlockPoolUsed(String bpid) throws IOException {
|
|
|
return getBPStorage(bpid).getUsed();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
int getNumFailedVolumes() {
|
|
|
return 0;
|
|
|
}
|
|
@@ -467,13 +469,13 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
getBPStorage(bpid).alloc(amount);
|
|
|
return true;
|
|
|
}
|
|
|
- return false;
|
|
|
+ return false;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
synchronized void free(String bpid, long amount) throws IOException {
|
|
|
getBPStorage(bpid).free(amount);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
SimulatedStorage(long cap, DatanodeStorage.State state,
|
|
|
FileIoProvider fileIoProvider, Configuration conf) {
|
|
|
capacity = cap;
|
|
@@ -484,7 +486,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
DataNodeVolumeMetrics.create(conf, dnStorage.getStorageID());
|
|
|
this.volume = new SimulatedVolume(this, fileIoProvider, volumeMetrics);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
synchronized void addBlockPool(String bpid) {
|
|
|
SimulatedBPStorage bpStorage = map.get(bpid);
|
|
|
if (bpStorage != null) {
|
|
@@ -492,11 +494,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
}
|
|
|
map.put(bpid, new SimulatedBPStorage());
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
synchronized void removeBlockPool(String bpid) {
|
|
|
map.remove(bpid);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private SimulatedBPStorage getBPStorage(String bpid) throws IOException {
|
|
|
SimulatedBPStorage bpStorage = map.get(bpid);
|
|
|
if (bpStorage == null) {
|
|
@@ -508,7 +510,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
String getStorageUuid() {
|
|
|
return dnStorage.getStorageID();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
DatanodeStorage getDnStorage() {
|
|
|
return dnStorage;
|
|
|
}
|
|
@@ -531,7 +533,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
return bpStorage.getBlockMap();
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
static class SimulatedVolume implements FsVolumeSpi {
|
|
|
private final SimulatedStorage storage;
|
|
|
private final FileIoProvider fileIoProvider;
|
|
@@ -635,10 +637,9 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public LinkedList<ScanInfo> compileReport(String bpid,
|
|
|
- LinkedList<ScanInfo> report, ReportCompiler reportCompiler)
|
|
|
+ public void compileReport(String bpid,
|
|
|
+ Collection<ScanInfo> report, ReportCompiler reportCompiler)
|
|
|
throws InterruptedException, IOException {
|
|
|
- return null;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -661,7 +662,6 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
private final List<SimulatedStorage> storages;
|
|
|
private final String datanodeUuid;
|
|
|
private final DataNode datanode;
|
|
|
-
|
|
|
|
|
|
public SimulatedFSDataset(DataStorage storage, Configuration conf) {
|
|
|
this(null, storage, conf);
|
|
@@ -724,7 +724,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
private SimulatedStorage getStorage(Block b) {
|
|
|
return storages.get(LongMath.mod(b.getBlockId(), storages.size()));
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Get the block map that a given block lives within, assuming it is within
|
|
|
* block pool bpid.
|
|
@@ -792,12 +792,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
|
|
|
@Override // FsDatasetSpi
|
|
|
public List<Long> getCacheReport(String bpid) {
|
|
|
- return new LinkedList<Long>();
|
|
|
+ return Collections.emptyList();
|
|
|
}
|
|
|
|
|
|
@Override // FSDatasetMBean
|
|
|
public long getCapacity() {
|
|
|
- long total = 0;
|
|
|
+ long total = 0L;
|
|
|
for (SimulatedStorage storage : storages) {
|
|
|
total += storage.getCapacity();
|
|
|
}
|
|
@@ -806,7 +806,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
|
|
|
@Override // FSDatasetMBean
|
|
|
public long getDfsUsed() {
|
|
|
- long total = 0;
|
|
|
+ long total = 0L;
|
|
|
for (SimulatedStorage storage : storages) {
|
|
|
total += storage.getUsed();
|
|
|
}
|
|
@@ -815,17 +815,16 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
|
|
|
@Override // FSDatasetMBean
|
|
|
public long getBlockPoolUsed(String bpid) throws IOException {
|
|
|
- long total = 0;
|
|
|
+ long total = 0L;
|
|
|
for (SimulatedStorage storage : storages) {
|
|
|
total += storage.getBlockPoolUsed(bpid);
|
|
|
}
|
|
|
return total;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override // FSDatasetMBean
|
|
|
public long getRemaining() {
|
|
|
-
|
|
|
- long total = 0;
|
|
|
+ long total = 0L;
|
|
|
for (SimulatedStorage storage : storages) {
|
|
|
total += storage.getFree();
|
|
|
}
|
|
@@ -834,7 +833,6 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
|
|
|
@Override // FSDatasetMBean
|
|
|
public int getNumFailedVolumes() {
|
|
|
-
|
|
|
int total = 0;
|
|
|
for (SimulatedStorage storage : storages) {
|
|
|
total += storage.getNumFailedVolumes();
|
|
@@ -849,12 +847,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
|
|
|
@Override // FSDatasetMBean
|
|
|
public long getLastVolumeFailureDate() {
|
|
|
- return 0;
|
|
|
+ return 0L;
|
|
|
}
|
|
|
|
|
|
@Override // FSDatasetMBean
|
|
|
public long getEstimatedCapacityLostTotal() {
|
|
|
- return 0;
|
|
|
+ return 0L;
|
|
|
}
|
|
|
|
|
|
@Override // FsDatasetSpi
|
|
@@ -864,27 +862,27 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
|
|
|
@Override // FSDatasetMBean
|
|
|
public long getCacheUsed() {
|
|
|
- return 0l;
|
|
|
+ return 0L;
|
|
|
}
|
|
|
|
|
|
@Override // FSDatasetMBean
|
|
|
public long getCacheCapacity() {
|
|
|
- return 0l;
|
|
|
+ return 0L;
|
|
|
}
|
|
|
|
|
|
@Override // FSDatasetMBean
|
|
|
public long getNumBlocksCached() {
|
|
|
- return 0l;
|
|
|
+ return 0L;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public long getNumBlocksFailedToCache() {
|
|
|
- return 0l;
|
|
|
+ return 0L;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public long getNumBlocksFailedToUncache() {
|
|
|
- return 0l;
|
|
|
+ return 0L;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -922,7 +920,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
+ @Override
|
|
|
public synchronized String getReplicaString(String bpid, long blockId) {
|
|
|
Replica r = null;
|
|
|
try {
|
|
@@ -931,7 +929,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
} catch (IOException ioe) {
|
|
|
// Ignore
|
|
|
}
|
|
|
- return r == null? "null": r.toString();
|
|
|
+ return Objects.toString(r);
|
|
|
}
|
|
|
|
|
|
@Override // FsDatasetSpi
|
|
@@ -1011,21 +1009,21 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
/**
|
|
|
* Check if a block is valid.
|
|
|
*
|
|
|
- * @param b The block to check.
|
|
|
- * @param minLength The minimum length that the block must have. May be 0.
|
|
|
- * @param state If this is null, it is ignored. If it is non-null, we
|
|
|
- * will check that the replica has this state.
|
|
|
+ * @param b The block to check.
|
|
|
+ * @param minLength The minimum length that the block must have. May be 0.
|
|
|
+ * @param state If this is null, it is ignored. If it is non-null, we will
|
|
|
+ * check that the replica has this state.
|
|
|
*
|
|
|
- * @throws ReplicaNotFoundException If the replica is not found
|
|
|
+ * @throws ReplicaNotFoundException If the replica is not found
|
|
|
*
|
|
|
- * @throws UnexpectedReplicaStateException If the replica is not in the
|
|
|
- * expected state.
|
|
|
+ * @throws UnexpectedReplicaStateException If the replica is not in the
|
|
|
+ * expected state.
|
|
|
*/
|
|
|
@Override // {@link FsDatasetSpi}
|
|
|
public void checkBlock(ExtendedBlock b, long minLength, ReplicaState state)
|
|
|
throws ReplicaNotFoundException, UnexpectedReplicaStateException {
|
|
|
final BInfo binfo = getBInfo(b);
|
|
|
-
|
|
|
+
|
|
|
if (binfo == null) {
|
|
|
throw new ReplicaNotFoundException(b);
|
|
|
}
|
|
@@ -1108,7 +1106,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
map.put(binfo.theBlock, binfo);
|
|
|
return binfo;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override // FsDatasetSpi
|
|
|
public synchronized ReplicaHandler recoverRbw(
|
|
|
ExtendedBlock b, long newGS, long minBytesRcvd, long maxBytesRcvd)
|
|
@@ -1157,12 +1155,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
throws IOException {
|
|
|
BInfo binfo = getBlockMap(b).get(b.getLocalBlock());
|
|
|
if (binfo == null) {
|
|
|
- throw new IOException("No such Block " + b );
|
|
|
+ throw new IOException("No such Block " + b);
|
|
|
}
|
|
|
-
|
|
|
return binfo.getIStream();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override // FsDatasetSpi
|
|
|
public synchronized InputStream getBlockInputStream(ExtendedBlock b,
|
|
|
long seekOffset) throws IOException {
|
|
@@ -1183,10 +1180,10 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
) throws IOException {
|
|
|
BInfo binfo = getBlockMap(b).get(b.getLocalBlock());
|
|
|
if (binfo == null) {
|
|
|
- throw new IOException("No such Block " + b );
|
|
|
+ throw new IOException("No such Block " + b);
|
|
|
}
|
|
|
if (!binfo.finalized) {
|
|
|
- throw new IOException("Block " + b +
|
|
|
+ throw new IOException("Block " + b +
|
|
|
" is being written, its meta cannot be read");
|
|
|
}
|
|
|
final SimulatedInputStream sin = binfo.getMetaIStream();
|
|
@@ -1199,23 +1196,20 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
|
|
|
@Override // FsDatasetSpi
|
|
|
public synchronized void adjustCrcChannelPosition(ExtendedBlock b,
|
|
|
- ReplicaOutputStreams stream,
|
|
|
- int checksumSize)
|
|
|
- throws IOException {
|
|
|
+ ReplicaOutputStreams stream, int checksumSize) throws IOException {
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Simulated input and output streams
|
|
|
- *
|
|
|
+ /**
|
|
|
+ * Simulated input and output streams.
|
|
|
*/
|
|
|
static private class SimulatedInputStream extends java.io.InputStream {
|
|
|
final long length; // bytes
|
|
|
int currentPos = 0;
|
|
|
byte[] data = null;
|
|
|
Block theBlock = null;
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
- * An input stream of size l with repeated bytes
|
|
|
+ * An input stream of size l with repeated bytes.
|
|
|
* @param l size of the stream
|
|
|
* @param iRepeatedData byte that is repeated in the stream
|
|
|
*/
|
|
@@ -1223,7 +1217,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
length = l;
|
|
|
theBlock = b;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* An input stream of of the supplied data
|
|
|
* @param iData data to construct the stream
|
|
@@ -1232,7 +1226,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
data = iData;
|
|
|
length = data.length;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* @return the lenght of the input stream
|
|
|
*/
|
|
@@ -1251,10 +1245,9 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
return simulatedByte(theBlock, currentPos++) & BYTE_MASK;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- @Override
|
|
|
- public int read(byte[] b) throws IOException {
|
|
|
|
|
|
+ @Override
|
|
|
+ public int read(byte[] b) throws IOException {
|
|
|
if (b == null) {
|
|
|
throw new NullPointerException();
|
|
|
}
|
|
@@ -1276,23 +1269,22 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
return bytesRead;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* This class implements an output stream that merely throws its data away, but records its
|
|
|
* length.
|
|
|
- *
|
|
|
*/
|
|
|
static private class SimulatedOutputStream extends OutputStream {
|
|
|
long length = 0;
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* constructor for Simulated Output Steram
|
|
|
*/
|
|
|
SimulatedOutputStream() {
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
- *
|
|
|
+ *
|
|
|
* @return the length of the data created so far.
|
|
|
*/
|
|
|
long getLength() {
|
|
@@ -1304,29 +1296,25 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
void setLength(long length) {
|
|
|
this.length = length;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public void write(int arg0) throws IOException {
|
|
|
length++;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public void write(byte[] b) throws IOException {
|
|
|
length += b.length;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
- public void write(byte[] b,
|
|
|
- int off,
|
|
|
- int len) throws IOException {
|
|
|
+ public void write(byte[] b, int off, int len) throws IOException {
|
|
|
length += len;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- private ObjectName mbeanName;
|
|
|
|
|
|
+ private ObjectName mbeanName;
|
|
|
|
|
|
-
|
|
|
/**
|
|
|
* Register the FSDataset MBean using the name
|
|
|
* "hadoop:service=DataNode,name=FSDatasetState-<storageid>"
|
|
@@ -1335,7 +1323,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
*/
|
|
|
void registerMBean(final String storageId) {
|
|
|
// We wrap to bypass standard mbean naming convetion.
|
|
|
- // This wraping can be removed in java 6 as it is more flexible in
|
|
|
+ // This wraping can be removed in java 6 as it is more flexible in
|
|
|
// package naming for mbeans and their impl.
|
|
|
StandardMBean bean;
|
|
|
|
|
@@ -1346,7 +1334,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
} catch (NotCompliantMBeanException e) {
|
|
|
DataNode.LOG.warn("Error registering FSDatasetState MBean", e);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
DataNode.LOG.info("Registered FSDatasetState MBean");
|
|
|
}
|
|
|
|
|
@@ -1359,7 +1347,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
public String getStorageInfo() {
|
|
|
return "Simulated FSDataset-" + datanodeUuid;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public boolean hasEnoughResource() {
|
|
|
return true;
|
|
@@ -1371,12 +1359,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
ExtendedBlock b = rBlock.getBlock();
|
|
|
BInfo binfo = getBlockMap(b).get(b.getLocalBlock());
|
|
|
if (binfo == null) {
|
|
|
- throw new IOException("No such Block " + b );
|
|
|
+ throw new IOException("No such Block " + b);
|
|
|
}
|
|
|
|
|
|
- return new ReplicaRecoveryInfo(binfo.getBlockId(), binfo.getBytesOnDisk(),
|
|
|
- binfo.getGenerationStamp(),
|
|
|
- binfo.isFinalized()?ReplicaState.FINALIZED : ReplicaState.RBW);
|
|
|
+ return new ReplicaRecoveryInfo(binfo.getBlockId(), binfo.getBytesOnDisk(),
|
|
|
+ binfo.getGenerationStamp(),
|
|
|
+ binfo.isFinalized() ? ReplicaState.FINALIZED : ReplicaState.RBW);
|
|
|
}
|
|
|
|
|
|
@Override // FsDatasetSpi
|
|
@@ -1398,14 +1386,14 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
storage.addBlockPool(bpid);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override // FsDatasetSpi
|
|
|
public void shutdownBlockPool(String bpid) {
|
|
|
for (SimulatedStorage storage : storages) {
|
|
|
storage.removeBlockPool(bpid);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override // FsDatasetSpi
|
|
|
public void deleteBlockPool(String bpid, boolean force) {
|
|
|
return;
|
|
@@ -1535,17 +1523,17 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
// TODO Auto-generated method stub
|
|
|
return null;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public void setPinning(ExtendedBlock b) throws IOException {
|
|
|
getBlockMap(b).get(b.getLocalBlock()).pinned = true;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public boolean getPinning(ExtendedBlock b) throws IOException {
|
|
|
return getBlockMap(b).get(b.getLocalBlock()).pinned;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public boolean isDeletingBlock(String bpid, long blockId) {
|
|
|
throw new UnsupportedOperationException();
|