|
@@ -30,19 +30,16 @@ import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
|
|
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
|
|
-import org.apache.hadoop.hdfs.server.common.Storage;
|
|
|
|
import org.apache.hadoop.util.PureJavaCrc32;
|
|
import org.apache.hadoop.util.PureJavaCrc32;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.*;
|
|
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.*;
|
|
import org.apache.hadoop.security.token.delegation.DelegationKey;
|
|
import org.apache.hadoop.security.token.delegation.DelegationKey;
|
|
import org.apache.hadoop.io.BytesWritable;
|
|
import org.apache.hadoop.io.BytesWritable;
|
|
import org.apache.hadoop.io.DataOutputBuffer;
|
|
import org.apache.hadoop.io.DataOutputBuffer;
|
|
-import org.apache.hadoop.io.LongWritable;
|
|
|
|
import org.apache.hadoop.io.ArrayWritable;
|
|
import org.apache.hadoop.io.ArrayWritable;
|
|
import org.apache.hadoop.io.Writable;
|
|
import org.apache.hadoop.io.Writable;
|
|
import org.apache.hadoop.io.WritableFactories;
|
|
import org.apache.hadoop.io.WritableFactories;
|
|
@@ -192,19 +189,17 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
- DeprecatedUTF8 nameReplicationPair[] = new DeprecatedUTF8[] {
|
|
|
|
- new DeprecatedUTF8(path),
|
|
|
|
- toLogReplication(replication),
|
|
|
|
- toLogLong(mtime),
|
|
|
|
- toLogLong(atime),
|
|
|
|
- toLogLong(blockSize)};
|
|
|
|
- new ArrayWritable(DeprecatedUTF8.class, nameReplicationPair).write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeString(path, out);
|
|
|
|
+ FSImageSerialization.writeShort(replication, out);
|
|
|
|
+ FSImageSerialization.writeLong(mtime, out);
|
|
|
|
+ FSImageSerialization.writeLong(atime, out);
|
|
|
|
+ FSImageSerialization.writeLong(blockSize, out);
|
|
new ArrayWritable(Block.class, blocks).write(out);
|
|
new ArrayWritable(Block.class, blocks).write(out);
|
|
permissions.write(out);
|
|
permissions.write(out);
|
|
|
|
|
|
if (this.opCode == OP_ADD) {
|
|
if (this.opCode == OP_ADD) {
|
|
- new DeprecatedUTF8(clientName).write(out);
|
|
|
|
- new DeprecatedUTF8(clientMachine).write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeString(clientName,out);
|
|
|
|
+ FSImageSerialization.writeString(clientMachine,out);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -213,25 +208,43 @@ public abstract class FSEditLogOp {
|
|
throws IOException {
|
|
throws IOException {
|
|
// versions > 0 support per file replication
|
|
// versions > 0 support per file replication
|
|
// get name and replication
|
|
// get name and replication
|
|
- this.length = in.readInt();
|
|
|
|
|
|
+ if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.length = in.readInt();
|
|
|
|
+ }
|
|
if (-7 == logVersion && length != 3||
|
|
if (-7 == logVersion && length != 3||
|
|
-17 < logVersion && logVersion < -7 && length != 4 ||
|
|
-17 < logVersion && logVersion < -7 && length != 4 ||
|
|
- logVersion <= -17 && length != 5) {
|
|
|
|
|
|
+ (logVersion <= -17 && length != 5 && !LayoutVersion.supports(
|
|
|
|
+ Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) {
|
|
throw new IOException("Incorrect data format." +
|
|
throw new IOException("Incorrect data format." +
|
|
" logVersion is " + logVersion +
|
|
" logVersion is " + logVersion +
|
|
" but writables.length is " +
|
|
" but writables.length is " +
|
|
length + ". ");
|
|
length + ". ");
|
|
}
|
|
}
|
|
this.path = FSImageSerialization.readString(in);
|
|
this.path = FSImageSerialization.readString(in);
|
|
- this.replication = readShort(in);
|
|
|
|
- this.mtime = readLong(in);
|
|
|
|
|
|
+
|
|
|
|
+ if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.replication = FSImageSerialization.readShort(in);
|
|
|
|
+ this.mtime = FSImageSerialization.readLong(in);
|
|
|
|
+ } else {
|
|
|
|
+ this.replication = readShort(in);
|
|
|
|
+ this.mtime = readLong(in);
|
|
|
|
+ }
|
|
|
|
+
|
|
if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, logVersion)) {
|
|
if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, logVersion)) {
|
|
- this.atime = readLong(in);
|
|
|
|
|
|
+ if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.atime = FSImageSerialization.readLong(in);
|
|
|
|
+ } else {
|
|
|
|
+ this.atime = readLong(in);
|
|
|
|
+ }
|
|
} else {
|
|
} else {
|
|
this.atime = 0;
|
|
this.atime = 0;
|
|
}
|
|
}
|
|
if (logVersion < -7) {
|
|
if (logVersion < -7) {
|
|
- this.blockSize = readLong(in);
|
|
|
|
|
|
+ if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.blockSize = FSImageSerialization.readLong(in);
|
|
|
|
+ } else {
|
|
|
|
+ this.blockSize = readLong(in);
|
|
|
|
+ }
|
|
} else {
|
|
} else {
|
|
this.blockSize = 0;
|
|
this.blockSize = 0;
|
|
}
|
|
}
|
|
@@ -335,15 +348,19 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
- new DeprecatedUTF8(path).write(out);
|
|
|
|
- new DeprecatedUTF8(Short.toString(replication)).write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeString(path, out);
|
|
|
|
+ FSImageSerialization.writeShort(replication, out);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void readFields(DataInputStream in, int logVersion)
|
|
void readFields(DataInputStream in, int logVersion)
|
|
throws IOException {
|
|
throws IOException {
|
|
this.path = FSImageSerialization.readString(in);
|
|
this.path = FSImageSerialization.readString(in);
|
|
- this.replication = readShort(in);
|
|
|
|
|
|
+ if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.replication = FSImageSerialization.readShort(in);
|
|
|
|
+ } else {
|
|
|
|
+ this.replication = readShort(in);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -379,32 +396,45 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
- int size = 1 + srcs.length + 1; // trg, srcs, timestamp
|
|
|
|
- DeprecatedUTF8 info[] = new DeprecatedUTF8[size];
|
|
|
|
|
|
+ FSImageSerialization.writeString(trg, out);
|
|
|
|
+
|
|
|
|
+ DeprecatedUTF8 info[] = new DeprecatedUTF8[srcs.length];
|
|
int idx = 0;
|
|
int idx = 0;
|
|
- info[idx++] = new DeprecatedUTF8(trg);
|
|
|
|
for(int i=0; i<srcs.length; i++) {
|
|
for(int i=0; i<srcs.length; i++) {
|
|
info[idx++] = new DeprecatedUTF8(srcs[i]);
|
|
info[idx++] = new DeprecatedUTF8(srcs[i]);
|
|
}
|
|
}
|
|
- info[idx] = toLogLong(timestamp);
|
|
|
|
new ArrayWritable(DeprecatedUTF8.class, info).write(out);
|
|
new ArrayWritable(DeprecatedUTF8.class, info).write(out);
|
|
|
|
+
|
|
|
|
+ FSImageSerialization.writeLong(timestamp, out);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void readFields(DataInputStream in, int logVersion)
|
|
void readFields(DataInputStream in, int logVersion)
|
|
throws IOException {
|
|
throws IOException {
|
|
- this.length = in.readInt();
|
|
|
|
- if (length < 3) { // trg, srcs.., timestam
|
|
|
|
- throw new IOException("Incorrect data format. "
|
|
|
|
- + "Concat delete operation.");
|
|
|
|
|
|
+ if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.length = in.readInt();
|
|
|
|
+ if (length < 3) { // trg, srcs.., timestamp
|
|
|
|
+ throw new IOException("Incorrect data format. "
|
|
|
|
+ + "Concat delete operation.");
|
|
|
|
+ }
|
|
}
|
|
}
|
|
this.trg = FSImageSerialization.readString(in);
|
|
this.trg = FSImageSerialization.readString(in);
|
|
- int srcSize = this.length - 1 - 1; //trg and timestamp
|
|
|
|
|
|
+ int srcSize = 0;
|
|
|
|
+ if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ srcSize = in.readInt();
|
|
|
|
+ } else {
|
|
|
|
+ srcSize = this.length - 1 - 1; // trg and timestamp
|
|
|
|
+ }
|
|
this.srcs = new String [srcSize];
|
|
this.srcs = new String [srcSize];
|
|
for(int i=0; i<srcSize;i++) {
|
|
for(int i=0; i<srcSize;i++) {
|
|
srcs[i]= FSImageSerialization.readString(in);
|
|
srcs[i]= FSImageSerialization.readString(in);
|
|
}
|
|
}
|
|
- this.timestamp = readLong(in);
|
|
|
|
|
|
+
|
|
|
|
+ if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.timestamp = FSImageSerialization.readLong(in);
|
|
|
|
+ } else {
|
|
|
|
+ this.timestamp = readLong(in);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -440,24 +470,28 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
- DeprecatedUTF8 info[] = new DeprecatedUTF8[] {
|
|
|
|
- new DeprecatedUTF8(src),
|
|
|
|
- new DeprecatedUTF8(dst),
|
|
|
|
- toLogLong(timestamp)};
|
|
|
|
- new ArrayWritable(DeprecatedUTF8.class, info).write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeString(src, out);
|
|
|
|
+ FSImageSerialization.writeString(dst, out);
|
|
|
|
+ FSImageSerialization.writeLong(timestamp, out);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void readFields(DataInputStream in, int logVersion)
|
|
void readFields(DataInputStream in, int logVersion)
|
|
throws IOException {
|
|
throws IOException {
|
|
- this.length = in.readInt();
|
|
|
|
- if (this.length != 3) {
|
|
|
|
- throw new IOException("Incorrect data format. "
|
|
|
|
- + "Old rename operation.");
|
|
|
|
|
|
+ if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.length = in.readInt();
|
|
|
|
+ if (this.length != 3) {
|
|
|
|
+ throw new IOException("Incorrect data format. "
|
|
|
|
+ + "Old rename operation.");
|
|
|
|
+ }
|
|
}
|
|
}
|
|
this.src = FSImageSerialization.readString(in);
|
|
this.src = FSImageSerialization.readString(in);
|
|
this.dst = FSImageSerialization.readString(in);
|
|
this.dst = FSImageSerialization.readString(in);
|
|
- this.timestamp = readLong(in);
|
|
|
|
|
|
+ if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.timestamp = FSImageSerialization.readLong(in);
|
|
|
|
+ } else {
|
|
|
|
+ this.timestamp = readLong(in);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -487,22 +521,25 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
- DeprecatedUTF8 info[] = new DeprecatedUTF8[] {
|
|
|
|
- new DeprecatedUTF8(path),
|
|
|
|
- toLogLong(timestamp)};
|
|
|
|
- new ArrayWritable(DeprecatedUTF8.class, info).write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeString(path, out);
|
|
|
|
+ FSImageSerialization.writeLong(timestamp, out);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void readFields(DataInputStream in, int logVersion)
|
|
void readFields(DataInputStream in, int logVersion)
|
|
throws IOException {
|
|
throws IOException {
|
|
- this.length = in.readInt();
|
|
|
|
- if (this.length != 2) {
|
|
|
|
- throw new IOException("Incorrect data format. "
|
|
|
|
- + "delete operation.");
|
|
|
|
|
|
+ if (!LayoutVersion.supports(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);
|
|
this.path = FSImageSerialization.readString(in);
|
|
- this.timestamp = readLong(in);
|
|
|
|
|
|
+ if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.timestamp = FSImageSerialization.readLong(in);
|
|
|
|
+ } else {
|
|
|
|
+ this.timestamp = readLong(in);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -538,12 +575,9 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
- DeprecatedUTF8 info[] = new DeprecatedUTF8[] {
|
|
|
|
- new DeprecatedUTF8(path),
|
|
|
|
- toLogLong(timestamp), // mtime
|
|
|
|
- toLogLong(timestamp) // atime, unused at this time
|
|
|
|
- };
|
|
|
|
- new ArrayWritable(DeprecatedUTF8.class, info).write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeString(path, out);
|
|
|
|
+ FSImageSerialization.writeLong(timestamp, out); // mtime
|
|
|
|
+ FSImageSerialization.writeLong(timestamp, out); // atime, unused at this
|
|
permissions.write(out);
|
|
permissions.write(out);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -551,20 +585,32 @@ public abstract class FSEditLogOp {
|
|
void readFields(DataInputStream in, int logVersion)
|
|
void readFields(DataInputStream in, int logVersion)
|
|
throws IOException {
|
|
throws IOException {
|
|
|
|
|
|
- this.length = in.readInt();
|
|
|
|
|
|
+ if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.length = in.readInt();
|
|
|
|
+ }
|
|
if (-17 < logVersion && length != 2 ||
|
|
if (-17 < logVersion && length != 2 ||
|
|
- logVersion <= -17 && length != 3) {
|
|
|
|
|
|
+ logVersion <= -17 && length != 3
|
|
|
|
+ && !LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
throw new IOException("Incorrect data format. "
|
|
throw new IOException("Incorrect data format. "
|
|
+ "Mkdir operation.");
|
|
+ "Mkdir operation.");
|
|
}
|
|
}
|
|
this.path = FSImageSerialization.readString(in);
|
|
this.path = FSImageSerialization.readString(in);
|
|
- this.timestamp = readLong(in);
|
|
|
|
|
|
+ if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.timestamp = FSImageSerialization.readLong(in);
|
|
|
|
+ } else {
|
|
|
|
+ this.timestamp = readLong(in);
|
|
|
|
+ }
|
|
|
|
|
|
// The disk format stores atimes for directories as well.
|
|
// The disk format stores atimes for directories as well.
|
|
// However, currently this is not being updated/used because of
|
|
// However, currently this is not being updated/used because of
|
|
// performance reasons.
|
|
// performance reasons.
|
|
if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, logVersion)) {
|
|
if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, logVersion)) {
|
|
- /*unused this.atime = */readLong(in);
|
|
|
|
|
|
+ /* unused this.atime = */
|
|
|
|
+ if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ FSImageSerialization.readLong(in);
|
|
|
|
+ } else {
|
|
|
|
+ readLong(in);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
if (logVersion <= -11) {
|
|
if (logVersion <= -11) {
|
|
@@ -594,13 +640,13 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
- new LongWritable(genStamp).write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeLong(genStamp, out);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void readFields(DataInputStream in, int logVersion)
|
|
void readFields(DataInputStream in, int logVersion)
|
|
throws IOException {
|
|
throws IOException {
|
|
- this.genStamp = in.readLong();
|
|
|
|
|
|
+ this.genStamp = FSImageSerialization.readLong(in);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -678,7 +724,7 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
- new DeprecatedUTF8(src).write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeString(src, out);
|
|
permissions.write(out);
|
|
permissions.write(out);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -721,11 +767,9 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
- DeprecatedUTF8 u = new DeprecatedUTF8(username == null? "": username);
|
|
|
|
- DeprecatedUTF8 g = new DeprecatedUTF8(groupname == null? "": groupname);
|
|
|
|
- new DeprecatedUTF8(src).write(out);
|
|
|
|
- u.write(out);
|
|
|
|
- g.write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeString(src, out);
|
|
|
|
+ FSImageSerialization.writeString(username == null ? "" : username, out);
|
|
|
|
+ FSImageSerialization.writeString(groupname == null ? "" : groupname, out);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -759,7 +803,7 @@ public abstract class FSEditLogOp {
|
|
void readFields(DataInputStream in, int logVersion)
|
|
void readFields(DataInputStream in, int logVersion)
|
|
throws IOException {
|
|
throws IOException {
|
|
this.src = FSImageSerialization.readString(in);
|
|
this.src = FSImageSerialization.readString(in);
|
|
- this.nsQuota = readLongWritable(in);
|
|
|
|
|
|
+ this.nsQuota = FSImageSerialization.readLong(in);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -818,17 +862,17 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
- new DeprecatedUTF8(src).write(out);
|
|
|
|
- new LongWritable(nsQuota).write(out);
|
|
|
|
- new LongWritable(dsQuota).write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeString(src, out);
|
|
|
|
+ FSImageSerialization.writeLong(nsQuota, out);
|
|
|
|
+ FSImageSerialization.writeLong(dsQuota, out);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void readFields(DataInputStream in, int logVersion)
|
|
void readFields(DataInputStream in, int logVersion)
|
|
throws IOException {
|
|
throws IOException {
|
|
this.src = FSImageSerialization.readString(in);
|
|
this.src = FSImageSerialization.readString(in);
|
|
- this.nsQuota = readLongWritable(in);
|
|
|
|
- this.dsQuota = readLongWritable(in);
|
|
|
|
|
|
+ this.nsQuota = FSImageSerialization.readLong(in);
|
|
|
|
+ this.dsQuota = FSImageSerialization.readLong(in);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -864,24 +908,29 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
- DeprecatedUTF8 info[] = new DeprecatedUTF8[] {
|
|
|
|
- new DeprecatedUTF8(path),
|
|
|
|
- toLogLong(mtime),
|
|
|
|
- toLogLong(atime)};
|
|
|
|
- new ArrayWritable(DeprecatedUTF8.class, info).write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeString(path, out);
|
|
|
|
+ FSImageSerialization.writeLong(mtime, out);
|
|
|
|
+ FSImageSerialization.writeLong(atime, out);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void readFields(DataInputStream in, int logVersion)
|
|
void readFields(DataInputStream in, int logVersion)
|
|
throws IOException {
|
|
throws IOException {
|
|
- this.length = in.readInt();
|
|
|
|
- if (length != 3) {
|
|
|
|
- throw new IOException("Incorrect data format. "
|
|
|
|
- + "times operation.");
|
|
|
|
|
|
+ if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.length = in.readInt();
|
|
|
|
+ if (length != 3) {
|
|
|
|
+ throw new IOException("Incorrect data format. " + "times operation.");
|
|
|
|
+ }
|
|
}
|
|
}
|
|
this.path = FSImageSerialization.readString(in);
|
|
this.path = FSImageSerialization.readString(in);
|
|
- this.mtime = readLong(in);
|
|
|
|
- this.atime = readLong(in);
|
|
|
|
|
|
+
|
|
|
|
+ if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.mtime = FSImageSerialization.readLong(in);
|
|
|
|
+ this.atime = FSImageSerialization.readLong(in);
|
|
|
|
+ } else {
|
|
|
|
+ this.mtime = readLong(in);
|
|
|
|
+ this.atime = readLong(in);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -929,28 +978,33 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
- DeprecatedUTF8 info[] = new DeprecatedUTF8[] {
|
|
|
|
- new DeprecatedUTF8(path),
|
|
|
|
- new DeprecatedUTF8(value),
|
|
|
|
- toLogLong(mtime),
|
|
|
|
- toLogLong(atime)};
|
|
|
|
- new ArrayWritable(DeprecatedUTF8.class, info).write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeString(path, out);
|
|
|
|
+ FSImageSerialization.writeString(value, out);
|
|
|
|
+ FSImageSerialization.writeLong(mtime, out);
|
|
|
|
+ FSImageSerialization.writeLong(atime, out);
|
|
permissionStatus.write(out);
|
|
permissionStatus.write(out);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void readFields(DataInputStream in, int logVersion)
|
|
void readFields(DataInputStream in, int logVersion)
|
|
throws IOException {
|
|
throws IOException {
|
|
-
|
|
|
|
- this.length = in.readInt();
|
|
|
|
- if (this.length != 4) {
|
|
|
|
- throw new IOException("Incorrect data format. "
|
|
|
|
- + "symlink operation.");
|
|
|
|
|
|
+ if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.length = in.readInt();
|
|
|
|
+ if (this.length != 4) {
|
|
|
|
+ throw new IOException("Incorrect data format. "
|
|
|
|
+ + "symlink operation.");
|
|
|
|
+ }
|
|
}
|
|
}
|
|
this.path = FSImageSerialization.readString(in);
|
|
this.path = FSImageSerialization.readString(in);
|
|
this.value = FSImageSerialization.readString(in);
|
|
this.value = FSImageSerialization.readString(in);
|
|
- this.mtime = readLong(in);
|
|
|
|
- this.atime = readLong(in);
|
|
|
|
|
|
+
|
|
|
|
+ if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.mtime = FSImageSerialization.readLong(in);
|
|
|
|
+ this.atime = FSImageSerialization.readLong(in);
|
|
|
|
+ } else {
|
|
|
|
+ this.mtime = readLong(in);
|
|
|
|
+ this.atime = readLong(in);
|
|
|
|
+ }
|
|
this.permissionStatus = PermissionStatus.read(in);
|
|
this.permissionStatus = PermissionStatus.read(in);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -993,25 +1047,29 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
- DeprecatedUTF8 info[] = new DeprecatedUTF8[] {
|
|
|
|
- new DeprecatedUTF8(src),
|
|
|
|
- new DeprecatedUTF8(dst),
|
|
|
|
- toLogLong(timestamp)};
|
|
|
|
- new ArrayWritable(DeprecatedUTF8.class, info).write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeString(src, out);
|
|
|
|
+ FSImageSerialization.writeString(dst, out);
|
|
|
|
+ FSImageSerialization.writeLong(timestamp, out);
|
|
toBytesWritable(options).write(out);
|
|
toBytesWritable(options).write(out);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void readFields(DataInputStream in, int logVersion)
|
|
void readFields(DataInputStream in, int logVersion)
|
|
throws IOException {
|
|
throws IOException {
|
|
- this.length = in.readInt();
|
|
|
|
- if (this.length != 3) {
|
|
|
|
- throw new IOException("Incorrect data format. "
|
|
|
|
- + "Rename operation.");
|
|
|
|
|
|
+ if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.length = in.readInt();
|
|
|
|
+ if (this.length != 3) {
|
|
|
|
+ throw new IOException("Incorrect data format. " + "Rename operation.");
|
|
|
|
+ }
|
|
}
|
|
}
|
|
this.src = FSImageSerialization.readString(in);
|
|
this.src = FSImageSerialization.readString(in);
|
|
this.dst = FSImageSerialization.readString(in);
|
|
this.dst = FSImageSerialization.readString(in);
|
|
- this.timestamp = readLong(in);
|
|
|
|
|
|
+
|
|
|
|
+ if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.timestamp = FSImageSerialization.readLong(in);
|
|
|
|
+ } else {
|
|
|
|
+ this.timestamp = readLong(in);
|
|
|
|
+ }
|
|
this.options = readRenameOptions(in);
|
|
this.options = readRenameOptions(in);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1068,9 +1126,9 @@ public abstract class FSEditLogOp {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
- new DeprecatedUTF8(leaseHolder).write(out);
|
|
|
|
- new DeprecatedUTF8(path).write(out);
|
|
|
|
- new DeprecatedUTF8(newHolder).write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeString(leaseHolder, out);
|
|
|
|
+ FSImageSerialization.writeString(path, out);
|
|
|
|
+ FSImageSerialization.writeString(newHolder, out);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -1109,7 +1167,7 @@ public abstract class FSEditLogOp {
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
token.write(out);
|
|
token.write(out);
|
|
- toLogLong(expiryTime).write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeLong(expiryTime, out);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -1117,7 +1175,11 @@ public abstract class FSEditLogOp {
|
|
throws IOException {
|
|
throws IOException {
|
|
this.token = new DelegationTokenIdentifier();
|
|
this.token = new DelegationTokenIdentifier();
|
|
this.token.readFields(in);
|
|
this.token.readFields(in);
|
|
- this.expiryTime = readLong(in);
|
|
|
|
|
|
+ if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.expiryTime = FSImageSerialization.readLong(in);
|
|
|
|
+ } else {
|
|
|
|
+ this.expiryTime = readLong(in);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1148,7 +1210,7 @@ public abstract class FSEditLogOp {
|
|
@Override
|
|
@Override
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
void writeFields(DataOutputStream out) throws IOException {
|
|
token.write(out);
|
|
token.write(out);
|
|
- toLogLong(expiryTime).write(out);
|
|
|
|
|
|
+ FSImageSerialization.writeLong(expiryTime, out);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -1156,7 +1218,11 @@ public abstract class FSEditLogOp {
|
|
throws IOException {
|
|
throws IOException {
|
|
this.token = new DelegationTokenIdentifier();
|
|
this.token = new DelegationTokenIdentifier();
|
|
this.token.readFields(in);
|
|
this.token.readFields(in);
|
|
- this.expiryTime = readLong(in);
|
|
|
|
|
|
+ if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
|
|
|
+ this.expiryTime = FSImageSerialization.readLong(in);
|
|
|
|
+ } else {
|
|
|
|
+ this.expiryTime = readLong(in);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1271,14 +1337,6 @@ public abstract class FSEditLogOp {
|
|
return Long.parseLong(FSImageSerialization.readString(in));
|
|
return Long.parseLong(FSImageSerialization.readString(in));
|
|
}
|
|
}
|
|
|
|
|
|
- static private DeprecatedUTF8 toLogReplication(short replication) {
|
|
|
|
- return new DeprecatedUTF8(Short.toString(replication));
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- static private DeprecatedUTF8 toLogLong(long timestamp) {
|
|
|
|
- return new DeprecatedUTF8(Long.toString(timestamp));
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* A class to read in blocks stored in the old format. The only two
|
|
* A class to read in blocks stored in the old format. The only two
|
|
* fields in the block were blockid and length.
|
|
* fields in the block were blockid and length.
|
|
@@ -1314,17 +1372,6 @@ public abstract class FSEditLogOp {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- // a place holder for reading a long
|
|
|
|
- private static final LongWritable longWritable = new LongWritable();
|
|
|
|
-
|
|
|
|
- /** Read an integer from an input stream */
|
|
|
|
- private static long readLongWritable(DataInputStream in) throws IOException {
|
|
|
|
- synchronized (longWritable) {
|
|
|
|
- longWritable.readFields(in);
|
|
|
|
- return longWritable.get();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Class for writing editlog ops
|
|
* Class for writing editlog ops
|
|
*/
|
|
*/
|