|
@@ -87,7 +87,6 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
|
|
-import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
|
|
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
|
|
import org.apache.hadoop.hdfs.util.XMLUtils;
|
|
|
import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
|
|
@@ -248,8 +247,8 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
void readRpcIds(DataInputStream in, int logVersion)
|
|
|
throws IOException {
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_SUPPORT_RETRYCACHE,
|
|
|
- logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_SUPPORT_RETRYCACHE, logVersion)) {
|
|
|
this.rpcClientId = FSImageSerialization.readBytes(in);
|
|
|
this.rpcCallId = FSImageSerialization.readInt(in);
|
|
|
}
|
|
@@ -385,18 +384,20 @@ public abstract class FSEditLogOp {
|
|
|
@Override
|
|
|
void readFields(DataInputStream in, int logVersion)
|
|
|
throws IOException {
|
|
|
- if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (!NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.length = in.readInt();
|
|
|
}
|
|
|
- if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
|
|
|
this.inodeId = in.readLong();
|
|
|
} else {
|
|
|
// The inodeId should be updated when this editLogOp is applied
|
|
|
this.inodeId = INodeId.GRANDFATHER_INODE_ID;
|
|
|
}
|
|
|
if ((-17 < logVersion && length != 4) ||
|
|
|
- (logVersion <= -17 && length != 5 && !LayoutVersion.supports(
|
|
|
- Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) {
|
|
|
+ (logVersion <= -17 && length != 5 && !NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) {
|
|
|
throw new IOException("Incorrect data format." +
|
|
|
" logVersion is " + logVersion +
|
|
|
" but writables.length is " +
|
|
@@ -404,7 +405,8 @@ public abstract class FSEditLogOp {
|
|
|
}
|
|
|
this.path = FSImageSerialization.readString(in);
|
|
|
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.replication = FSImageSerialization.readShort(in);
|
|
|
this.mtime = FSImageSerialization.readLong(in);
|
|
|
} else {
|
|
@@ -412,8 +414,10 @@ public abstract class FSEditLogOp {
|
|
|
this.mtime = readLong(in);
|
|
|
}
|
|
|
|
|
|
- if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, logVersion)) {
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.FILE_ACCESS_TIME, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.atime = FSImageSerialization.readLong(in);
|
|
|
} else {
|
|
|
this.atime = readLong(in);
|
|
@@ -422,7 +426,8 @@ public abstract class FSEditLogOp {
|
|
|
this.atime = 0;
|
|
|
}
|
|
|
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.blockSize = FSImageSerialization.readLong(in);
|
|
|
} else {
|
|
|
this.blockSize = readLong(in);
|
|
@@ -831,7 +836,8 @@ public abstract class FSEditLogOp {
|
|
|
void readFields(DataInputStream in, int logVersion)
|
|
|
throws IOException {
|
|
|
this.path = FSImageSerialization.readString(in);
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.replication = FSImageSerialization.readShort(in);
|
|
|
} else {
|
|
|
this.replication = readShort(in);
|
|
@@ -922,7 +928,8 @@ public abstract class FSEditLogOp {
|
|
|
@Override
|
|
|
void readFields(DataInputStream in, int logVersion)
|
|
|
throws IOException {
|
|
|
- if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (!NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.length = in.readInt();
|
|
|
if (length < 3) { // trg, srcs.., timestamp
|
|
|
throw new IOException("Incorrect data format " +
|
|
@@ -931,7 +938,8 @@ public abstract class FSEditLogOp {
|
|
|
}
|
|
|
this.trg = FSImageSerialization.readString(in);
|
|
|
int srcSize = 0;
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
srcSize = in.readInt();
|
|
|
} else {
|
|
|
srcSize = this.length - 1 - 1; // trg and timestamp
|
|
@@ -950,7 +958,8 @@ public abstract class FSEditLogOp {
|
|
|
srcs[i]= FSImageSerialization.readString(in);
|
|
|
}
|
|
|
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.timestamp = FSImageSerialization.readLong(in);
|
|
|
} else {
|
|
|
this.timestamp = readLong(in);
|
|
@@ -1056,7 +1065,8 @@ public abstract class FSEditLogOp {
|
|
|
@Override
|
|
|
void readFields(DataInputStream in, int logVersion)
|
|
|
throws IOException {
|
|
|
- if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (!NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.length = in.readInt();
|
|
|
if (this.length != 3) {
|
|
|
throw new IOException("Incorrect data format. "
|
|
@@ -1065,7 +1075,8 @@ public abstract class FSEditLogOp {
|
|
|
}
|
|
|
this.src = FSImageSerialization.readString(in);
|
|
|
this.dst = FSImageSerialization.readString(in);
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.timestamp = FSImageSerialization.readLong(in);
|
|
|
} else {
|
|
|
this.timestamp = readLong(in);
|
|
@@ -1152,14 +1163,16 @@ public abstract class FSEditLogOp {
|
|
|
@Override
|
|
|
void readFields(DataInputStream in, int logVersion)
|
|
|
throws IOException {
|
|
|
- if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (!NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.length = in.readInt();
|
|
|
if (this.length != 2) {
|
|
|
throw new IOException("Incorrect data format. " + "delete operation.");
|
|
|
}
|
|
|
}
|
|
|
this.path = FSImageSerialization.readString(in);
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.timestamp = FSImageSerialization.readLong(in);
|
|
|
} else {
|
|
|
this.timestamp = readLong(in);
|
|
@@ -1253,22 +1266,26 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
@Override
|
|
|
void readFields(DataInputStream in, int logVersion) throws IOException {
|
|
|
- if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (!NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.length = in.readInt();
|
|
|
}
|
|
|
if (-17 < logVersion && length != 2 ||
|
|
|
logVersion <= -17 && length != 3
|
|
|
- && !LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ && !NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
throw new IOException("Incorrect data format. Mkdir operation.");
|
|
|
}
|
|
|
- if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
|
|
|
this.inodeId = FSImageSerialization.readLong(in);
|
|
|
} else {
|
|
|
// This id should be updated when this editLogOp is applied
|
|
|
this.inodeId = INodeId.GRANDFATHER_INODE_ID;
|
|
|
}
|
|
|
this.path = FSImageSerialization.readString(in);
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.timestamp = FSImageSerialization.readLong(in);
|
|
|
} else {
|
|
|
this.timestamp = readLong(in);
|
|
@@ -1277,8 +1294,10 @@ public abstract class FSEditLogOp {
|
|
|
// The disk format stores atimes for directories as well.
|
|
|
// However, currently this is not being updated/used because of
|
|
|
// performance reasons.
|
|
|
- if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, logVersion)) {
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.FILE_ACCESS_TIME, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
FSImageSerialization.readLong(in);
|
|
|
} else {
|
|
|
readLong(in);
|
|
@@ -1861,7 +1880,8 @@ public abstract class FSEditLogOp {
|
|
|
@Override
|
|
|
void readFields(DataInputStream in, int logVersion)
|
|
|
throws IOException {
|
|
|
- if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (!NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.length = in.readInt();
|
|
|
if (length != 3) {
|
|
|
throw new IOException("Incorrect data format. " + "times operation.");
|
|
@@ -1869,7 +1889,8 @@ public abstract class FSEditLogOp {
|
|
|
}
|
|
|
this.path = FSImageSerialization.readString(in);
|
|
|
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.mtime = FSImageSerialization.readLong(in);
|
|
|
this.atime = FSImageSerialization.readLong(in);
|
|
|
} else {
|
|
@@ -1978,14 +1999,16 @@ public abstract class FSEditLogOp {
|
|
|
@Override
|
|
|
void readFields(DataInputStream in, int logVersion)
|
|
|
throws IOException {
|
|
|
- if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (!NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.length = in.readInt();
|
|
|
if (this.length != 4) {
|
|
|
throw new IOException("Incorrect data format. "
|
|
|
+ "symlink operation.");
|
|
|
}
|
|
|
}
|
|
|
- if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
|
|
|
this.inodeId = FSImageSerialization.readLong(in);
|
|
|
} else {
|
|
|
// This id should be updated when the editLogOp is applied
|
|
@@ -1994,7 +2017,8 @@ public abstract class FSEditLogOp {
|
|
|
this.path = FSImageSerialization.readString(in);
|
|
|
this.value = FSImageSerialization.readString(in);
|
|
|
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.mtime = FSImageSerialization.readLong(in);
|
|
|
this.atime = FSImageSerialization.readLong(in);
|
|
|
} else {
|
|
@@ -2112,7 +2136,8 @@ public abstract class FSEditLogOp {
|
|
|
@Override
|
|
|
void readFields(DataInputStream in, int logVersion)
|
|
|
throws IOException {
|
|
|
- if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (!NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.length = in.readInt();
|
|
|
if (this.length != 3) {
|
|
|
throw new IOException("Incorrect data format. " + "Rename operation.");
|
|
@@ -2121,7 +2146,8 @@ public abstract class FSEditLogOp {
|
|
|
this.src = FSImageSerialization.readString(in);
|
|
|
this.dst = FSImageSerialization.readString(in);
|
|
|
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.timestamp = FSImageSerialization.readLong(in);
|
|
|
} else {
|
|
|
this.timestamp = readLong(in);
|
|
@@ -2332,7 +2358,8 @@ public abstract class FSEditLogOp {
|
|
|
throws IOException {
|
|
|
this.token = new DelegationTokenIdentifier();
|
|
|
this.token.readFields(in);
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.expiryTime = FSImageSerialization.readLong(in);
|
|
|
} else {
|
|
|
this.expiryTime = readLong(in);
|
|
@@ -2404,7 +2431,8 @@ public abstract class FSEditLogOp {
|
|
|
throws IOException {
|
|
|
this.token = new DelegationTokenIdentifier();
|
|
|
this.token.readFields(in);
|
|
|
- if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
this.expiryTime = FSImageSerialization.readLong(in);
|
|
|
} else {
|
|
|
this.expiryTime = readLong(in);
|
|
@@ -3488,7 +3516,8 @@ public abstract class FSEditLogOp {
|
|
|
*/
|
|
|
public Reader(DataInputStream in, StreamLimiter limiter, int logVersion) {
|
|
|
this.logVersion = logVersion;
|
|
|
- if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.EDITS_CHESKUM, logVersion)) {
|
|
|
this.checksum = new PureJavaCrc32();
|
|
|
} else {
|
|
|
this.checksum = null;
|
|
@@ -3627,7 +3656,8 @@ public abstract class FSEditLogOp {
|
|
|
throw new IOException("Read invalid opcode " + opCode);
|
|
|
}
|
|
|
|
|
|
- if (LayoutVersion.supports(Feature.STORED_TXIDS, logVersion)) {
|
|
|
+ if (NameNodeLayoutVersion.supports(
|
|
|
+ LayoutVersion.Feature.STORED_TXIDS, logVersion)) {
|
|
|
// Read the txid
|
|
|
op.setTransactionId(in.readLong());
|
|
|
} else {
|