|
@@ -32,12 +32,9 @@ import org.apache.hadoop.hdfs.DeprecatedUTF8;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
|
|
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
|
|
|
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
|
|
|
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
|
|
@@ -50,7 +47,6 @@ import org.apache.hadoop.io.LongWritable;
|
|
|
import org.apache.hadoop.io.ShortWritable;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.io.WritableUtils;
|
|
|
-import org.apache.hadoop.io.erasurecode.ECSchema;
|
|
|
import org.xml.sax.ContentHandler;
|
|
|
import org.xml.sax.SAXException;
|
|
|
|
|
@@ -128,45 +124,22 @@ public class FSImageSerialization {
|
|
|
short blockReplication = in.readShort();
|
|
|
long modificationTime = in.readLong();
|
|
|
long preferredBlockSize = in.readLong();
|
|
|
- final boolean isStriped = NameNodeLayoutVersion.supports(
|
|
|
- NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)
|
|
|
- && (in.readBoolean());
|
|
|
-
|
|
|
- // TODO: ECSchema can be restored from persisted file (HDFS-7859).
|
|
|
- final ECSchema schema = isStriped ?
|
|
|
- ErasureCodingSchemaManager.getSystemDefaultSchema() : null;
|
|
|
|
|
|
int numBlocks = in.readInt();
|
|
|
|
|
|
- final BlockInfoContiguous[] blocksContiguous;
|
|
|
- BlockInfoStriped[] blocksStriped = null;
|
|
|
- if (isStriped) {
|
|
|
- blocksContiguous = new BlockInfoContiguous[0];
|
|
|
- blocksStriped = new BlockInfoStriped[numBlocks];
|
|
|
- int i = 0;
|
|
|
- for (; i < numBlocks - 1; i++) {
|
|
|
- blocksStriped[i] = new BlockInfoStriped(new Block(), schema);
|
|
|
- blocksStriped[i].readFields(in);
|
|
|
- }
|
|
|
- if (numBlocks > 0) {
|
|
|
- blocksStriped[i] = new BlockInfoStripedUnderConstruction(new Block(),
|
|
|
- schema, BlockUCState.UNDER_CONSTRUCTION, null);
|
|
|
- blocksStriped[i].readFields(in);
|
|
|
- }
|
|
|
- } else {
|
|
|
- blocksContiguous = new BlockInfoContiguous[numBlocks];
|
|
|
- Block blk = new Block();
|
|
|
- int i = 0;
|
|
|
- for (; i < numBlocks-1; i++) {
|
|
|
- blk.readFields(in);
|
|
|
- blocksContiguous[i] = new BlockInfoContiguous(blk, blockReplication);
|
|
|
- }
|
|
|
- // last block is UNDER_CONSTRUCTION
|
|
|
- if(numBlocks > 0) {
|
|
|
- blk.readFields(in);
|
|
|
- blocksContiguous[i] = new BlockInfoContiguousUnderConstruction(
|
|
|
- blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
|
|
|
- }
|
|
|
+ final BlockInfoContiguous[] blocksContiguous =
|
|
|
+ new BlockInfoContiguous[numBlocks];
|
|
|
+ Block blk = new Block();
|
|
|
+ int i = 0;
|
|
|
+ for (; i < numBlocks - 1; i++) {
|
|
|
+ blk.readFields(in);
|
|
|
+ blocksContiguous[i] = new BlockInfoContiguous(blk, blockReplication);
|
|
|
+ }
|
|
|
+ // last block is UNDER_CONSTRUCTION
|
|
|
+ if(numBlocks > 0) {
|
|
|
+ blk.readFields(in);
|
|
|
+ blocksContiguous[i] = new BlockInfoContiguousUnderConstruction(
|
|
|
+ blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
|
|
|
}
|
|
|
|
|
|
PermissionStatus perm = PermissionStatus.read(in);
|
|
@@ -180,19 +153,8 @@ public class FSImageSerialization {
|
|
|
|
|
|
// Images in the pre-protobuf format will not have the lazyPersist flag,
|
|
|
// so it is safe to pass false always.
|
|
|
- INodeFile file;
|
|
|
- if (isStriped) {
|
|
|
- file = new INodeFile(inodeId, name, perm, modificationTime,
|
|
|
- modificationTime, blocksContiguous, (short) 0, preferredBlockSize);
|
|
|
- file.addStripedBlocksFeature();
|
|
|
- for (int i = 0; i < numBlocks; i++) {
|
|
|
- file.getStripedBlocksFeature().addBlock(blocksStriped[i]);
|
|
|
- }
|
|
|
- } else {
|
|
|
- file = new INodeFile(inodeId, name, perm, modificationTime,
|
|
|
- modificationTime, blocksContiguous, blockReplication,
|
|
|
- preferredBlockSize);
|
|
|
- }
|
|
|
+ INodeFile file = new INodeFile(inodeId, name, perm, modificationTime,
|
|
|
+ modificationTime, blocksContiguous, blockReplication, preferredBlockSize);
|
|
|
file.toUnderConstruction(clientName, clientMachine);
|
|
|
return file;
|
|
|
}
|
|
@@ -207,8 +169,7 @@ public class FSImageSerialization {
|
|
|
out.writeShort(cons.getFileReplication());
|
|
|
out.writeLong(cons.getModificationTime());
|
|
|
out.writeLong(cons.getPreferredBlockSize());
|
|
|
- // whether the file has striped blocks
|
|
|
- out.writeBoolean(cons.isStriped());
|
|
|
+
|
|
|
writeBlocks(cons.getBlocks(), out);
|
|
|
cons.getPermissionStatus().write(out);
|
|
|
|
|
@@ -233,8 +194,7 @@ public class FSImageSerialization {
|
|
|
out.writeLong(file.getModificationTime());
|
|
|
out.writeLong(file.getAccessTime());
|
|
|
out.writeLong(file.getPreferredBlockSize());
|
|
|
- // whether the file has striped blocks
|
|
|
- out.writeBoolean(file.isStriped());
|
|
|
+
|
|
|
writeBlocks(file.getBlocks(), out);
|
|
|
SnapshotFSImageFormat.saveFileDiffList(file, out);
|
|
|
|
|
@@ -347,7 +307,7 @@ public class FSImageSerialization {
|
|
|
if (!isWithName) {
|
|
|
Preconditions.checkState(ref instanceof INodeReference.DstReference);
|
|
|
// dst snapshot id
|
|
|
- out.writeInt(((INodeReference.DstReference) ref).getDstSnapshotId());
|
|
|
+ out.writeInt(ref.getDstSnapshotId());
|
|
|
} else {
|
|
|
out.writeInt(((INodeReference.WithName) ref).getLastSnapshotId());
|
|
|
}
|