Browse Source

HDFS-4962. Use enum for nfs constants. Contributed by Tsz Wo (Nicholas) SZE.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1501851 13f79535-47bb-0310-9956-ffa450edef68
Jing Zhao 12 years ago
parent
commit
58d75576c4
21 changed files with 263 additions and 227 deletions
  1. 25 8
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountInterface.java
  2. 3 3
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountResponse.java
  3. 58 30
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Constant.java
  4. 11 16
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcAcceptedReply.java
  5. 6 5
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcCall.java
  6. 7 11
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcDeniedReply.java
  7. 22 7
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcMessage.java
  8. 7 11
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcReply.java
  9. 15 20
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/PortmapInterface.java
  10. 2 2
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java
  11. 5 4
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java
  12. 2 2
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcDeniedReply.java
  13. 7 13
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcMessage.java
  14. 2 2
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcReply.java
  15. 9 8
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
  16. 4 3
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
  17. 27 36
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
  18. 19 16
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java
  19. 2 2
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java
  20. 28 28
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java
  21. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

+ 25 - 8
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountInterface.java

@@ -26,14 +26,31 @@ import org.apache.hadoop.oncrpc.XDR;
  * requests. See RFC 1094 for more details.
  */
 public interface MountInterface {
-  public static int MNTPROC_NULL = 0;
-  public static int MNTPROC_MNT = 1;
-  public static int MNTPROC_DUMP = 2;
-  public static int MNTPROC_UMNT = 3;
-  public static int MNTPROC_UMNTALL = 4;
-  public static int MNTPROC_EXPORT = 5;
-  public static int MNTPROC_EXPORTALL = 6;
-  public static int MNTPROC_PATHCONF = 7;
+  /** Mount procedures */
+  public static enum MNTPROC {
+    // the order of the values below are significant.
+    NULL,
+    MNT,
+    DUMP,
+    UMNT,
+    UMNTALL,
+    EXPORT,
+    EXPORTALL,
+    PATHCONF;
+
+    /** @return the int value representing the procedure. */
+    public int getValue() {
+      return ordinal();
+    }
+
+    /** @return the procedure corresponding to the value. */
+    public static MNTPROC fromValue(int value) {
+      if (value < 0 || value >= values().length) {
+        return null;
+      }
+      return values()[value];
+    }
+  }
 
   /** MNTPROC_NULL - Do Nothing */
   public XDR nullOp(XDR out, int xid, InetAddress client);

+ 3 - 3
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountResponse.java

@@ -33,7 +33,7 @@ public class MountResponse {
   private MountResponse() {
   }
   
-  /** Response for RPC call {@link MountInterface#MNTPROC_MNT} */
+  /** Response for RPC call {@link MountInterface.MNTPROC#MNT} */
   public static XDR writeMNTResponse(int status, XDR xdr, int xid,
       byte[] handle) {
     RpcAcceptedReply.voidReply(xdr, xid);
@@ -48,7 +48,7 @@ public class MountResponse {
     return xdr;
   }
 
-  /** Response for RPC call {@link MountInterface#MNTPROC_DUMP} */
+  /** Response for RPC call {@link MountInterface.MNTPROC#DUMP} */
   public static XDR writeMountList(XDR xdr, int xid, List<MountEntry> mounts) {
     RpcAcceptedReply.voidReply(xdr, xid);
     for (MountEntry mountEntry : mounts) {
@@ -60,7 +60,7 @@ public class MountResponse {
     return xdr;
   }
 
-  /** Response for RPC call {@link MountInterface#MNTPROC_EXPORT} */
+  /** Response for RPC call {@link MountInterface.MNTPROC#EXPORT} */
   public static XDR writeExportList(XDR xdr, int xid, List<String> exports) {
     RpcAcceptedReply.voidReply(xdr, xid);
     for (String export : exports) {

+ 58 - 30
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Constant.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.nfs.nfs3;
 
+
 /**
  * Some constants for NFSv3
  */
@@ -34,28 +35,58 @@ public class Nfs3Constant {
   public final static int VERSION = 3;
   
   // The procedures
-  public final static int NFSPROC3_NULL = 0;
-  public final static int NFSPROC3_GETATTR = 1;
-  public final static int NFSPROC3_SETATTR = 2;
-  public final static int NFSPROC3_LOOKUP = 3;
-  public final static int NFSPROC3_ACCESS = 4;
-  public final static int NFSPROC3_READLINK = 5;
-  public final static int NFSPROC3_READ = 6;
-  public final static int NFSPROC3_WRITE = 7;
-  public final static int NFSPROC3_CREATE = 8;
-  public final static int NFSPROC3_MKDIR = 9;
-  public final static int NFSPROC3_SYMLINK = 10;
-  public final static int NFSPROC3_MKNOD = 11;
-  public final static int NFSPROC3_REMOVE = 12;
-  public final static int NFSPROC3_RMDIR = 13;
-  public final static int NFSPROC3_RENAME = 14;
-  public final static int NFSPROC3_LINK = 15;
-  public final static int NFSPROC3_READDIR = 16;
-  public final static int NFSPROC3_READDIRPLUS = 17;
-  public final static int NFSPROC3_FSSTAT = 18;
-  public final static int NFSPROC3_FSINFO = 19;
-  public final static int NFSPROC3_PATHCONF = 20;
-  public final static int NFSPROC3_COMMIT = 21;
+  public static enum NFSPROC3 {
+    // the order of the values below are significant.
+    NULL,
+    GETATTR,
+    SETATTR,
+    LOOKUP,
+    ACCESS,
+    READLINK,
+    READ,
+    WRITE,
+    CREATE(false),
+    MKDIR(false),
+    SYMLINK(false),
+    MKNOD(false),
+    REMOVE(false),
+    RMDIR(false),
+    RENAME(false),
+    LINK(false),
+    READDIR,
+    READDIRPLUS,
+    FSSTAT,
+    FSINFO,
+    PATHCONF,
+    COMMIT;
+
+    private final boolean isIdempotent;
+
+    private NFSPROC3(boolean isIdempotent) {
+      this.isIdempotent = isIdempotent;
+    }
+
+    private NFSPROC3() {
+      this(true);
+    }
+
+    public boolean isIdempotent() {
+      return isIdempotent;
+    }
+
+    /** @return the int value representing the procedure. */
+    public int getValue() {
+      return ordinal();
+    }
+
+    /** @return the procedure corresponding to the value. */
+    public static NFSPROC3 fromValue(int value) {
+      if (value < 0 || value >= values().length) {
+        return null;
+      }
+      return values()[value];
+    }
+  }
   
   // The maximum size in bytes of the opaque file handle.
   public final static int NFS3_FHSIZE = 64;
@@ -125,16 +156,13 @@ public class Nfs3Constant {
 
   /** Write call flavors */
   public enum WriteStableHow {
-    UNSTABLE(0), DATA_SYNC(1), FILE_SYNC(2);
-
-    private final int id;
-
-    WriteStableHow(int id) {
-      this.id = id;
-    }
+    // the order of the values below are significant.
+    UNSTABLE,
+    DATA_SYNC,
+    FILE_SYNC;
 
     public int getValue() {
-      return id;
+      return ordinal();
     }
 
     public static WriteStableHow fromValue(int id) {

+ 11 - 16
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcAcceptedReply.java

@@ -25,39 +25,34 @@ import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
  */
 public class RpcAcceptedReply extends RpcReply {
   public enum AcceptState {
-    SUCCESS(0), /* RPC executed successfully */
-    PROG_UNAVAIL(1), /* remote hasn't exported program */
-    PROG_MISMATCH(2), /* remote can't support version # */
-    PROC_UNAVAIL(3), /* program can't support procedure */
-    GARBAGE_ARGS(4), /* procedure can't decode params */
-    SYSTEM_ERR(5); /* e.g. memory allocation failure */
+    // the order of the values below are significant.
+    SUCCESS, /* RPC executed successfully */
+    PROG_UNAVAIL, /* remote hasn't exported program */
+    PROG_MISMATCH, /* remote can't support version # */
+    PROC_UNAVAIL, /* program can't support procedure */
+    GARBAGE_ARGS, /* procedure can't decode params */
+    SYSTEM_ERR; /* e.g. memory allocation failure */
     
-    private final int value;
-
-    AcceptState(int value) {
-      this.value = value;
-    }
-
     public static AcceptState fromValue(int value) {
       return values()[value];
     }
 
     public int getValue() {
-      return value;
+      return ordinal();
     }
   };
 
   private final RpcAuthInfo verifier;
   private final AcceptState acceptState;
 
-  RpcAcceptedReply(int xid, int messageType, ReplyState state,
+  RpcAcceptedReply(int xid, RpcMessage.Type messageType, ReplyState state,
       RpcAuthInfo verifier, AcceptState acceptState) {
     super(xid, messageType, state);
     this.verifier = verifier;
     this.acceptState = acceptState;
   }
 
-  public static RpcAcceptedReply read(int xid, int messageType,
+  public static RpcAcceptedReply read(int xid, RpcMessage.Type messageType,
       ReplyState replyState, XDR xdr) {
     RpcAuthInfo verifier = RpcAuthInfo.read(xdr);
     AcceptState acceptState = AcceptState.fromValue(xdr.readInt());
@@ -79,7 +74,7 @@ public class RpcAcceptedReply extends RpcReply {
   
   public static XDR voidReply(XDR xdr, int xid, AcceptState acceptState) {
     xdr.writeInt(xid);
-    xdr.writeInt(RpcMessage.RPC_REPLY);
+    xdr.writeInt(RpcMessage.Type.RPC_REPLY.getValue());
     xdr.writeInt(ReplyState.MSG_ACCEPTED.getValue());
     xdr.writeInt(AuthFlavor.AUTH_NONE.getValue());
     xdr.writeVariableOpaque(new byte[0]);

+ 6 - 5
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcCall.java

@@ -33,7 +33,7 @@ public class RpcCall extends RpcMessage {
   private final RpcAuthInfo credential;
   private final RpcAuthInfo verifier;
 
-  protected RpcCall(int xid, int messageType, int rpcVersion, int program,
+  protected RpcCall(int xid, RpcMessage.Type messageType, int rpcVersion, int program,
       int version, int procedure, RpcAuthInfo credential, RpcAuthInfo verifier) {
     super(xid, messageType);
     this.rpcVersion = rpcVersion;
@@ -56,7 +56,7 @@ public class RpcCall extends RpcMessage {
   }
   
   public void validate() {
-    validateMessageType(RPC_CALL);
+    validateMessageType(RpcMessage.Type.RPC_CALL);
     validateRpcVersion();
     // Validate other members
     // Throw exception if validation fails
@@ -88,7 +88,8 @@ public class RpcCall extends RpcMessage {
   }
   
   public static RpcCall read(XDR xdr) {
-    return new RpcCall(xdr.readInt(), xdr.readInt(), xdr.readInt(), xdr.readInt(),
+    return new RpcCall(xdr.readInt(), RpcMessage.Type.fromValue(xdr.readInt()),
+        xdr.readInt(), xdr.readInt(),
         xdr.readInt(), xdr.readInt(), RpcAuthInfo.read(xdr),
         RpcAuthInfo.read(xdr));
   }
@@ -96,7 +97,7 @@ public class RpcCall extends RpcMessage {
   public static void write(XDR out, int xid, int program, int progVersion,
       int procedure) {
     out.writeInt(xid);
-    out.writeInt(RpcMessage.RPC_CALL);
+    out.writeInt(RpcMessage.Type.RPC_CALL.getValue());
     out.writeInt(2);
     out.writeInt(program);
     out.writeInt(progVersion);
@@ -105,7 +106,7 @@ public class RpcCall extends RpcMessage {
   
   @Override
   public String toString() {
-    return String.format("Xid:%d, messageType:%d, rpcVersion:%d, program:%d,"
+    return String.format("Xid:%d, messageType:%s, rpcVersion:%d, program:%d,"
         + " version:%d, procedure:%d, credential:%s, verifier:%s", xid,
         messageType, rpcVersion, program, version, procedure,
         credential.toString(), verifier.toString());

+ 7 - 11
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcDeniedReply.java

@@ -25,16 +25,12 @@ import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
  */
 public class RpcDeniedReply extends RpcReply {
   public enum RejectState {
-    RPC_MISMATCH(0), AUTH_ERROR(1);
-
-    private final int value;
-
-    RejectState(int value) {
-      this.value = value;
-    }
+    // the order of the values below are significant.
+    RPC_MISMATCH,
+    AUTH_ERROR;
 
     int getValue() {
-      return value;
+      return ordinal();
     }
 
     static RejectState fromValue(int value) {
@@ -44,13 +40,13 @@ public class RpcDeniedReply extends RpcReply {
 
   private final RejectState rejectState;
 
-  RpcDeniedReply(int xid, int messageType, ReplyState replyState,
+  RpcDeniedReply(int xid, RpcMessage.Type messageType, ReplyState replyState,
       RejectState rejectState) {
     super(xid, messageType, replyState);
     this.rejectState = rejectState;
   }
 
-  public static RpcDeniedReply read(int xid, int messageType,
+  public static RpcDeniedReply read(int xid, RpcMessage.Type messageType,
       ReplyState replyState, XDR xdr) {
     RejectState rejectState = RejectState.fromValue(xdr.readInt());
     return new RpcDeniedReply(xid, messageType, replyState, rejectState);
@@ -70,7 +66,7 @@ public class RpcDeniedReply extends RpcReply {
   public static XDR voidReply(XDR xdr, int xid, ReplyState msgAccepted,
       RejectState rejectState) {
     xdr.writeInt(xid);
-    xdr.writeInt(RpcMessage.RPC_REPLY);
+    xdr.writeInt(RpcMessage.Type.RPC_REPLY.getValue());
     xdr.writeInt(msgAccepted.getValue());
     xdr.writeInt(AuthFlavor.AUTH_NONE.getValue());
     xdr.writeVariableOpaque(new byte[0]);

+ 22 - 7
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcMessage.java

@@ -21,14 +21,29 @@ package org.apache.hadoop.oncrpc;
  * Represent an RPC message as defined in RFC 1831.
  */
 public abstract class RpcMessage {
-  public static final int RPC_CALL = 0;
-  public static final int RPC_REPLY = 1;
+  /** Message type */
+  public static enum Type {
+    // the order of the values below are significant.
+    RPC_CALL,
+    RPC_REPLY;
+    
+    public int getValue() {
+      return ordinal();
+    }
+
+    public static Type fromValue(int value) {
+      if (value < 0 || value >= values().length) {
+        return null;
+      }
+      return values()[value];
+    }
+  }
 
   protected final int xid;
-  protected final int messageType;
+  protected final Type messageType;
   
-  RpcMessage(int xid, int messageType) {
-    if (messageType != RPC_CALL && messageType != RPC_REPLY) {
+  RpcMessage(int xid, Type messageType) {
+    if (messageType != Type.RPC_CALL && messageType != Type.RPC_REPLY) {
       throw new IllegalArgumentException("Invalid message type " + messageType);
     }
     this.xid = xid;
@@ -39,11 +54,11 @@ public abstract class RpcMessage {
     return xid;
   }
 
-  public int getMessageType() {
+  public Type getMessageType() {
     return messageType;
   }
   
-  protected void validateMessageType(int expected) {
+  protected void validateMessageType(Type expected) {
     if (expected != messageType) {
       throw new IllegalArgumentException("Message type is expected to be "
           + expected + " but got " + messageType);

+ 7 - 11
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcReply.java

@@ -23,16 +23,12 @@ package org.apache.hadoop.oncrpc;
 public abstract class RpcReply extends RpcMessage {
   /** RPC reply_stat as defined in RFC 1831 */
   public enum ReplyState {
-    MSG_ACCEPTED(0),
-    MSG_DENIED(1);
-    
-    private final int value;
-    ReplyState(int value) {
-      this.value = value;
-    }
+    // the order of the values below are significant.
+    MSG_ACCEPTED,
+    MSG_DENIED;
     
     int getValue() {
-      return value;
+      return ordinal();
     }
     
     public static ReplyState fromValue(int value) {
@@ -42,15 +38,15 @@ public abstract class RpcReply extends RpcMessage {
   
   private final ReplyState state;
   
-  RpcReply(int xid, int messageType, ReplyState state) {
+  RpcReply(int xid, RpcMessage.Type messageType, ReplyState state) {
     super(xid, messageType);
     this.state = state;
-    validateMessageType(RPC_REPLY);
+    validateMessageType(RpcMessage.Type.RPC_REPLY);
   }
 
   public static RpcReply read(XDR xdr) {
     int xid = xdr.readInt();
-    int messageType = xdr.readInt();
+    final Type messageType = Type.fromValue(xdr.readInt());
     ReplyState stat = ReplyState.fromValue(xdr.readInt());
     switch (stat) {
     case MSG_ACCEPTED:

+ 15 - 20
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/PortmapInterface.java

@@ -25,28 +25,23 @@ import org.apache.hadoop.oncrpc.XDR;
  */
 public interface PortmapInterface {
   public enum Procedure {
-    PMAPPROC_NULL(0),
-    PMAPPROC_SET(1),
-    PMAPPROC_UNSET(2),
-    PMAPPROC_GETPORT(3),
-    PMAPPROC_DUMP(4),
-    PMAPPROC_CALLIT(5),
-    PMAPPROC_GETTIME(6),
-    PMAPPROC_UADDR2TADDR(7),
-    PMAPPROC_TADDR2UADDR(8),
-    PMAPPROC_GETVERSADDR(9),
-    PMAPPROC_INDIRECT(10),
-    PMAPPROC_GETADDRLIST(11),
-    PMAPPROC_GETSTAT(12);
-    
-    private final int value;
-    
-    Procedure(int value) {
-      this.value = value;
-    }
+    // the order of the values below are significant.
+    PMAPPROC_NULL,
+    PMAPPROC_SET,
+    PMAPPROC_UNSET,
+    PMAPPROC_GETPORT,
+    PMAPPROC_DUMP,
+    PMAPPROC_CALLIT,
+    PMAPPROC_GETTIME,
+    PMAPPROC_UADDR2TADDR,
+    PMAPPROC_TADDR2UADDR,
+    PMAPPROC_GETVERSADDR,
+    PMAPPROC_INDIRECT,
+    PMAPPROC_GETADDRLIST,
+    PMAPPROC_GETSTAT;
     
     public int getValue() {
-      return value;
+      return ordinal();
     }
     
     public static Procedure fromValue(int value) {

+ 2 - 2
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcAcceptedReply.java

@@ -46,10 +46,10 @@ public class TestRpcAcceptedReply {
   @Test
   public void testConstructor() {
     RpcAuthInfo verifier = new RpcAuthInfo(AuthFlavor.AUTH_NONE, new byte[0]);
-    RpcAcceptedReply reply = new RpcAcceptedReply(0, RpcMessage.RPC_REPLY,
+    RpcAcceptedReply reply = new RpcAcceptedReply(0, RpcMessage.Type.RPC_REPLY,
         ReplyState.MSG_ACCEPTED, verifier, AcceptState.SUCCESS);
     assertEquals(0, reply.getXid());
-    assertEquals(RpcMessage.RPC_REPLY, reply.getMessageType());
+    assertEquals(RpcMessage.Type.RPC_REPLY, reply.getMessageType());
     assertEquals(ReplyState.MSG_ACCEPTED, reply.getState());
     assertEquals(verifier, reply.getVerifier());
     assertEquals(AcceptState.SUCCESS, reply.getAcceptState());

+ 5 - 4
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcCall.java

@@ -34,9 +34,10 @@ public class TestRpcCall {
     int program = 2;
     int version = 3;
     int procedure = 4;
-    RpcCall call = new RpcCall(0, RpcMessage.RPC_CALL, rpcVersion, program, version, procedure, credential, verifier);
+    RpcCall call = new RpcCall(0, RpcMessage.Type.RPC_CALL, rpcVersion, program,
+        version, procedure, credential, verifier);
     assertEquals(0, call.getXid());
-    assertEquals(RpcMessage.RPC_CALL, call.getMessageType());
+    assertEquals(RpcMessage.Type.RPC_CALL, call.getMessageType());
     assertEquals(rpcVersion, call.getRpcVersion());
     assertEquals(program, call.getProgram());
     assertEquals(version, call.getVersion());
@@ -48,12 +49,12 @@ public class TestRpcCall {
   @Test(expected=IllegalArgumentException.class)
   public void testInvalidRpcVersion() {
     int invalidRpcVersion = 3;
-    new RpcCall(0, RpcMessage.RPC_CALL, invalidRpcVersion, 2, 3, 4, null, null);
+    new RpcCall(0, RpcMessage.Type.RPC_CALL, invalidRpcVersion, 2, 3, 4, null, null);
   }
   
   @Test(expected=IllegalArgumentException.class)
   public void testInvalidRpcMessageType() {
-    int invalidMessageType = 3; // Message typ is not RpcMessage.RPC_CALL
+    RpcMessage.Type invalidMessageType = RpcMessage.Type.RPC_REPLY; // Message typ is not RpcMessage.RPC_CALL
     new RpcCall(0, invalidMessageType, RpcCall.RPC_VERSION, 2, 3, 4, null, null);
   }
 }

+ 2 - 2
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcDeniedReply.java

@@ -39,12 +39,12 @@ public class TestRpcDeniedReply {
   
   @Test
   public void testConstructor() {
-    RpcDeniedReply reply = new RpcDeniedReply(0, RpcMessage.RPC_REPLY,
+    RpcDeniedReply reply = new RpcDeniedReply(0, RpcMessage.Type.RPC_REPLY,
         ReplyState.MSG_ACCEPTED, RejectState.AUTH_ERROR) {
       // Anonymous class
     };
     Assert.assertEquals(0, reply.getXid());
-    Assert.assertEquals(RpcMessage.RPC_REPLY, reply.getMessageType());
+    Assert.assertEquals(RpcMessage.Type.RPC_REPLY, reply.getMessageType());
     Assert.assertEquals(ReplyState.MSG_ACCEPTED, reply.getState());
     Assert.assertEquals(RejectState.AUTH_ERROR, reply.getRejectState());
   }

+ 7 - 13
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcMessage.java

@@ -24,34 +24,28 @@ import org.junit.Test;
  * Test for {@link RpcMessage}
  */
 public class TestRpcMessage {
-  private RpcMessage getRpcMessage(int xid, int msgType) {
+  private RpcMessage getRpcMessage(int xid, RpcMessage.Type msgType) {
     return new RpcMessage(xid, msgType) {
       // Anonymous class
     };
   }
   
-  @Test(expected=IllegalArgumentException.class)
-  public void testInvalidMessageType() {
-    int invalidMsgType = 2; // valid values are 0 and 1
-    getRpcMessage(0, invalidMsgType);
-  }
-  
   @Test
   public void testRpcMessage() {
-    RpcMessage msg = getRpcMessage(0, RpcMessage.RPC_CALL);
+    RpcMessage msg = getRpcMessage(0, RpcMessage.Type.RPC_CALL);
     Assert.assertEquals(0, msg.getXid());
-    Assert.assertEquals(RpcMessage.RPC_CALL, msg.getMessageType());
+    Assert.assertEquals(RpcMessage.Type.RPC_CALL, msg.getMessageType());
   }
   
   @Test
   public void testValidateMessage() {
-    RpcMessage msg = getRpcMessage(0, RpcMessage.RPC_CALL);
-    msg.validateMessageType(RpcMessage.RPC_CALL);
+    RpcMessage msg = getRpcMessage(0, RpcMessage.Type.RPC_CALL);
+    msg.validateMessageType(RpcMessage.Type.RPC_CALL);
   }
   
   @Test(expected = IllegalArgumentException.class)
   public void testValidateMessageException() {
-    RpcMessage msg = getRpcMessage(0, RpcMessage.RPC_CALL);
-    msg.validateMessageType(RpcMessage.RPC_REPLY);
+    RpcMessage msg = getRpcMessage(0, RpcMessage.Type.RPC_CALL);
+    msg.validateMessageType(RpcMessage.Type.RPC_REPLY);
   }
 }

+ 2 - 2
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestRpcReply.java

@@ -39,11 +39,11 @@ public class TestRpcReply {
   
   @Test
   public void testRpcReply() {
-    RpcReply reply = new RpcReply(0, 1, ReplyState.MSG_ACCEPTED) {
+    RpcReply reply = new RpcReply(0, RpcMessage.Type.RPC_REPLY, ReplyState.MSG_ACCEPTED) {
       // Anonymous class
     };
     Assert.assertEquals(0, reply.getXid());
-    Assert.assertEquals(1, reply.getMessageType());
+    Assert.assertEquals(RpcMessage.Type.RPC_REPLY, reply.getMessageType());
     Assert.assertEquals(ReplyState.MSG_ACCEPTED, reply.getState());
   }
 }

+ 9 - 8
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java

@@ -154,24 +154,25 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
   @Override
   public XDR handleInternal(RpcCall rpcCall, XDR xdr, XDR out,
       InetAddress client, Channel channel) {
-    int procedure = rpcCall.getProcedure();
+    final MNTPROC mntproc = MNTPROC.fromValue(rpcCall.getProcedure());
     int xid = rpcCall.getXid();
-    if (procedure == MNTPROC_NULL) {
+    if (mntproc == MNTPROC.NULL) {
       out = nullOp(out, xid, client);
-    } else if (procedure == MNTPROC_MNT) {
+    } else if (mntproc == MNTPROC.MNT) {
       out = mnt(xdr, out, xid, client);
-    } else if (procedure == MNTPROC_DUMP) {
+    } else if (mntproc == MNTPROC.DUMP) {
       out = dump(out, xid, client);
-    } else if (procedure == MNTPROC_UMNT) {      
+    } else if (mntproc == MNTPROC.UMNT) {      
       out = umnt(xdr, out, xid, client);
-    } else if (procedure == MNTPROC_UMNTALL) {
+    } else if (mntproc == MNTPROC.UMNTALL) {
       umntall(out, xid, client);
-    } else if (procedure == MNTPROC_EXPORT) {
+    } else if (mntproc == MNTPROC.EXPORT) {
       out = MountResponse.writeExportList(out, xid, exports);
     } else {
       // Invalid procedure
       RpcAcceptedReply.voidReply(out, xid,
-          RpcAcceptedReply.AcceptState.PROC_UNAVAIL);    }  
+          RpcAcceptedReply.AcceptState.PROC_UNAVAIL);
+    }  
     return out;
   }
   

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
+import org.apache.hadoop.hdfs.nfs.nfs3.WriteCtx.DataState;
 import org.apache.hadoop.io.BytesWritable.Comparator;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
@@ -345,7 +346,7 @@ class OpenFileCtx {
           + " nextOffset:" + nextOffset);
       WriteCtx writeCtx = new WriteCtx(request.getHandle(),
           request.getOffset(), request.getCount(), request.getStableHow(),
-          request.getData().array(), channel, xid, false, WriteCtx.NO_DUMP);
+          request.getData().array(), channel, xid, false, DataState.NO_DUMP);
       addWrite(writeCtx);
       
       // Create an async task and change openFileCtx status to indicate async
@@ -373,7 +374,7 @@ class OpenFileCtx {
           + nextOffset);
       WriteCtx writeCtx = new WriteCtx(request.getHandle(),
           request.getOffset(), request.getCount(), request.getStableHow(),
-          request.getData().array(), channel, xid, false, WriteCtx.ALLOW_DUMP);
+          request.getData().array(), channel, xid, false, DataState.ALLOW_DUMP);
       addWrite(writeCtx);
 
       // Check if need to dump some pending requests to file
@@ -693,7 +694,7 @@ class OpenFileCtx {
       nextOffset = fos.getPos();
 
       // Reduce memory occupation size if request was allowed dumped
-      if (writeCtx.getDataState() == WriteCtx.ALLOW_DUMP) {
+      if (writeCtx.getDataState() == DataState.ALLOW_DUMP) {
         updateNonSequentialWriteInMemory(-count);
       }
       

+ 27 - 36
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.nfs.NfsTime;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
 import org.apache.hadoop.nfs.nfs3.IdUserGroup;
 import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant.NFSPROC3;
 import org.apache.hadoop.nfs.nfs3.Nfs3Constant.WriteStableHow;
 import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
 import org.apache.hadoop.nfs.nfs3.Nfs3Interface;
@@ -1530,12 +1531,12 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   @Override
   public XDR handleInternal(RpcCall rpcCall, final XDR xdr, XDR out,
       InetAddress client, Channel channel) {
-    int procedure = rpcCall.getProcedure();
+    final NFSPROC3 nfsproc3 = NFSPROC3.fromValue(rpcCall.getProcedure());
     int xid = rpcCall.getXid();
     RpcAuthSys authSys = null;
     
     // Ignore auth only for NFSPROC3_NULL, especially for Linux clients.
-    if (procedure != Nfs3Constant.NFSPROC3_NULL) {
+    if (nfsproc3 != NFSPROC3.NULL) {
       if (rpcCall.getCredential().getFlavor() != AuthFlavor.AUTH_SYS) {
         LOG.info("Wrong RPC AUTH flavor, "
             + rpcCall.getCredential().getFlavor() + " is not AUTH_SYS.");
@@ -1549,49 +1550,49 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
     
     NFS3Response response = null;
-    if (procedure == Nfs3Constant.NFSPROC3_NULL) {
+    if (nfsproc3 == NFSPROC3.NULL) {
       response = nullProcedure();
-    } else if (procedure == Nfs3Constant.NFSPROC3_GETATTR) {
+    } else if (nfsproc3 == NFSPROC3.GETATTR) {
       response = getattr(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_SETATTR) {
+    } else if (nfsproc3 == NFSPROC3.SETATTR) {
       response = setattr(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_LOOKUP) {
+    } else if (nfsproc3 == NFSPROC3.LOOKUP) {
       response = lookup(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_ACCESS) {
+    } else if (nfsproc3 == NFSPROC3.ACCESS) {
       response = access(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_READLINK) {
+    } else if (nfsproc3 == NFSPROC3.READLINK) {
       response = readlink(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_READ) {
+    } else if (nfsproc3 == NFSPROC3.READ) {
       response = read(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_WRITE) {
+    } else if (nfsproc3 == NFSPROC3.WRITE) {
       response = write(xdr, channel, xid, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_CREATE) {
+    } else if (nfsproc3 == NFSPROC3.CREATE) {
       response = create(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_MKDIR) {      
+    } else if (nfsproc3 == NFSPROC3.MKDIR) {      
       response = mkdir(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_SYMLINK) {
+    } else if (nfsproc3 == NFSPROC3.SYMLINK) {
       response = symlink(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_MKNOD) {
+    } else if (nfsproc3 == NFSPROC3.MKNOD) {
       response = mknod(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_REMOVE) {
+    } else if (nfsproc3 == NFSPROC3.REMOVE) {
       response = remove(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_RMDIR) {
+    } else if (nfsproc3 == NFSPROC3.RMDIR) {
       response = rmdir(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_RENAME) {
+    } else if (nfsproc3 == NFSPROC3.RENAME) {
       response = rename(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_LINK) {
+    } else if (nfsproc3 == NFSPROC3.LINK) {
       response = link(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_READDIR) {
+    } else if (nfsproc3 == NFSPROC3.READDIR) {
       response = readdir(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_READDIRPLUS) {
+    } else if (nfsproc3 == NFSPROC3.READDIRPLUS) {
       response = readdirplus(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_FSSTAT) {
+    } else if (nfsproc3 == NFSPROC3.FSSTAT) {
       response = fsstat(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_FSINFO) {
+    } else if (nfsproc3 == NFSPROC3.FSINFO) {
       response = fsinfo(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_PATHCONF) {
+    } else if (nfsproc3 == NFSPROC3.PATHCONF) {
       response = pathconf(xdr, authSys);
-    } else if (procedure == Nfs3Constant.NFSPROC3_COMMIT) {
+    } else if (nfsproc3 == NFSPROC3.COMMIT) {
       response = commit(xdr, authSys);
     } else {
       // Invalid procedure
@@ -1606,17 +1607,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   
   @Override
   protected boolean isIdempotent(RpcCall call) {
-    return isIdempotent(call.getProcedure());
-  }
-  
-  public static boolean isIdempotent(int procedure) {
-    return !(procedure == Nfs3Constant.NFSPROC3_CREATE
-        || procedure == Nfs3Constant.NFSPROC3_REMOVE
-        || procedure == Nfs3Constant.NFSPROC3_MKDIR
-        || procedure == Nfs3Constant.NFSPROC3_MKNOD
-        || procedure == Nfs3Constant.NFSPROC3_LINK
-        || procedure == Nfs3Constant.NFSPROC3_RMDIR
-        || procedure == Nfs3Constant.NFSPROC3_SYMLINK
-        || procedure == Nfs3Constant.NFSPROC3_RENAME);
+    final NFSPROC3 nfsproc3 = NFSPROC3.fromValue(call.getProcedure()); 
+    return nfsproc3 == null || nfsproc3.isIdempotent();
   }
 }

+ 19 - 16
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java

@@ -33,6 +33,17 @@ import org.jboss.netty.channel.Channel;
  */
 class WriteCtx {
   public static final Log LOG = LogFactory.getLog(WriteCtx.class);
+  
+  /**
+   * In memory write data has 3 states. ALLOW_DUMP: not sequential write, still
+   * wait for prerequisit writes. NO_DUMP: sequential write, no need to dump
+   * since it will be written to HDFS soon. DUMPED: already dumped to a file.
+   */
+  public static enum DataState {
+    ALLOW_DUMP,
+    NO_DUMP,
+    DUMPED;
+  }
 
   private final FileHandle handle;
   private final long offset;
@@ -43,22 +54,14 @@ class WriteCtx {
   private final Channel channel;
   private final int xid;
   private boolean replied;
-  
-  /**
-   * In memory write data has 3 states. ALLOW_DUMP: not sequential write, still
-   * wait for prerequisit writes. NO_DUMP: sequential write, no need to dump
-   * since it will be written to HDFS soon. DUMPED: already dumped to a file.
-   */
-  public final static int ALLOW_DUMP = 0;
-  public final static int NO_DUMP = 1;
-  public final static int DUMPED = 2;
-  private int dataState;
 
-  public int getDataState() {
+  private DataState dataState;
+
+  public DataState getDataState() {
     return dataState;
   }
 
-  public void setDataState(int dataState) {
+  public void setDataState(DataState dataState) {
     this.dataState = dataState;
   }
 
@@ -68,7 +71,7 @@ class WriteCtx {
   // Return the dumped data size
   public long dumpData(FileOutputStream dumpOut, RandomAccessFile raf)
       throws IOException {
-    if (dataState != ALLOW_DUMP) {
+    if (dataState != DataState.ALLOW_DUMP) {
       if (LOG.isTraceEnabled()) {
         LOG.trace("No need to dump with status(replied,dataState):" + "("
             + replied + "," + dataState + ")");
@@ -82,7 +85,7 @@ class WriteCtx {
       LOG.debug("After dump, new dumpFileOffset:" + dumpFileOffset);
     }
     data = null;
-    dataState = DUMPED;
+    dataState = DataState.DUMPED;
     return count;
   }
 
@@ -103,7 +106,7 @@ class WriteCtx {
   }
 
   public byte[] getData() throws IOException {
-    if (dataState != DUMPED) {
+    if (dataState != DataState.DUMPED) {
       if (data == null) {
         throw new IOException("Data is not dumpted but has null:" + this);
       }
@@ -140,7 +143,7 @@ class WriteCtx {
   }
   
   WriteCtx(FileHandle handle, long offset, int count, WriteStableHow stableHow,
-      byte[] data, Channel channel, int xid, boolean replied, int dataState) {
+      byte[] data, Channel channel, int xid, boolean replied, DataState dataState) {
     this.handle = handle;
     this.offset = offset;
     this.count = count;

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java

@@ -59,7 +59,7 @@ public class TestOutOfOrderWrite {
   static XDR create() {
     XDR request = new XDR();
     RpcCall.write(request, 0x8000004c, Nfs3Constant.PROGRAM,
-        Nfs3Constant.VERSION, Nfs3Constant.NFSPROC3_CREATE);
+        Nfs3Constant.VERSION, Nfs3Constant.NFSPROC3.CREATE.getValue());
 
     // credentials
     request.writeInt(0); // auth null
@@ -79,7 +79,7 @@ public class TestOutOfOrderWrite {
       byte[] data) {
     XDR request = new XDR();
     RpcCall.write(request, xid, Nfs3Constant.PROGRAM, Nfs3Constant.VERSION,
-        Nfs3Constant.NFSPROC3_WRITE);
+        Nfs3Constant.NFSPROC3.WRITE.getValue());
 
     // credentials
     request.writeInt(0); // auth null

+ 28 - 28
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java

@@ -28,38 +28,38 @@ import org.junit.Test;
 public class TestRpcProgramNfs3 {
   @Test(timeout=1000)
   public void testIdempotent() {
-    int[][] procedures = {
-        { Nfs3Constant.NFSPROC3_NULL, 1 },
-        { Nfs3Constant.NFSPROC3_GETATTR, 1 },
-        { Nfs3Constant.NFSPROC3_SETATTR, 1 },
-        { Nfs3Constant.NFSPROC3_LOOKUP, 1 },
-        { Nfs3Constant.NFSPROC3_ACCESS, 1 },
-        { Nfs3Constant.NFSPROC3_READLINK, 1 },
-        { Nfs3Constant.NFSPROC3_READ, 1 },
-        { Nfs3Constant.NFSPROC3_WRITE, 1 },
-        { Nfs3Constant.NFSPROC3_CREATE, 0 },
-        { Nfs3Constant.NFSPROC3_MKDIR, 0 },
-        { Nfs3Constant.NFSPROC3_SYMLINK, 0 },
-        { Nfs3Constant.NFSPROC3_MKNOD, 0 },
-        { Nfs3Constant.NFSPROC3_REMOVE, 0 },
-        { Nfs3Constant.NFSPROC3_RMDIR, 0 },
-        { Nfs3Constant.NFSPROC3_RENAME, 0 },
-        { Nfs3Constant.NFSPROC3_LINK, 0 },
-        { Nfs3Constant.NFSPROC3_READDIR, 1 },
-        { Nfs3Constant.NFSPROC3_READDIRPLUS, 1 },
-        { Nfs3Constant.NFSPROC3_FSSTAT, 1 },
-        { Nfs3Constant.NFSPROC3_FSINFO, 1 },
-        { Nfs3Constant.NFSPROC3_PATHCONF, 1 },
-        { Nfs3Constant.NFSPROC3_COMMIT, 1 } };
-    for (int[] procedure : procedures) {
-      boolean idempotent = procedure[1] == 1;
-      int proc = procedure[0];
+    Object[][] procedures = {
+        { Nfs3Constant.NFSPROC3.NULL, 1 },
+        { Nfs3Constant.NFSPROC3.GETATTR, 1 },
+        { Nfs3Constant.NFSPROC3.SETATTR, 1 },
+        { Nfs3Constant.NFSPROC3.LOOKUP, 1 },
+        { Nfs3Constant.NFSPROC3.ACCESS, 1 },
+        { Nfs3Constant.NFSPROC3.READLINK, 1 },
+        { Nfs3Constant.NFSPROC3.READ, 1 },
+        { Nfs3Constant.NFSPROC3.WRITE, 1 },
+        { Nfs3Constant.NFSPROC3.CREATE, 0 },
+        { Nfs3Constant.NFSPROC3.MKDIR, 0 },
+        { Nfs3Constant.NFSPROC3.SYMLINK, 0 },
+        { Nfs3Constant.NFSPROC3.MKNOD, 0 },
+        { Nfs3Constant.NFSPROC3.REMOVE, 0 },
+        { Nfs3Constant.NFSPROC3.RMDIR, 0 },
+        { Nfs3Constant.NFSPROC3.RENAME, 0 },
+        { Nfs3Constant.NFSPROC3.LINK, 0 },
+        { Nfs3Constant.NFSPROC3.READDIR, 1 },
+        { Nfs3Constant.NFSPROC3.READDIRPLUS, 1 },
+        { Nfs3Constant.NFSPROC3.FSSTAT, 1 },
+        { Nfs3Constant.NFSPROC3.FSINFO, 1 },
+        { Nfs3Constant.NFSPROC3.PATHCONF, 1 },
+        { Nfs3Constant.NFSPROC3.COMMIT, 1 } };
+    for (Object[] procedure : procedures) {
+      boolean idempotent = procedure[1].equals(Integer.valueOf(1));
+      Nfs3Constant.NFSPROC3 proc = (Nfs3Constant.NFSPROC3)procedure[0];
       if (idempotent) {
         Assert.assertTrue(("Procedure " + proc + " should be idempotent"),
-            RpcProgramNfs3.isIdempotent(proc));
+            proc.isIdempotent());
       } else {
         Assert.assertFalse(("Procedure " + proc + " should be non-idempotent"),
-            RpcProgramNfs3.isIdempotent(proc));
+            proc.isIdempotent());
       }
     }
   }

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -14,6 +14,8 @@ Trunk (Unreleased)
 
     HDFS-4762 Provide HDFS based NFSv3 and Mountd implementation (brandonli)
 
+    HDFS-4962 Use enum for nfs constants (Nicholas SZE via jing9)
+
   IMPROVEMENTS
 
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.