You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by el...@apache.org on 2012/01/16 23:42:47 UTC

svn commit: r1232203 - in /hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/protocol/ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/main/java/org/apache/hadoop/hdfs/tools/offlineEdi...

Author: eli
Date: Mon Jan 16 22:42:46 2012
New Revision: 1232203

URL: http://svn.apache.org/viewvc?rev=1232203&view=rev
Log:
HDFS-362. svn merge -c 1171945 from trunk

Modified:
    hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
    hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
    hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
    hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoaderCurrent.java
    hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
    hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java

Modified: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1232203&r1=1232202&r2=1232203&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Mon Jan 16 22:42:46 2012
@@ -85,6 +85,10 @@ Release 0.23.1 - UNRELEASED
 
     HDFS-2788. HdfsServerConstants#DN_KEEPALIVE_TIMEOUT is dead code (eli)
 
+    HDFS-362.  FSEditLog should not writes long and short as UTF8, and should
+    not use ArrayWritable for writing non-array items.  (Uma Maheswara Rao G
+    via szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-2130. Switch default checksum to CRC32C. (todd)

Modified: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java?rev=1232203&r1=1232202&r2=1232203&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java Mon Jan 16 22:42:46 2012
@@ -80,7 +80,9 @@ public class LayoutVersion {
     FEDERATION(-35, "Support for namenode federation"),
     LEASE_REASSIGNMENT(-36, "Support for persisting lease holder reassignment"),
     STORED_TXIDS(-37, "Transaction IDs are stored in edits log and image files"),
-    TXID_BASED_LAYOUT(-38, "File names in NN Storage are based on transaction IDs");
+    TXID_BASED_LAYOUT(-38, "File names in NN Storage are based on transaction IDs"), 
+    EDITLOG_OP_OPTIMIZATION(-39,
+        "Use LongWritable and ShortWritable directly instead of ArrayWritable of UTF8");
     
     final int lv;
     final int ancestorLV;

Modified: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java?rev=1232203&r1=1232202&r2=1232203&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java Mon Jan 16 22:42:46 2012
@@ -40,7 +40,6 @@ import static org.apache.hadoop.hdfs.ser
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
@@ -190,19 +189,17 @@ public abstract class FSEditLogOp {
 
     @Override 
     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);
       permissions.write(out);
 
       if (this.opCode == OP_ADD) {
-        new DeprecatedUTF8(clientName).write(out);
-        new DeprecatedUTF8(clientMachine).write(out);
+        FSImageSerialization.writeString(clientName,out);
+        FSImageSerialization.writeString(clientMachine,out);
       }
     }
 
@@ -211,25 +208,43 @@ public abstract class FSEditLogOp {
         throws IOException {
       // versions > 0 support per file 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||
           -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."  +
                               " logVersion is " + logVersion +
                               " but writables.length is " +
                               length + ". ");
       }
       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)) {
-        this.atime = readLong(in);
+        if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+          this.atime = FSImageSerialization.readLong(in);
+        } else {
+          this.atime = readLong(in);
+        }
       } else {
         this.atime = 0;
       }
       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 {
         this.blockSize = 0;
       }
@@ -333,15 +348,19 @@ public abstract class FSEditLogOp {
 
     @Override 
     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
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
       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);
+      }
     }
   }
 
@@ -377,32 +396,45 @@ public abstract class FSEditLogOp {
 
     @Override 
     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;
-      info[idx++] = new DeprecatedUTF8(trg);
       for(int i=0; i<srcs.length; i++) {
         info[idx++] = new DeprecatedUTF8(srcs[i]);
       }
-      info[idx] = toLogLong(timestamp);
       new ArrayWritable(DeprecatedUTF8.class, info).write(out);
+
+      FSImageSerialization.writeLong(timestamp, out);
     }
 
     @Override
     void readFields(DataInputStream in, int logVersion)
         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);
-      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];
       for(int i=0; i<srcSize;i++) {
         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);
+      }
     }
   }
 
@@ -438,24 +470,28 @@ public abstract class FSEditLogOp {
 
     @Override 
     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
     void readFields(DataInputStream in, int logVersion)
         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.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);
+      }
     }
   }
 
@@ -485,22 +521,25 @@ public abstract class FSEditLogOp {
 
     @Override 
     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
     void readFields(DataInputStream in, int logVersion)
         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.timestamp = readLong(in);
+      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.timestamp = FSImageSerialization.readLong(in);
+      } else {
+        this.timestamp = readLong(in);
+      }
     }
   }
 
@@ -536,12 +575,9 @@ public abstract class FSEditLogOp {
 
     @Override 
     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);
     }
     
@@ -549,20 +585,32 @@ public abstract class FSEditLogOp {
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
 
-      this.length = in.readInt();
+      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.length = in.readInt();
+      }
       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. "
                               + "Mkdir operation.");
       }
       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.
       // However, currently this is not being updated/used because of
       // performance reasons.
       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) {
@@ -592,13 +640,13 @@ public abstract class FSEditLogOp {
     
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      new LongWritable(genStamp).write(out);
+      FSImageSerialization.writeLong(genStamp, out);
     }
     
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
-      this.genStamp = in.readLong();
+      this.genStamp = FSImageSerialization.readLong(in);
     }
   }
 
@@ -676,7 +724,7 @@ public abstract class FSEditLogOp {
 
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      new DeprecatedUTF8(src).write(out);
+      FSImageSerialization.writeString(src, out);
       permissions.write(out);
      }
  
@@ -719,11 +767,9 @@ public abstract class FSEditLogOp {
 
     @Override 
     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
@@ -757,7 +803,7 @@ public abstract class FSEditLogOp {
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
       this.src = FSImageSerialization.readString(in);
-      this.nsQuota = readLongWritable(in);
+      this.nsQuota = FSImageSerialization.readLong(in);
     }
   }
 
@@ -816,17 +862,17 @@ public abstract class FSEditLogOp {
 
     @Override 
     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
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
       this.src = FSImageSerialization.readString(in);
-      this.nsQuota = readLongWritable(in);
-      this.dsQuota = readLongWritable(in);
+      this.nsQuota = FSImageSerialization.readLong(in);
+      this.dsQuota = FSImageSerialization.readLong(in);
     }
   }
 
@@ -862,24 +908,29 @@ public abstract class FSEditLogOp {
 
     @Override 
     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
     void readFields(DataInputStream in, int logVersion)
         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.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);
+      }
     }
   }
 
@@ -927,28 +978,33 @@ public abstract class FSEditLogOp {
 
     @Override 
     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);
     }
 
     @Override
     void readFields(DataInputStream in, int logVersion)
         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.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);
     }
   }
@@ -991,25 +1047,29 @@ public abstract class FSEditLogOp {
 
     @Override 
     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);
     }
 
     @Override
     void readFields(DataInputStream in, int logVersion)
         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.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);
     }
 
@@ -1066,9 +1126,9 @@ public abstract class FSEditLogOp {
 
     @Override 
     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
@@ -1107,7 +1167,7 @@ public abstract class FSEditLogOp {
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
       token.write(out);
-      toLogLong(expiryTime).write(out);
+      FSImageSerialization.writeLong(expiryTime, out);
     }
 
     @Override
@@ -1115,7 +1175,11 @@ public abstract class FSEditLogOp {
         throws IOException {
       this.token = new DelegationTokenIdentifier();
       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);
+      }
     }
   }
 
@@ -1146,7 +1210,7 @@ public abstract class FSEditLogOp {
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
       token.write(out);
-      toLogLong(expiryTime).write(out);
+      FSImageSerialization.writeLong(expiryTime, out);
     }
 
     @Override
@@ -1154,7 +1218,11 @@ public abstract class FSEditLogOp {
         throws IOException {
       this.token = new DelegationTokenIdentifier();
       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);
+      }
     }
   }
 
@@ -1269,14 +1337,6 @@ public abstract class FSEditLogOp {
     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
    * fields in the block were blockid and length.
@@ -1312,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
    */

Modified: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java?rev=1232203&r1=1232202&r2=1232203&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java Mon Jan 16 22:42:46 2012
@@ -36,6 +36,8 @@ import org.apache.hadoop.hdfs.server.blo
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 
@@ -72,6 +74,8 @@ public class FSImageSerialization {
    */
   static private final class TLData {
     final DeprecatedUTF8 U_STR = new DeprecatedUTF8();
+    final ShortWritable U_SHORT = new ShortWritable();
+    final LongWritable U_LONG = new LongWritable();
     final FsPermission FILE_PERM = new FsPermission((short) 0);
   }
 
@@ -215,7 +219,35 @@ public class FSImageSerialization {
     ustr.write(out);
   }
 
+  
+  /** read the long value */
+  static long readLong(DataInputStream in) throws IOException {
+    LongWritable ustr = TL_DATA.get().U_LONG;
+    ustr.readFields(in);
+    return ustr.get();
+  }
+
+  /** write the long value */
+  static void writeLong(long value, DataOutputStream out) throws IOException {
+    LongWritable uLong = TL_DATA.get().U_LONG;
+    uLong.set(value);
+    uLong.write(out);
+  }
 
+  /** read short value */
+  static short readShort(DataInputStream in) throws IOException {
+    ShortWritable uShort = TL_DATA.get().U_SHORT;
+    uShort.readFields(in);
+    return uShort.get();
+  }
+
+  /** write short value */
+  static void writeShort(short value, DataOutputStream out) throws IOException {
+    ShortWritable uShort = TL_DATA.get().U_SHORT;
+    uShort.set(value);
+    uShort.write(out);
+  }
+  
   // Same comments apply for this method as for readString()
   @SuppressWarnings("deprecation")
   public static byte[] readBytes(DataInputStream in) throws IOException {

Modified: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoaderCurrent.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoaderCurrent.java?rev=1232203&r1=1232202&r2=1232203&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoaderCurrent.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoaderCurrent.java Mon Jan 16 22:42:46 2012
@@ -41,7 +41,7 @@ import static org.apache.hadoop.hdfs.too
 class EditsLoaderCurrent implements EditsLoader {
 
   private static int[] supportedVersions = { -18, -19, -20, -21, -22, -23, -24,
-      -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38};
+      -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39};
 
   private EditsVisitor v;
   private int editsVersion = 0;
@@ -102,20 +102,29 @@ class EditsLoaderCurrent implements Edit
   private void visit_OP_ADD_or_OP_CLOSE(FSEditLogOpCodes editsOpCode)
     throws IOException {
     visitTxId();
-
-    IntToken opAddLength = v.visitInt(EditsElement.LENGTH);
-    // this happens if the edits is not properly ended (-1 op code),
-    // it is padded at the end with all zeros, OP_ADD is zero so
-    // without this check we would treat all zeros as empty OP_ADD)
-    if(opAddLength.value == 0) {
-      throw new IOException("OpCode " + editsOpCode +
-        " has zero length (corrupted edits)");
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      IntToken opAddLength = v.visitInt(EditsElement.LENGTH);
+      // this happens if the edits is not properly ended (-1 op code),
+      // it is padded at the end with all zeros, OP_ADD is zero so
+      // without this check we would treat all zeros as empty OP_ADD)
+      if (opAddLength.value == 0) {
+        throw new IOException("OpCode " + editsOpCode
+            + " has zero length (corrupted edits)");
+      }
     }
+    
     v.visitStringUTF8(EditsElement.PATH);
-    v.visitStringUTF8(EditsElement.REPLICATION);
-    v.visitStringUTF8(EditsElement.MTIME);
-    v.visitStringUTF8(EditsElement.ATIME);
-    v.visitStringUTF8(EditsElement.BLOCKSIZE);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitShort(EditsElement.REPLICATION);
+      v.visitLong(EditsElement.MTIME);
+      v.visitLong(EditsElement.ATIME);
+      v.visitLong(EditsElement.BLOCKSIZE);
+    } else {
+      v.visitStringUTF8(EditsElement.REPLICATION);
+      v.visitStringUTF8(EditsElement.MTIME);
+      v.visitStringUTF8(EditsElement.ATIME);
+      v.visitStringUTF8(EditsElement.BLOCKSIZE);
+    }
     // now read blocks
     IntToken numBlocksToken = v.visitInt(EditsElement.NUMBLOCKS);
     for (int i = 0; i < numBlocksToken.value; i++) {
@@ -146,11 +155,16 @@ class EditsLoaderCurrent implements Edit
    */
   private void visit_OP_RENAME_OLD() throws IOException {
     visitTxId();
-
-    v.visitInt(        EditsElement.LENGTH);
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitInt(EditsElement.LENGTH);
+    }
     v.visitStringUTF8( EditsElement.SOURCE);
     v.visitStringUTF8( EditsElement.DESTINATION);
-    v.visitStringUTF8( EditsElement.TIMESTAMP);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.TIMESTAMP);
+    } else {
+      v.visitStringUTF8(EditsElement.TIMESTAMP);
+    }
   }
 
   /**
@@ -158,10 +172,15 @@ class EditsLoaderCurrent implements Edit
    */
   private void visit_OP_DELETE() throws IOException {
     visitTxId();
-
-    v.visitInt(        EditsElement.LENGTH);
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitInt(EditsElement.LENGTH);
+    }
     v.visitStringUTF8( EditsElement.PATH);
-    v.visitStringUTF8( EditsElement.TIMESTAMP);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.TIMESTAMP);
+    } else {
+      v.visitStringUTF8(EditsElement.TIMESTAMP);
+    }
   }
 
   /**
@@ -169,11 +188,17 @@ class EditsLoaderCurrent implements Edit
    */
   private void visit_OP_MKDIR() throws IOException {
     visitTxId();
-
-    v.visitInt(        EditsElement.LENGTH);
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitInt(EditsElement.LENGTH);
+    }
     v.visitStringUTF8( EditsElement.PATH);
-    v.visitStringUTF8( EditsElement.TIMESTAMP);
-    v.visitStringUTF8( EditsElement.ATIME);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.TIMESTAMP);
+      v.visitLong(EditsElement.ATIME);
+    } else {
+      v.visitStringUTF8(EditsElement.TIMESTAMP);
+      v.visitStringUTF8(EditsElement.ATIME);
+    }
     // PERMISSION_STATUS
     v.visitEnclosingElement( EditsElement.PERMISSION_STATUS);
 
@@ -191,7 +216,11 @@ class EditsLoaderCurrent implements Edit
     visitTxId();
 
     v.visitStringUTF8(EditsElement.PATH);
-    v.visitStringUTF8(EditsElement.REPLICATION);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitShort(EditsElement.REPLICATION);
+    } else {
+      v.visitStringUTF8(EditsElement.REPLICATION);
+    }
   }
 
   /**
@@ -229,11 +258,17 @@ class EditsLoaderCurrent implements Edit
    */
   private void visit_OP_TIMES() throws IOException {
     visitTxId();
-
-    v.visitInt(        EditsElement.LENGTH);
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitInt(EditsElement.LENGTH);
+    }
     v.visitStringUTF8( EditsElement.PATH);
-    v.visitStringUTF8( EditsElement.MTIME);
-    v.visitStringUTF8( EditsElement.ATIME);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.MTIME);
+      v.visitLong(EditsElement.ATIME);
+    } else {
+      v.visitStringUTF8(EditsElement.MTIME);
+      v.visitStringUTF8(EditsElement.ATIME);
+    }
   }
 
   /**
@@ -252,11 +287,16 @@ class EditsLoaderCurrent implements Edit
    */
   private void visit_OP_RENAME() throws IOException {
     visitTxId();
-
-    v.visitInt(           EditsElement.LENGTH);
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitInt(EditsElement.LENGTH);
+    }
     v.visitStringUTF8(    EditsElement.SOURCE);
     v.visitStringUTF8(    EditsElement.DESTINATION);
-    v.visitStringUTF8(    EditsElement.TIMESTAMP);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.TIMESTAMP);
+    } else {
+      v.visitStringUTF8(EditsElement.TIMESTAMP);
+    }
     v.visitBytesWritable( EditsElement.RENAME_OPTIONS);
   }
 
@@ -265,15 +305,25 @@ class EditsLoaderCurrent implements Edit
    */
   private void visit_OP_CONCAT_DELETE() throws IOException {
     visitTxId();
-
-    IntToken lengthToken = v.visitInt(EditsElement.LENGTH);
+    int sourceCount = 0;
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      IntToken lengthToken = v.visitInt(EditsElement.LENGTH);
+      sourceCount = lengthToken.value - 2;
+    }
     v.visitStringUTF8(EditsElement.CONCAT_TARGET);
     // all except of CONCAT_TARGET and TIMESTAMP
-    int sourceCount = lengthToken.value - 2;
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      IntToken lengthToken = v.visitInt(EditsElement.LENGTH);
+      sourceCount = lengthToken.value;
+    }
     for(int i = 0; i < sourceCount; i++) {
       v.visitStringUTF8(EditsElement.CONCAT_SOURCE);
     }
-    v.visitStringUTF8(EditsElement.TIMESTAMP);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.TIMESTAMP);
+    } else {
+      v.visitStringUTF8(EditsElement.TIMESTAMP);
+    }
   }
 
   /**
@@ -281,12 +331,18 @@ class EditsLoaderCurrent implements Edit
    */
   private void visit_OP_SYMLINK() throws IOException {
     visitTxId();
-
-    v.visitInt(        EditsElement.LENGTH);
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitInt(EditsElement.LENGTH);
+    }
     v.visitStringUTF8( EditsElement.SOURCE);
     v.visitStringUTF8( EditsElement.DESTINATION);
-    v.visitStringUTF8( EditsElement.MTIME);
-    v.visitStringUTF8( EditsElement.ATIME);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.MTIME);
+      v.visitLong(EditsElement.ATIME);
+    } else {
+      v.visitStringUTF8(EditsElement.MTIME);
+      v.visitStringUTF8(EditsElement.ATIME);
+    }
     // PERMISSION_STATUS
     v.visitEnclosingElement(EditsElement.PERMISSION_STATUS);
 
@@ -303,15 +359,19 @@ class EditsLoaderCurrent implements Edit
   private void visit_OP_GET_DELEGATION_TOKEN() throws IOException {
     visitTxId();
     
-      v.visitByte(       EditsElement.T_VERSION);
-      v.visitStringText( EditsElement.T_OWNER);
-      v.visitStringText( EditsElement.T_RENEWER);
-      v.visitStringText( EditsElement.T_REAL_USER);
-      v.visitVLong(      EditsElement.T_ISSUE_DATE);
-      v.visitVLong(      EditsElement.T_MAX_DATE);
-      v.visitVInt(       EditsElement.T_SEQUENCE_NUMBER);
-      v.visitVInt(       EditsElement.T_MASTER_KEY_ID);
-      v.visitStringUTF8( EditsElement.T_EXPIRY_TIME);
+    v.visitByte(EditsElement.T_VERSION);
+    v.visitStringText(EditsElement.T_OWNER);
+    v.visitStringText(EditsElement.T_RENEWER);
+    v.visitStringText(EditsElement.T_REAL_USER);
+    v.visitVLong(EditsElement.T_ISSUE_DATE);
+    v.visitVLong(EditsElement.T_MAX_DATE);
+    v.visitVInt(EditsElement.T_SEQUENCE_NUMBER);
+    v.visitVInt(EditsElement.T_MASTER_KEY_ID);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.T_EXPIRY_TIME);
+    } else {
+      v.visitStringUTF8(EditsElement.T_EXPIRY_TIME);
+    }
   }
 
   /**
@@ -321,15 +381,19 @@ class EditsLoaderCurrent implements Edit
     throws IOException {
     visitTxId();
 
-      v.visitByte(       EditsElement.T_VERSION);
-      v.visitStringText( EditsElement.T_OWNER);
-      v.visitStringText( EditsElement.T_RENEWER);
-      v.visitStringText( EditsElement.T_REAL_USER);
-      v.visitVLong(      EditsElement.T_ISSUE_DATE);
-      v.visitVLong(      EditsElement.T_MAX_DATE);
-      v.visitVInt(       EditsElement.T_SEQUENCE_NUMBER);
-      v.visitVInt(       EditsElement.T_MASTER_KEY_ID);
-      v.visitStringUTF8( EditsElement.T_EXPIRY_TIME);
+    v.visitByte(EditsElement.T_VERSION);
+    v.visitStringText(EditsElement.T_OWNER);
+    v.visitStringText(EditsElement.T_RENEWER);
+    v.visitStringText(EditsElement.T_REAL_USER);
+    v.visitVLong(EditsElement.T_ISSUE_DATE);
+    v.visitVLong(EditsElement.T_MAX_DATE);
+    v.visitVInt(EditsElement.T_SEQUENCE_NUMBER);
+    v.visitVInt(EditsElement.T_MASTER_KEY_ID);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.T_EXPIRY_TIME);
+    } else {
+      v.visitStringUTF8(EditsElement.T_EXPIRY_TIME);
+    }
   }
 
   /**

Modified: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java?rev=1232203&r1=1232202&r2=1232203&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java Mon Jan 16 22:42:46 2012
@@ -122,7 +122,7 @@ class ImageLoaderCurrent implements Imag
   protected final DateFormat dateFormat = 
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
   private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
-      -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38};
+      -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39};
   private int imageVersion = 0;
 
   /* (non-Javadoc)

Modified: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java?rev=1232203&r1=1232202&r2=1232203&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java Mon Jan 16 22:42:46 2012
@@ -17,7 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.junit.Assert.*;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.FileInputStream;