|
@@ -68,18 +68,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;
|
|
@@ -95,7 +95,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
return true;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
public static void setFactory(Configuration conf) {
|
|
|
conf.set(DFSConfigKeys.DFS_DATANODE_FSDATASET_FACTORY_KEY,
|
|
|
Factory.class.getName());
|
|
@@ -105,12 +105,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
byte firstByte = (byte) (b.getBlockId() & BYTE_MASK);
|
|
|
return (byte) ((firstByte + offsetInBlk) & 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 =
|
|
@@ -145,7 +145,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
if (theBlock.getNumBytes() < 0) {
|
|
|
theBlock.setNumBytes(0);
|
|
|
}
|
|
|
- if (!storage.alloc(bpid, theBlock.getNumBytes())) {
|
|
|
+ if (!storage.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");
|
|
@@ -160,7 +160,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
oStream = null;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public String getStorageUuid() {
|
|
|
return storage.getStorageUuid();
|
|
@@ -188,7 +188,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
theBlock.setNumBytes(length);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
synchronized SimulatedInputStream getIStream() {
|
|
|
if (!finalized) {
|
|
|
// throw new IOException("Trying to read an unfinalized block");
|
|
@@ -197,12 +197,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) {
|
|
@@ -216,7 +216,7 @@ 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) {
|
|
@@ -227,7 +227,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
} else {
|
|
|
storage.free(bpid, -extraLen);
|
|
|
}
|
|
|
- theBlock.setNumBytes(finalSize);
|
|
|
+ theBlock.setNumBytes(finalSize);
|
|
|
|
|
|
finalized = true;
|
|
|
oStream = null;
|
|
@@ -248,7 +248,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
}
|
|
|
|
|
|
SimulatedInputStream getMetaIStream() {
|
|
|
- return new SimulatedInputStream(nullCrcFileData);
|
|
|
+ return new SimulatedInputStream(nullCrcFileData);
|
|
|
}
|
|
|
|
|
|
synchronized boolean isFinalized() {
|
|
@@ -257,8 +257,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
|
|
|
@Override
|
|
|
synchronized public ReplicaOutputStreams createStreams(boolean isCreate,
|
|
|
- DataChecksum requestedChecksum)
|
|
|
- throws IOException {
|
|
|
+ DataChecksum requestedChecksum) throws IOException {
|
|
|
if (finalized) {
|
|
|
throw new IOException("Trying to write to a finalized replica "
|
|
|
+ theBlock);
|
|
@@ -333,31 +332,31 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
return false;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Class is used for tracking block pool storage utilization similar
|
|
|
* to {@link BlockPoolSlice}
|
|
|
*/
|
|
|
private static class SimulatedBPStorage {
|
|
|
private long used; // in bytes
|
|
|
-
|
|
|
+
|
|
|
long getUsed() {
|
|
|
return used;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
void alloc(long amount) {
|
|
|
used += amount;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
void free(long amount) {
|
|
|
used -= amount;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
SimulatedBPStorage() {
|
|
|
- used = 0;
|
|
|
+ used = 0;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Class used for tracking datanode level storage utilization similar
|
|
|
* to {@link FSVolumeSet}
|
|
@@ -368,15 +367,15 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
|
|
|
private final long capacity; // in bytes
|
|
|
private final DatanodeStorage dnStorage;
|
|
|
-
|
|
|
+
|
|
|
synchronized long getFree() {
|
|
|
return capacity - getUsed();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
long getCapacity() {
|
|
|
return capacity;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
synchronized long getUsed() {
|
|
|
long used = 0;
|
|
|
for (SimulatedBPStorage bpStorage : map.values()) {
|
|
@@ -384,11 +383,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
}
|
|
|
return used;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
synchronized long getBlockPoolUsed(String bpid) throws IOException {
|
|
|
return getBPStorage(bpid).getUsed();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
int getNumFailedVolumes() {
|
|
|
return 0;
|
|
|
}
|
|
@@ -398,20 +397,20 @@ 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) {
|
|
|
capacity = cap;
|
|
|
dnStorage = new DatanodeStorage(
|
|
|
"SimulatedStorage-" + DatanodeStorage.generateUuid(),
|
|
|
state, StorageType.DEFAULT);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
synchronized void addBlockPool(String bpid) {
|
|
|
SimulatedBPStorage bpStorage = map.get(bpid);
|
|
|
if (bpStorage != null) {
|
|
@@ -419,11 +418,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) {
|
|
@@ -435,7 +434,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
String getStorageUuid() {
|
|
|
return dnStorage.getStorageID();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
DatanodeStorage getDnStorage() {
|
|
|
return dnStorage;
|
|
|
}
|
|
@@ -462,7 +461,17 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
|
|
|
@Override
|
|
|
public FsVolumeReference obtainReference() throws ClosedChannelException {
|
|
|
- return null;
|
|
|
+ return new FsVolumeReference() {
|
|
|
+ @Override
|
|
|
+ public void close() throws IOException {
|
|
|
+ // no-op.
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public FsVolumeSpi getVolume() {
|
|
|
+ return SimulatedVolume.this;
|
|
|
+ }
|
|
|
+ };
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -562,7 +571,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
private final SimulatedVolume volume;
|
|
|
private final String datanodeUuid;
|
|
|
private final DataNode datanode;
|
|
|
-
|
|
|
+
|
|
|
|
|
|
public SimulatedFSDataset(DataStorage storage, Configuration conf) {
|
|
|
this(null, storage, conf);
|
|
@@ -611,14 +620,14 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
map = new HashMap<Block, BInfo>();
|
|
|
blockMap.put(bpid, map);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
for (Block b: injectBlocks) {
|
|
|
BInfo binfo = new BInfo(bpid, b, false);
|
|
|
map.put(binfo.theBlock, binfo);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/** 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);
|
|
@@ -684,7 +693,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
public long getBlockPoolUsed(String bpid) throws IOException {
|
|
|
return storage.getBlockPoolUsed(bpid);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override // FSDatasetMBean
|
|
|
public long getRemaining() {
|
|
|
return storage.getFree();
|
|
@@ -775,7 +784,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
+ @Override
|
|
|
public synchronized String getReplicaString(String bpid, long blockId) {
|
|
|
Replica r = null;
|
|
|
final Map<Block, BInfo> map = blockMap.get(bpid);
|
|
@@ -865,14 +874,14 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
*
|
|
|
* @throws ReplicaNotFoundException If the replica is not found
|
|
|
*
|
|
|
- * @throws UnexpectedReplicaStateException If the replica is not in the
|
|
|
+ * @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);
|
|
|
}
|
|
@@ -956,7 +965,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)
|
|
@@ -988,11 +997,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
public synchronized ReplicaHandler createTemporary(
|
|
|
StorageType storageType, ExtendedBlock b) throws IOException {
|
|
|
if (isValidBlock(b)) {
|
|
|
- throw new ReplicaAlreadyExistsException("Block " + b +
|
|
|
+ throw new ReplicaAlreadyExistsException("Block " + b +
|
|
|
" is valid, and cannot be written to.");
|
|
|
}
|
|
|
if (isValidRbw(b)) {
|
|
|
- throw new ReplicaAlreadyExistsException("Block " + b +
|
|
|
+ throw new ReplicaAlreadyExistsException("Block " + b +
|
|
|
" is being written, and cannot be written to.");
|
|
|
}
|
|
|
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
|
|
@@ -1006,12 +1015,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
|
|
|
BInfo binfo = map.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 {
|
|
@@ -1033,10 +1042,10 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
|
|
|
BInfo binfo = map.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();
|
|
@@ -1044,19 +1053,17 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public Set<File> checkDataDir() {
|
|
|
- // nothing to check for simulated data set
|
|
|
- return null;
|
|
|
+ public void handleVolumeFailures(Set<FsVolumeSpi> failedVolumes) {
|
|
|
}
|
|
|
|
|
|
@Override // FsDatasetSpi
|
|
|
public synchronized void adjustCrcChannelPosition(ExtendedBlock b,
|
|
|
- ReplicaOutputStreams stream,
|
|
|
+ ReplicaOutputStreams stream,
|
|
|
int checksumSize)
|
|
|
throws IOException {
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
+ /**
|
|
|
* Simulated input and output streams
|
|
|
*
|
|
|
*/
|
|
@@ -1065,7 +1072,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
int currentPos = 0;
|
|
|
byte[] data = null;
|
|
|
Block theBlock = null;
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* An input stream of size l with repeated bytes
|
|
|
* @param l size of the stream
|
|
@@ -1075,7 +1082,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
|
|
@@ -1084,7 +1091,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
data = iData;
|
|
|
length = data.length;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* @return the lenght of the input stream
|
|
|
*/
|
|
@@ -1103,9 +1110,9 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
return simulatedByte(theBlock, currentPos++) & BYTE_MASK;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
- public int read(byte[] b) throws IOException {
|
|
|
+ public int read(byte[] b) throws IOException {
|
|
|
|
|
|
if (b == null) {
|
|
|
throw new NullPointerException();
|
|
@@ -1128,7 +1135,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
return bytesRead;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* This class implements an output stream that merely throws its data away, but records its
|
|
|
* length.
|
|
@@ -1136,15 +1143,15 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
*/
|
|
|
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() {
|
|
@@ -1156,17 +1163,17 @@ 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,
|
|
@@ -1174,11 +1181,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
length += len;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private ObjectName mbeanName;
|
|
|
|
|
|
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Register the FSDataset MBean using the name
|
|
|
* "hadoop:service=DataNode,name=FSDatasetState-<storageid>"
|
|
@@ -1187,7 +1194,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;
|
|
|
|
|
@@ -1198,7 +1205,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
} catch (NotCompliantMBeanException e) {
|
|
|
DataNode.LOG.warn("Error registering FSDatasetState MBean", e);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
DataNode.LOG.info("Registered FSDatasetState MBean");
|
|
|
}
|
|
|
|
|
@@ -1211,7 +1218,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
public String getStorageInfo() {
|
|
|
return "Simulated FSDataset-" + datanodeUuid;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public boolean hasEnoughResource() {
|
|
|
return true;
|
|
@@ -1224,11 +1231,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
|
|
|
BInfo binfo = map.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(),
|
|
|
+ return new ReplicaRecoveryInfo(binfo.getBlockId(), binfo.getBytesOnDisk(),
|
|
|
+ binfo.getGenerationStamp(),
|
|
|
binfo.isFinalized()?ReplicaState.FINALIZED : ReplicaState.RBW);
|
|
|
}
|
|
|
|
|
@@ -1251,13 +1258,13 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
blockMap.put(bpid, map);
|
|
|
storage.addBlockPool(bpid);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override // FsDatasetSpi
|
|
|
public void shutdownBlockPool(String bpid) {
|
|
|
blockMap.remove(bpid);
|
|
|
storage.removeBlockPool(bpid);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override // FsDatasetSpi
|
|
|
public void deleteBlockPool(String bpid, boolean force) {
|
|
|
return;
|
|
@@ -1321,7 +1328,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
|
|
|
@Override
|
|
|
public FsVolumeReferences getFsVolumeReferences() {
|
|
|
- throw new UnsupportedOperationException();
|
|
|
+ return new FsVolumeReferences(Collections.singletonList(volume));
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -1386,17 +1393,17 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|
|
// TODO Auto-generated method stub
|
|
|
return null;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public void setPinning(ExtendedBlock b) throws IOException {
|
|
|
blockMap.get(b.getBlockPoolId()).get(b.getLocalBlock()).pinned = true;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public boolean getPinning(ExtendedBlock b) throws IOException {
|
|
|
return blockMap.get(b.getBlockPoolId()).get(b.getLocalBlock()).pinned;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public boolean isDeletingBlock(String bpid, long blockId) {
|
|
|
throw new UnsupportedOperationException();
|