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 sz...@apache.org on 2013/12/18 03:50:02 UTC

svn commit: r1551813 - in /hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/test/java/org/apache/hadoop/hdfs/server/namenode/

Author: szetszwo
Date: Wed Dec 18 02:50:01 2013
New Revision: 1551813

URL: http://svn.apache.org/r1551813
Log:
svn merge -c 1551812 from trunk for HDFS-5674. Editlog code cleanup: remove @SuppressWarnings("deprecation") in FSEditLogOp; change FSEditLogOpCodes.fromByte(..) to be more efficient; and change Some fields in FSEditLog to final.

Modified:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/   (props changed)
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/   (props changed)
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java

Propchange: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs:r1551812

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1551813&r1=1551812&r2=1551813&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Wed Dec 18 02:50:01 2013
@@ -174,6 +174,10 @@ Release 2.4.0 - UNRELEASED
     HDFS-5629. Support HTTPS in JournalNode and SecondaryNameNode. 
     (Haohui Mai via jing9)
 
+    HDFS-5674. Editlog code cleanup: remove @SuppressWarnings("deprecation") in
+    FSEditLogOp; change FSEditLogOpCodes.fromByte(..) to be more efficient; and
+    change Some fields in FSEditLog to final.  (szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)

Propchange: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java:r1551812

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java?rev=1551813&r1=1551812&r2=1551813&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java Wed Dec 18 02:50:01 2013
@@ -155,10 +155,10 @@ public class FSEditLog implements LogsPu
   private long totalTimeTransactions;  // total time for all transactions
   private NameNodeMetrics metrics;
 
-  private NNStorage storage;
-  private Configuration conf;
+  private final NNStorage storage;
+  private final Configuration conf;
   
-  private List<URI> editsDirs;
+  private final List<URI> editsDirs;
 
   private ThreadLocal<OpInstanceCache> cache =
       new ThreadLocal<OpInstanceCache>() {
@@ -171,7 +171,7 @@ public class FSEditLog implements LogsPu
   /**
    * The edit directories that are shared between primary and secondary.
    */
-  private List<URI> sharedEditsDirs;
+  private final List<URI> sharedEditsDirs;
 
   private static class TransactionId {
     public long txid;
@@ -198,10 +198,6 @@ public class FSEditLog implements LogsPu
    * @param editsDirs List of journals to use
    */
   FSEditLog(Configuration conf, NNStorage storage, List<URI> editsDirs) {
-    init(conf, storage, editsDirs);
-  }
-  
-  private void init(Configuration conf, NNStorage storage, List<URI> editsDirs) {
     isSyncRunning = false;
     this.conf = conf;
     this.storage = storage;

Modified: hadoop/common/branches/branch-2/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-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java?rev=1551813&r1=1551812&r2=1551813&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java Wed Dec 18 02:50:01 2013
@@ -108,11 +108,10 @@ import com.google.common.base.Preconditi
 @InterfaceStability.Unstable
 public abstract class FSEditLogOp {
   public final FSEditLogOpCodes opCode;
-  long txid;
+  long txid = HdfsConstants.INVALID_TXID;
   byte[] rpcClientId = RpcConstants.DUMMY_CLIENT_ID;
   int rpcCallId = RpcConstants.INVALID_CALL_ID;
 
-  @SuppressWarnings("deprecation")
   final public static class OpInstanceCache {
     private EnumMap<FSEditLogOpCodes, FSEditLogOp> inst = 
         new EnumMap<FSEditLogOpCodes, FSEditLogOp>(FSEditLogOpCodes.class);
@@ -137,13 +136,10 @@ public abstract class FSEditLogOp {
       inst.put(OP_REASSIGN_LEASE, new ReassignLeaseOp());
       inst.put(OP_GET_DELEGATION_TOKEN, new GetDelegationTokenOp());
       inst.put(OP_RENEW_DELEGATION_TOKEN, new RenewDelegationTokenOp());
-      inst.put(OP_CANCEL_DELEGATION_TOKEN, 
-                    new CancelDelegationTokenOp());
+      inst.put(OP_CANCEL_DELEGATION_TOKEN, new CancelDelegationTokenOp());
       inst.put(OP_UPDATE_MASTER_KEY, new UpdateMasterKeyOp());
-      inst.put(OP_START_LOG_SEGMENT,
-                    new LogSegmentOp(OP_START_LOG_SEGMENT));
-      inst.put(OP_END_LOG_SEGMENT,
-                    new LogSegmentOp(OP_END_LOG_SEGMENT));
+      inst.put(OP_START_LOG_SEGMENT, new LogSegmentOp(OP_START_LOG_SEGMENT));
+      inst.put(OP_END_LOG_SEGMENT, new LogSegmentOp(OP_END_LOG_SEGMENT));
       inst.put(OP_UPDATE_BLOCKS, new UpdateBlocksOp());
 
       inst.put(OP_ALLOW_SNAPSHOT, new AllowSnapshotOp());
@@ -166,7 +162,6 @@ public abstract class FSEditLogOp {
    */
   private FSEditLogOp(FSEditLogOpCodes opCode) {
     this.opCode = opCode;
-    this.txid = HdfsConstants.INVALID_TXID;
   }
 
   public long getTransactionId() {
@@ -2941,9 +2936,7 @@ public abstract class FSEditLogOp {
      * @param in The stream to read from.
      * @param logVersion The version of the data coming from the stream.
      */
-    @SuppressWarnings("deprecation")
-    public Reader(DataInputStream in, StreamLimiter limiter,
-        int logVersion) {
+    public Reader(DataInputStream in, StreamLimiter limiter, int logVersion) {
       this.logVersion = logVersion;
       if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) {
         this.checksum = new PureJavaCrc32();

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java?rev=1551813&r1=1551812&r2=1551813&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java Wed Dec 18 02:50:01 2013
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.util.Map;
-import java.util.HashMap;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
@@ -30,7 +27,6 @@ import org.apache.hadoop.classification.
 @InterfaceStability.Unstable
 public enum FSEditLogOpCodes {
   // last op code in file
-  OP_INVALID                    ((byte) -1),
   OP_ADD                        ((byte)  0),
   OP_RENAME_OLD                 ((byte)  1), // deprecated operation
   OP_DELETE                     ((byte)  2),
@@ -63,8 +59,12 @@ public enum FSEditLogOpCodes {
   OP_ALLOW_SNAPSHOT             ((byte) 29),
   OP_DISALLOW_SNAPSHOT          ((byte) 30),
   OP_SET_GENSTAMP_V2            ((byte) 31),
-  OP_ALLOCATE_BLOCK_ID          ((byte) 32);
-  private byte opCode;
+  OP_ALLOCATE_BLOCK_ID          ((byte) 32),
+
+  // Note that fromByte(..) depends on OP_INVALID being at the last position.  
+  OP_INVALID                    ((byte) -1);
+
+  private final byte opCode;
 
   /**
    * Constructor
@@ -84,14 +84,7 @@ public enum FSEditLogOpCodes {
     return opCode;
   }
 
-  private static final Map<Byte, FSEditLogOpCodes> byteToEnum =
-    new HashMap<Byte, FSEditLogOpCodes>();
-
-  static {
-    // initialize byte to enum map
-    for(FSEditLogOpCodes opCode : values())
-      byteToEnum.put(opCode.getOpCode(), opCode);
-  }
+  private static final FSEditLogOpCodes[] VALUES = FSEditLogOpCodes.values();
 
   /**
    * Converts byte to FSEditLogOpCodes enum value
@@ -100,6 +93,12 @@ public enum FSEditLogOpCodes {
    * @return enum with byte value of opCode
    */
   public static FSEditLogOpCodes fromByte(byte opCode) {
-    return byteToEnum.get(opCode);
+    if (opCode == -1) {
+      return OP_INVALID;
+    }
+    if (opCode >= 0 && opCode < OP_INVALID.ordinal()) {
+      return VALUES[opCode];
+    }
+    return null;
   }
 }

Modified: hadoop/common/branches/branch-2/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-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java?rev=1551813&r1=1551812&r2=1551813&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java Wed Dec 18 02:50:01 2013
@@ -31,6 +31,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.channels.FileChannel;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.SortedMap;
 
@@ -383,4 +384,33 @@ public class TestFSEditLogLoader {
     assertTrue(!validation.hasCorruptHeader());
     assertEquals(HdfsConstants.INVALID_TXID, validation.getEndTxId());
   }
+
+  private static final Map<Byte, FSEditLogOpCodes> byteToEnum =
+      new HashMap<Byte, FSEditLogOpCodes>();
+  static {
+    for(FSEditLogOpCodes opCode : FSEditLogOpCodes.values()) {
+      byteToEnum.put(opCode.getOpCode(), opCode);
+    }
+  }
+
+  private static FSEditLogOpCodes fromByte(byte opCode) {
+    return byteToEnum.get(opCode);
+  }
+
+  @Test
+  public void testFSEditLogOpCodes() throws IOException {
+    //try all codes
+    for(FSEditLogOpCodes c : FSEditLogOpCodes.values()) {
+      final byte code = c.getOpCode();
+      assertEquals("c=" + c + ", code=" + code,
+          c, FSEditLogOpCodes.fromByte(code));
+    }
+
+    //try all byte values
+    for(int b = 0; b < (1 << Byte.SIZE); b++) {
+      final byte code = (byte)b;
+      assertEquals("b=" + b + ", code=" + code,
+          fromByte(code), FSEditLogOpCodes.fromByte(code));
+    }
+  }
 }