Browse Source

HADOOP-9515. Merging change r1493924 and r1493929 from trunk to branch-2

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1499167 13f79535-47bb-0310-9956-ffa450edef68
Brandon Li 12 years ago
parent
commit
b1d5f9519c
60 changed files with 4270 additions and 0 deletions
  1. 2 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 59 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountEntry.java
  3. 58 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountInterface.java
  4. 74 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountResponse.java
  5. 71 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountdBase.java
  6. 41 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsFileType.java
  7. 88 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsTime.java
  8. 157 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/FileHandle.java
  9. 191 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java
  10. 77 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java
  11. 164 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Constant.java
  12. 217 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3FileAttributes.java
  13. 83 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Interface.java
  14. 162 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Status.java
  15. 31 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/ACCESS3Request.java
  16. 44 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/COMMIT3Request.java
  17. 84 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/CREATE3Request.java
  18. 31 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/FSINFO3Request.java
  19. 31 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/FSSTAT3Request.java
  20. 31 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/GETATTR3Request.java
  21. 58 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/LOOKUP3Request.java
  22. 45 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/MKDIR3Request.java
  23. 31 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/PATHCONF3Request.java
  24. 44 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/READ3Request.java
  25. 50 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/READDIR3Request.java
  26. 56 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/READDIRPLUS3Request.java
  27. 32 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/READLINK3Request.java
  28. 38 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/REMOVE3Request.java
  29. 62 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/RENAME3Request.java
  30. 38 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/RMDIR3Request.java
  31. 49 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/RequestWithHandle.java
  32. 64 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/SETATTR3Request.java
  33. 51 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/SYMLINK3Request.java
  34. 168 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/SetAttr3.java
  35. 78 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/WRITE3Request.java
  36. 55 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/ACCESS3Response.java
  37. 58 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/COMMIT3Response.java
  38. 73 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/CREATE3Response.java
  39. 131 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/FSINFO3Response.java
  40. 113 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/FSSTAT3Response.java
  41. 50 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/GETATTR3Response.java
  42. 76 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/LOOKUP3Response.java
  43. 69 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/MKDIR3Response.java
  44. 46 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/NFS3Response.java
  45. 95 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/PATHCONF3Response.java
  46. 78 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READ3Response.java
  47. 117 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIR3Response.java
  48. 114 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIRPLUS3Response.java
  49. 53 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READLINK3Response.java
  50. 46 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/REMOVE3Response.java
  51. 54 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/RENAME3Response.java
  52. 47 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/RMDIR3Response.java
  53. 47 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/SETATTR3Response.java
  54. 70 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/SYMLINK3Response.java
  55. 37 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/VoidResponse.java
  56. 71 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/WRITE3Response.java
  57. 66 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/WccAttr.java
  58. 58 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/WccData.java
  59. 47 0
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/TestNfsTime.java
  60. 39 0
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/nfs3/TestFileHandle.java

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

@@ -8,6 +8,8 @@ Release 2.3.0 - UNRELEASED
 
 
     HADOOP-9509. Implement ONCRPC and XDR. (brandonli)
     HADOOP-9509. Implement ONCRPC and XDR. (brandonli)
 
 
+    HADOOP-9515. Add general interface for NFS and Mount. (brandonli)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
   OPTIMIZATIONS
   OPTIMIZATIONS

+ 59 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountEntry.java

@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mount;
+
+/**
+ * Represents a mount entry.
+ */
+public class MountEntry {
+  /** Host correspoinding to the mount entry */
+  private final String host;
+  /** Path correspoinding to the mount entry */
+  private final String path;
+
+  public MountEntry(String host, String path) {
+    this.host = host;
+    this.path = path;
+  }
+
+  public String host() {
+    return this.host;
+  }
+
+  public String path() {
+    return this.path;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o)
+      return true;
+
+    if (!(o instanceof MountEntry)) {
+      return false;
+    }
+
+    MountEntry m = (MountEntry) o;
+    return host().equals(m.host()) && path().equals(m.path());
+  }
+
+  @Override
+  public int hashCode() {
+    return host.hashCode() * 31 + path.hashCode();
+  }
+}

+ 58 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountInterface.java

@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mount;
+
+import java.net.InetAddress;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * This is an interface that should be implemented for handle Mountd related
+ * 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;
+
+  /** MNTPROC_NULL - Do Nothing */
+  public XDR nullOp(XDR out, int xid, InetAddress client);
+
+  /** MNTPROC_MNT - Add mount entry */
+  public XDR mnt(XDR xdr, XDR out, int xid, InetAddress client);
+
+  /** MNTPROC_DUMP - Return mount entries */
+  public XDR dump(XDR out, int xid, InetAddress client);
+
+  /** MNTPROC_UMNT - Remove mount entry */
+  public XDR umnt(XDR xdr, XDR out, int xid, InetAddress client);
+
+  /** MNTPROC_UMNTALL - Remove all mount entries */
+  public XDR umntall(XDR out, int xid, InetAddress client);
+  
+  /** MNTPROC_EXPORT and MNTPROC_EXPORTALL - Return export list */
+  //public XDR exportall(XDR out, int xid, InetAddress client);
+  
+  /** MNTPROC_PATHCONF - POSIX pathconf information */
+  //public XDR pathconf(XDR out, int xid, InetAddress client);
+}

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

@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mount;
+
+import java.util.List;
+
+import org.apache.hadoop.oncrpc.RpcAcceptedReply;
+import org.apache.hadoop.oncrpc.XDR;
+import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
+
+/**
+ * Helper class for sending MountResponse
+ */
+public class MountResponse {
+  public final static int MNT_OK = 0;
+  
+  /** Hidden constructor */
+  private MountResponse() {
+  }
+  
+  /** Response for RPC call {@link MountInterface#MNTPROC_MNT} */
+  public static XDR writeMNTResponse(int status, XDR xdr, int xid,
+      byte[] handle) {
+    RpcAcceptedReply.voidReply(xdr, xid);
+    xdr.writeInt(status);
+    if (status == MNT_OK) {
+      xdr.writeInt(handle.length);
+      xdr.writeFixedOpaque(handle);
+      // Only MountV3 returns a list of supported authFlavors
+      xdr.writeInt(1);
+      xdr.writeInt(AuthFlavor.AUTH_SYS.getValue());
+    }
+    return xdr;
+  }
+
+  /** 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) {
+      xdr.writeBoolean(true); // Value follows yes
+      xdr.writeString(mountEntry.host());
+      xdr.writeString(mountEntry.path());
+    }
+    xdr.writeBoolean(false); // Value follows no
+    return xdr;
+  }
+
+  /** 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) {
+      xdr.writeBoolean(true); // Value follows - yes
+      xdr.writeString(export);
+      xdr.writeInt(0);
+    }
+    xdr.writeBoolean(false); // Value follows - no
+    return xdr;
+  }
+}

+ 71 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountdBase.java

@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mount;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.oncrpc.RpcProgram;
+import org.apache.hadoop.oncrpc.SimpleTcpServer;
+import org.apache.hadoop.oncrpc.SimpleUdpServer;
+import org.apache.hadoop.portmap.PortmapMapping;
+
+/**
+ * Main class for starting mountd daemon. This daemon implements the NFS
+ * mount protocol. When receiving a MOUNT request from an NFS client, it checks
+ * the request against the list of currently exported file systems. If the
+ * client is permitted to mount the file system, rpc.mountd obtains a file
+ * handle for requested directory and returns it to the client.
+ */
+abstract public class MountdBase {
+  private final RpcProgram rpcProgram;
+
+  public RpcProgram getRpcProgram() {
+    return rpcProgram;
+  }
+  
+  /**
+   * Constructor
+   * @param exports
+   * @throws IOException 
+   */
+  public MountdBase(List<String> exports, RpcProgram program) throws IOException {
+    rpcProgram = program;
+  }
+
+  /* Start UDP server */
+  private void startUDPServer() {
+    rpcProgram.register(PortmapMapping.TRANSPORT_UDP);
+    SimpleUdpServer udpServer = new SimpleUdpServer(rpcProgram.getPort(),
+        rpcProgram, 1);
+    udpServer.run();
+  }
+
+  /* Start TCP server */
+  private void startTCPServer() {
+    rpcProgram.register(PortmapMapping.TRANSPORT_TCP);
+    SimpleTcpServer tcpServer = new SimpleTcpServer(rpcProgram.getPort(),
+        rpcProgram, 1);
+    tcpServer.run();
+  }
+
+  public void start() {
+    startUDPServer();
+    startTCPServer();
+  }
+}

+ 41 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsFileType.java

@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs;
+
+/**
+ * Class encapsulates different types of files
+ */
+public enum NfsFileType {
+  NFSREG(1),    // a regular file
+  NFSDIR(2),    // a directory
+  NFSBLK(3),    // a block special device file
+  NFSCHR(4),    // a character special device
+  NFSLNK(5),    // a symbolic link
+  NFSSOCK(6),   // a socket
+  NFSFIFO(7);   // a named pipe
+  
+  private final int value;
+  
+  NfsFileType(int val) {
+    value = val;
+  }
+  
+  public int toValue() {
+    return value;
+  }
+}

+ 88 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsTime.java

@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * Class that encapsulates time.
+ */
+public class NfsTime {
+  static final int MILLISECONDS_IN_SECOND = 1000;
+  static final int NANOSECONDS_IN_MILLISECOND = 1000000;
+  private final int seconds;
+  private final int nseconds;
+
+  public NfsTime(int seconds, int nseconds) {
+    this.seconds = seconds;
+    this.nseconds = nseconds;
+  }
+
+  public NfsTime(NfsTime other) {
+    seconds = other.getNseconds();
+    nseconds = other.getNseconds();
+  }
+  
+  public NfsTime(long milliseconds) {
+    seconds = (int) (milliseconds / MILLISECONDS_IN_SECOND);
+    nseconds = (int) ((milliseconds - this.seconds * MILLISECONDS_IN_SECOND) * 
+        NANOSECONDS_IN_MILLISECOND);
+  }
+
+  public int getSeconds() {
+    return seconds;
+  }
+  
+  public int getNseconds() {
+    return nseconds;
+  }
+
+  /**
+   * Get the total time in milliseconds
+   */
+  public long getMilliSeconds() {
+    return (long) (seconds) * 1000 + (long) (nseconds) / 1000000;
+  }
+
+  public void serialize(XDR xdr) {
+    xdr.writeInt(getSeconds());
+    xdr.writeInt(getNseconds());
+  }
+
+  public static NfsTime deserialize(XDR xdr) {
+    return new NfsTime(xdr.readInt(), xdr.readInt());
+  }
+
+  @Override
+  public int hashCode() {
+    return seconds ^ nseconds;
+  }
+  
+  @Override
+  public boolean equals(Object o) {
+    if (!(o instanceof NfsTime)) {
+      return false;
+    }
+    return ((NfsTime) o).getMilliSeconds() == this.getMilliSeconds();
+  }
+  
+  @Override
+  public String toString() {
+    return "(NfsTime-" + seconds + "s, " + nseconds + "ns)";
+  }
+}

+ 157 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/FileHandle.java

@@ -0,0 +1,157 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3;
+
+import java.nio.ByteBuffer;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * This is a file handle use by the NFS clients.
+ * Server returns this handle to the client, which is used by the client
+ * on subsequent operations to reference the file.
+ */
+public class FileHandle {
+  private static final Log LOG = LogFactory.getLog(FileHandle.class);
+  private static final String HEXES = "0123456789abcdef";
+  private static final int HANDLE_LEN = 32;
+  private byte[] handle; // Opaque handle
+  private long fileId = -1;
+
+  public FileHandle() {
+    handle = null;
+  }
+
+  /**
+   * Handle is a 32 bytes number. For HDFS, the last 8 bytes is fileId.
+   */
+  public FileHandle(long v) {
+    fileId = v;
+    handle = new byte[HANDLE_LEN];
+    handle[0] = (byte)(v >>> 56);
+    handle[1] = (byte)(v >>> 48);
+    handle[2] = (byte)(v >>> 40);
+    handle[3] = (byte)(v >>> 32);
+    handle[4] = (byte)(v >>> 24);
+    handle[5] = (byte)(v >>> 16);
+    handle[6] = (byte)(v >>>  8);
+    handle[7] = (byte)(v >>>  0);
+    for (int i = 8; i < HANDLE_LEN; i++) {
+      handle[i] = (byte) 0;
+    }
+  }
+  
+  public FileHandle(String s) {
+    MessageDigest digest;
+    try {
+      digest = MessageDigest.getInstance("MD5");
+      handle = new byte[HANDLE_LEN];
+    } catch (NoSuchAlgorithmException e) {
+      LOG.warn("MD5 MessageDigest unavailable.");
+      handle = null;
+      return;
+    }
+
+    byte[] in = s.getBytes();
+    for (int i = 0; i < in.length; i++) {
+      digest.update(in[i]);
+    }
+
+    byte[] digestbytes = digest.digest();
+    for (int i = 0; i < 16; i++) {
+      handle[i] = (byte) 0;
+    }
+
+    for (int i = 16; i < 32; i++) {
+      handle[i] = digestbytes[i - 16];
+    }
+  }
+
+  public boolean serialize(XDR out) {
+    out.writeInt(handle.length);
+    out.writeFixedOpaque(handle);
+    return true;
+  }
+
+  private long bytesToLong(byte[] data) {
+    ByteBuffer buffer = ByteBuffer.allocate(8);
+    for (int i = 0; i < 8; i++) {
+      buffer.put(data[i]);
+    }
+    buffer.flip();// need flip
+    return buffer.getLong();
+  }
+  
+  public boolean deserialize(XDR xdr) {
+    if (!XDR.verifyLength(xdr, 32)) {
+      return false;
+    }
+    int size = xdr.readInt();
+    handle = xdr.readFixedOpaque(size);
+    fileId = bytesToLong(handle);
+    return true;
+  }
+  
+  private static String hex(byte b) {
+    StringBuilder strBuilder = new StringBuilder();
+    strBuilder.append(HEXES.charAt((b & 0xF0) >> 4)).append(
+        HEXES.charAt((b & 0x0F)));
+    return strBuilder.toString();
+  }
+  
+  public long getFileId() {    
+    return fileId;
+  }
+  
+  public byte[] getContent() {
+    return handle.clone();
+  }
+  
+  @Override
+  public String toString() {
+    StringBuilder s = new StringBuilder();
+    for (int i = 0; i < handle.length; i++) {
+      s.append(hex(handle[i]));
+    }
+    return s.toString();
+  }
+  
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (!(o instanceof FileHandle)) {
+      return false;
+    }
+
+    FileHandle h = (FileHandle) o;
+    return Arrays.equals(handle, h.handle);
+  }
+
+  @Override
+  public int hashCode() {
+    return Arrays.hashCode(handle);
+  }
+}

+ 191 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java

@@ -0,0 +1,191 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
+
+/**
+ * Map id to user name or group name. It does update every 15 minutes. Only a
+ * single instance of this class is expected to be on the server.
+ */
+public class IdUserGroup {
+  static final Log LOG = LogFactory.getLog(IdUserGroup.class);
+  private final static String OS = System.getProperty("os.name");
+
+  /** Shell commands to get users and groups */
+  static final String LINUX_GET_ALL_USERS_CMD = "getent passwd | cut -d: -f1,3";
+  static final String LINUX_GET_ALL_GROUPS_CMD = "getent group | cut -d: -f1,3";
+  static final String MAC_GET_ALL_USERS_CMD = "dscl . -list /Users UniqueID";
+  static final String MAC_GET_ALL_GROUPS_CMD = "dscl . -list /Groups PrimaryGroupID";
+
+  // Do update every 15 minutes
+  final static long TIMEOUT = 15 * 60 * 1000; // ms
+
+  // Maps for id to name map. Guarded by this object monitor lock */
+  private BiMap<Integer, String> uidNameMap = HashBiMap.create();
+  private BiMap<Integer, String> gidNameMap = HashBiMap.create();
+
+  private long lastUpdateTime = 0; // Last time maps were updated
+
+  public IdUserGroup() {
+    updateMaps();
+  }
+
+  private boolean isExpired() {
+    return lastUpdateTime - System.currentTimeMillis() > TIMEOUT;
+  }
+
+  private void checkAndUpdateMaps() {
+    if (isExpired()) {
+      LOG.info("Update cache now");
+      updateMaps();
+    }
+  }
+
+  /**
+   * Get the whole list of users and groups and save them in the maps.
+   */
+  private void updateMapInternal(BiMap<Integer, String> map, String name,
+      String command, String regex) throws IOException {
+    BufferedReader br = null;
+    try {
+      Process process = Runtime.getRuntime().exec(
+          new String[] { "bash", "-c", command });
+      br = new BufferedReader(new InputStreamReader(process.getInputStream()));
+      String line = null;
+      while ((line = br.readLine()) != null) {
+        String[] nameId = line.split(regex);
+        if ((nameId == null) || (nameId.length != 2)) {
+          throw new IOException("Can't parse " + name + " list entry:" + line);
+        }
+        LOG.debug("add " + name + ":" + nameId[0] + " id:" + nameId[1]);
+        map.put(Integer.valueOf(nameId[1]), nameId[0]);
+      }
+      LOG.info("Updated " + name + " map size:" + map.size());
+      
+    } catch (IOException e) {
+      LOG.error("Can't update map " + name);
+      throw e;
+    } finally {
+      if (br != null) {
+        try {
+          br.close();
+        } catch (IOException e1) {
+          LOG.error("Can't close BufferedReader of command result");
+          e1.printStackTrace();
+        }
+      }
+    }
+  }
+
+  synchronized public void updateMaps() {
+    BiMap<Integer, String> uMap = HashBiMap.create();
+    BiMap<Integer, String> gMap = HashBiMap.create();
+
+    try {
+      if (OS.startsWith("Linux")) {
+        updateMapInternal(uMap, "user", LINUX_GET_ALL_USERS_CMD, ":");
+        updateMapInternal(gMap, "group", LINUX_GET_ALL_GROUPS_CMD, ":");
+      } else if (OS.startsWith("Mac")) {
+        updateMapInternal(uMap, "user", MAC_GET_ALL_USERS_CMD, "\\s+");
+        updateMapInternal(gMap, "group", MAC_GET_ALL_GROUPS_CMD, "\\s+");
+      } else {
+        throw new IOException("Platform is not supported:" + OS);
+      }
+    } catch (IOException e) {
+      LOG.error("Can't update maps:" + e);
+      return;
+    }
+    uidNameMap = uMap;
+    gidNameMap = gMap;
+    lastUpdateTime = System.currentTimeMillis();
+  }
+
+  synchronized public int getUid(String user) throws IOException {
+    checkAndUpdateMaps();
+
+    Integer id = uidNameMap.inverse().get(user);
+    if (id == null) {
+      throw new IOException("User just deleted?:" + user);
+    }
+    return id.intValue();
+  }
+
+  synchronized public int getGid(String group) throws IOException {
+    checkAndUpdateMaps();
+
+    Integer id = gidNameMap.inverse().get(group);
+    if (id == null) {
+      throw new IOException("No such group:" + group);
+
+    }
+    return id.intValue();
+  }
+
+  synchronized public String getUserName(int uid, String unknown) {
+    checkAndUpdateMaps();
+    String uname = uidNameMap.get(Integer.valueOf(uid));
+    if (uname == null) {
+      uname = unknown;
+    }
+    return uname;
+  }
+
+  synchronized public String getGroupName(int gid, String unknown) {
+    checkAndUpdateMaps();
+    String gname = gidNameMap.get(Integer.valueOf(gid));
+    if (gname == null) {
+      gname = unknown;
+    }
+    return gname;
+  }
+
+  // When can't map user, return user name's string hashcode
+  public int getUidAllowingUnknown(String user) {
+    checkAndUpdateMaps();
+    int uid;
+    try {
+      uid = getUid(user);
+    } catch (IOException e) {
+      uid = user.hashCode();
+      LOG.info("Can't map user " + user + ". Use its string hashcode:" + uid);
+    }
+    return uid;
+  }
+
+  // When can't map group, return group name's string hashcode
+  public int getGidAllowingUnknown(String group) {
+    checkAndUpdateMaps();
+    int gid;
+    try {
+      gid = getGid(group);
+    } catch (IOException e) {
+      gid = group.hashCode();
+      LOG.debug("Can't map group " + group + ". Use its string hashcode:" + gid);
+    }
+    return gid;
+  }
+}

+ 77 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java

@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mount.MountdBase;
+import org.apache.hadoop.oncrpc.RpcFrameDecoder;
+import org.apache.hadoop.oncrpc.RpcProgram;
+import org.apache.hadoop.oncrpc.SimpleTcpServer;
+import org.apache.hadoop.oncrpc.SimpleTcpServerHandler;
+import org.apache.hadoop.portmap.PortmapMapping;
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.Channels;
+
+/**
+ * Nfs server. Supports NFS v3 using {@link RpcProgram}.
+ * Currently Mountd program is also started inside this class.
+ * Only TCP server is supported and UDP is not supported.
+ */
+public abstract class Nfs3Base {
+  public static final Log LOG = LogFactory.getLog(Nfs3Base.class);
+  private final MountdBase mountd;
+  private final RpcProgram rpcProgram;
+  
+  public MountdBase getMountBase() {
+    return mountd;
+  }
+  
+  public RpcProgram getRpcProgram() {
+    return rpcProgram;
+  }
+
+  protected Nfs3Base(MountdBase mountd, RpcProgram program) {
+    this.mountd = mountd;
+    this.rpcProgram = program;
+  }
+
+  public void start() {
+    mountd.start();     // Start mountd
+    rpcProgram.register(PortmapMapping.TRANSPORT_TCP);
+    startTCPServer();   // Start TCP server
+  }
+
+  private void startTCPServer() {
+    SimpleTcpServer tcpServer = new SimpleTcpServer(Nfs3Constant.PORT,
+        rpcProgram, 0) {
+      @Override
+      public ChannelPipelineFactory getPipelineFactory() {
+        return new ChannelPipelineFactory() {
+          @Override
+          public ChannelPipeline getPipeline() {
+            return Channels.pipeline(new RpcFrameDecoder(),
+                new SimpleTcpServerHandler(rpcProgram));
+          }
+        };
+      }
+    };
+    tcpServer.run();
+  }
+}

+ 164 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Constant.java

@@ -0,0 +1,164 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3;
+
+/**
+ * Some constants for NFSv3
+ */
+public class Nfs3Constant {
+  // The local rpcbind/portmapper port.
+  public final static int SUN_RPCBIND = 111;
+
+  // The IP port number for NFS.
+  public final static int PORT = 2049;
+
+  // The RPC program number for NFS.
+  public final static int PROGRAM = 100003;
+
+  // The program version number that this server implements.
+  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;
+  
+  // The maximum size in bytes of the opaque file handle.
+  public final static int NFS3_FHSIZE = 64;
+
+  // The byte size of cookie verifier passed by READDIR and READDIRPLUS.
+  public final static int NFS3_COOKIEVERFSIZE = 8;
+
+  // The size in bytes of the opaque verifier used for exclusive CREATE.
+  public final static int NFS3_CREATEVERFSIZE = 8;
+
+  // The size in bytes of the opaque verifier used for asynchronous WRITE.
+  public final static int NFS3_WRITEVERFSIZE = 8;
+
+  /** Access call request mode */
+  // File access mode
+  public static final int ACCESS_MODE_READ = 0x04;
+  public static final int ACCESS_MODE_WRITE = 0x02;
+  public static final int ACCESS_MODE_EXECUTE = 0x01;
+
+  /** Access call response rights */
+  // Read data from file or read a directory.
+  public final static int ACCESS3_READ = 0x0001;
+  // Look up a name in a directory (no meaning for non-directory objects).
+  public final static int ACCESS3_LOOKUP = 0x0002;
+  // Rewrite existing file data or modify existing directory entries.
+  public final static int ACCESS3_MODIFY = 0x0004;
+  // Write new data or add directory entries.
+  public final static int ACCESS3_EXTEND = 0x0008;
+  // Delete an existing directory entry.
+  public final static int ACCESS3_DELETE = 0x0010;
+  // Execute file (no meaning for a directory).
+  public final static int ACCESS3_EXECUTE = 0x0020;
+
+  /** File and directory attribute mode bits */
+  // Set user ID on execution.
+  public final static int MODE_S_ISUID = 0x00800;
+  // Set group ID on execution.
+  public final static int MODE_S_ISGID = 0x00400;
+  // Save swapped text (not defined in POSIX).
+  public final static int MODE_S_ISVTX = 0x00200;
+  // Read permission for owner.
+  public final static int MODE_S_IRUSR = 0x00100;
+  // Write permission for owner.
+  public final static int MODE_S_IWUSR = 0x00080;
+  // Execute permission for owner on a file. Or lookup (search) permission for
+  // owner in directory.
+  public final static int MODE_S_IXUSR = 0x00040;
+  // Read permission for group.
+  public final static int MODE_S_IRGRP = 0x00020;
+  // Write permission for group.
+  public final static int MODE_S_IWGRP = 0x00010;
+  // Execute permission for group on a file. Or lookup (search) permission for
+  // group in directory.
+  public final static int MODE_S_IXGRP = 0x00008;
+  // Read permission for others.
+  public final static int MODE_S_IROTH = 0x00004;
+  // Write permission for others.
+  public final static int MODE_S_IWOTH = 0x00002;
+  // Execute permission for others on a file. Or lookup (search) permission for
+  // others in directory.
+  public final static int MODE_S_IXOTH = 0x00001;
+
+  public final static int MODE_ALL = MODE_S_ISUID | MODE_S_ISGID | MODE_S_ISVTX
+      | MODE_S_ISVTX | MODE_S_IRUSR | MODE_S_IRUSR | MODE_S_IWUSR
+      | MODE_S_IXUSR | MODE_S_IRGRP | MODE_S_IWGRP | MODE_S_IXGRP
+      | MODE_S_IROTH | MODE_S_IWOTH | MODE_S_IXOTH;
+
+  /** Write call flavors */
+  public enum WriteStableHow {
+    UNSTABLE(0), DATA_SYNC(1), FILE_SYNC(2);
+
+    private final int id;
+
+    WriteStableHow(int id) {
+      this.id = id;
+    }
+
+    public int getValue() {
+      return id;
+    }
+
+    public static WriteStableHow fromValue(int id) {
+      return values()[id];
+    }
+  }
+
+  /**
+   * This is a cookie that the client can use to determine whether the server
+   * has changed state between a call to WRITE and a subsequent call to either
+   * WRITE or COMMIT. This cookie must be consistent during a single instance of
+   * the NFS version 3 protocol service and must be unique between instances of
+   * the NFS version 3 protocol server, where uncommitted data may be lost.
+   */
+  public final static long WRITE_COMMIT_VERF = System.currentTimeMillis();
+  
+  /** FileSystemProperties */
+  public final static int FSF3_LINK = 0x0001;
+  public final static int FSF3_SYMLINK = 0x0002;
+  public final static int FSF3_HOMOGENEOUS = 0x0008;
+  public final static int FSF3_CANSETTIME = 0x0010;
+
+  /** Create options */
+  public final static int CREATE_UNCHECKED = 0;
+  public final static int CREATE_GUARDED = 1;
+  public final static int CREATE_EXCLUSIVE = 2;
+}

+ 217 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3FileAttributes.java

@@ -0,0 +1,217 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3;
+
+import org.apache.hadoop.nfs.NfsFileType;
+import org.apache.hadoop.nfs.NfsTime;
+import org.apache.hadoop.nfs.nfs3.response.WccAttr;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * File attrbutes reported in NFS.
+ */
+public class Nfs3FileAttributes {
+  private int type;
+  private int mode;
+  private int nlink;
+  private int uid;
+  private int gid;
+  private long size;
+  private long used;
+  private Specdata3 rdev;
+  private long fsid;
+  private long fileid;
+  private NfsTime atime;
+  private NfsTime mtime;
+  private NfsTime ctime;
+
+  /*
+   * The interpretation of the two words depends on the type of file system
+   * object. For a block special (NF3BLK) or character special (NF3CHR) file,
+   * specdata1 and specdata2 are the major and minor device numbers,
+   * respectively. (This is obviously a UNIX-specific interpretation.) For all
+   * other file types, these two elements should either be set to 0 or the
+   * values should be agreed upon by the client and server. If the client and
+   * server do not agree upon the values, the client should treat these fields
+   * as if they are set to 0.
+   * <br>
+   * For Hadoop, currently this field is always zero.
+   */
+  public static class Specdata3 {
+    final static int specdata1 = 0;
+    final static int specdata2 = 0;
+
+    public int getSpecdata1() {
+      return specdata1;
+    }
+
+    public int getSpecdata2() {
+      return specdata2;
+    }
+    
+    @Override
+    public String toString() {
+      return "(Specdata3: specdata1" + specdata1 + ", specdata2:" + specdata2
+          + ")";
+    }
+  }
+   
+  public Nfs3FileAttributes() {
+    this(false, 0, (short)0, 0, 0, 0, 0, 0, 0, 0);
+  }
+
+  public Nfs3FileAttributes(boolean isDir, int nlink, short mode, int uid,
+      int gid, long size, long fsid, long fileid, long mtime, long atime) {
+    this.type = isDir ? NfsFileType.NFSDIR.toValue() : NfsFileType.NFSREG
+        .toValue();
+    this.mode = mode;
+    this.nlink = isDir ? (nlink + 2) : 1;
+    this.uid = uid;
+    this.gid = gid;
+    this.size = size;
+    if(isDir) {
+      this.size = getDirSize(nlink);
+    }
+    this.used = this.size;
+    this.rdev = new Specdata3();
+    this.fsid = fsid;
+    this.fileid = fileid;
+    this.mtime = new NfsTime(mtime);
+    this.atime = atime != 0 ? new NfsTime(atime) : this.mtime;
+    this.ctime = this.mtime;
+  }
+  
+  public void serialize(XDR xdr) {
+    xdr.writeInt(type);
+    xdr.writeInt(mode);
+    xdr.writeInt(nlink);
+    xdr.writeInt(uid);
+    xdr.writeInt(gid);
+    xdr.writeLongAsHyper(size);
+    xdr.writeLongAsHyper(used);
+    xdr.writeInt(rdev.getSpecdata1());
+    xdr.writeInt(rdev.getSpecdata2());
+    xdr.writeLongAsHyper(fsid);
+    xdr.writeLongAsHyper(fileid);
+    atime.serialize(xdr);
+    mtime.serialize(xdr);
+    ctime.serialize(xdr);
+  }
+  
+  public static Nfs3FileAttributes deserialize(XDR xdr) {
+    Nfs3FileAttributes attr = new Nfs3FileAttributes();
+    attr.type = xdr.readInt();
+    attr.mode = xdr.readInt();
+    attr.nlink = xdr.readInt();
+    attr.uid = xdr.readInt();
+    attr.gid = xdr.readInt();
+    attr.size = xdr.readHyper();
+    attr.used = xdr.readHyper();
+    // Ignore rdev
+    xdr.readInt();
+    xdr.readInt();
+    attr.rdev = new Specdata3();
+    attr.fsid = xdr.readHyper();
+    attr.fileid = xdr.readHyper();
+    attr.atime = NfsTime.deserialize(xdr);
+    attr.mtime = NfsTime.deserialize(xdr);
+    attr.ctime = NfsTime.deserialize(xdr);
+    return attr;
+  }
+  
+  @Override
+  public String toString() {
+    return String.format("type:%d, mode:%d, nlink:%d, uid:%d, gid:%d, " + 
+            "size:%d, used:%d, rdev:%s, fsid:%d, fileid:%d, atime:%s, " + 
+            "mtime:%s, ctime:%s",
+            type, mode, nlink, uid, gid, size, used, rdev, fsid, fileid, atime,
+            mtime, ctime);
+  }
+
+  public int getNlink() {
+    return nlink;
+  }
+
+  public long getUsed() {
+    return used;
+  }
+
+  public long getFsid() {
+    return fsid;
+  }
+
+  public long getFileid() {
+    return fileid;
+  }
+
+  public NfsTime getAtime() {
+    return atime;
+  }
+
+  public NfsTime getMtime() {
+    return mtime;
+  }
+
+  public NfsTime getCtime() {
+    return ctime;
+  }
+
+  public int getType() {
+    return type;
+  }
+  
+  public WccAttr getWccAttr() {
+    return new WccAttr(size, mtime, ctime);
+  }
+  
+  public long getFileId() {
+    return fileid;
+  }
+  
+  public long getSize() {
+    return size;
+  }
+  
+  public void setSize(long size) {
+    this.size = size;
+  }
+  
+  public void setUsed(long used) {
+    this.used = used;
+  }
+  
+  public int getMode() {
+    return this.mode;
+  }
+  
+  public int getUid() {
+    return this.uid;
+  }
+  
+  public int getGid() {
+    return this.gid;
+  }
+  
+  /**
+   * HDFS directory size is always zero. Try to return something meaningful
+   * here. Assume each child take 32bytes.
+   */
+  public static long getDirSize(int childNum) {
+    return (childNum + 2) * 32;
+  }
+}

+ 83 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Interface.java

@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3;
+
+import org.apache.hadoop.nfs.nfs3.response.NFS3Response;
+import org.apache.hadoop.oncrpc.RpcAuthSys;
+import org.apache.hadoop.oncrpc.XDR;
+import org.jboss.netty.channel.Channel;
+
+/**
+ * RPC procedures as defined in RFC 1813.
+ */
+public interface Nfs3Interface {
+  
+  /** NULL: Do nothing */
+  public NFS3Response nullProcedure();
+  
+  /** GETATTR: Get file attributes */
+  public NFS3Response getattr(XDR xdr, RpcAuthSys authSys);
+  
+  /** SETATTR: Set file attributes */
+  public NFS3Response setattr(XDR xdr, RpcAuthSys authSys);
+  
+  /** LOOKUP: Lookup filename */
+  public NFS3Response lookup(XDR xdr, RpcAuthSys authSys);
+  
+  /** ACCESS: Check access permission  */
+  public NFS3Response access(XDR xdr, RpcAuthSys authSys);
+  
+  /** READ: Read from file */
+  public NFS3Response read(XDR xdr, RpcAuthSys authSys);
+  
+  /** WRITE: Write to file */
+  public NFS3Response write(XDR xdr, Channel channel, int xid, RpcAuthSys authSys);
+  
+  /** CREATE: Create a file  */
+  public NFS3Response create(XDR xdr, RpcAuthSys authSys);
+  
+  /** MKDIR: Create a directory  */
+  public NFS3Response mkdir(XDR xdr, RpcAuthSys authSys);
+  
+  /** REMOVE: Remove a file  */
+  public NFS3Response remove(XDR xdr, RpcAuthSys authSys);
+  
+  /** RMDIR: Remove a directory  */
+  public NFS3Response rmdir(XDR xdr, RpcAuthSys authSys);
+  
+  /** RENAME: Rename a file or directory */
+  public NFS3Response rename(XDR xdr, RpcAuthSys authSys);
+  
+  /** SYMLINK: Create a symbolic link  */
+  public NFS3Response symlink(XDR xdr, RpcAuthSys authSys);
+  
+  /** READDIR: Read From directory */
+  public NFS3Response readdir(XDR xdr, RpcAuthSys authSys);
+  
+  /** FSSTAT: Get dynamic file system information  */
+  public NFS3Response fsstat(XDR xdr, RpcAuthSys authSys);
+  
+  /** FSINFO: Get static file system information */
+  public NFS3Response fsinfo(XDR xdr, RpcAuthSys authSys);
+  
+  /** PATHCONF: Retrieve POSIX information */
+  public NFS3Response pathconf(XDR xdr, RpcAuthSys authSys);
+  
+  /** COMMIT: Commit cached data on a server to stable storage  */
+  public NFS3Response commit(XDR xdr, RpcAuthSys authSys);
+}

+ 162 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Status.java

@@ -0,0 +1,162 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3;
+
+/**
+ * Success or error status is reported in NFS3 responses.
+ */
+public class Nfs3Status {
+  
+  /** Indicates the call completed successfully. */
+  public final static int NFS3_OK = 0;
+  
+  /**
+   * The operation was not allowed because the caller is either not a
+   * privileged user (root) or not the owner of the target of the operation.
+   */
+  public final static int NFS3ERR_PERM = 1;
+  
+  /**
+   * No such file or directory. The file or directory name specified does not
+   * exist.
+   */
+  public final static int NFS3ERR_NOENT = 2;
+  
+  /**
+   * I/O error. A hard error (for example, a disk error) occurred while
+   * processing the requested operation.
+   */
+  public final static int NFS3ERR_IO = 5;
+  
+  /** I/O error. No such device or address. */
+  public final static int NFS3ERR_NXIO = 6;
+  
+  /**
+   * Permission denied. The caller does not have the correct permission to
+   * perform the requested operation. Contrast this with NFS3ERR_PERM, which
+   * restricts itself to owner or privileged user permission failures.
+   */
+  public final static int NFS3ERR_ACCES = 13;
+  
+  /** File exists. The file specified already exists. */
+  public final static int NFS3ERR_EXIST = 17;
+  
+  /** Attempt to do a cross-device hard link. */
+  public final static int NFS3ERR_XDEV = 18;
+  
+  /** No such device. */
+  public final static int NFS3ERR_NODEV = 19;
+  
+  /** The caller specified a non-directory in a directory operation. */
+  public static int NFS3ERR_NOTDIR = 20;
+  
+  /** The caller specified a directory in a non-directory operation. */
+  public final static int NFS3ERR_ISDIR = 21;
+  
+  /**
+   * Invalid argument or unsupported argument for an operation. Two examples are
+   * attempting a READLINK on an object other than a symbolic link or attempting
+   * to SETATTR a time field on a server that does not support this operation.
+   */
+  public final static int NFS3ERR_INVAL = 22;
+  
+  /**
+   * File too large. The operation would have caused a file to grow beyond the
+   * server's limit.
+   */
+  public final static int NFS3ERR_FBIG = 27;
+  
+  /**
+   * No space left on device. The operation would have caused the server's file
+   * system to exceed its limit.
+   */
+  public final static int NFS3ERR_NOSPC = 28;
+  
+  /**
+   * Read-only file system. A modifying operation was attempted on a read-only
+   * file system.
+   */
+  public final static int NFS3ERR_ROFS = 30;
+  
+  /** Too many hard links. */
+  public final static int NFS3ERR_MLINK = 31;
+  
+  /** The filename in an operation was too long. */
+  public final static int NFS3ERR_NAMETOOLONG = 63;
+  
+  /** An attempt was made to remove a directory that was not empty. */
+  public final static int NFS3ERR_NOTEMPTY = 66;
+  
+  /**
+   * Resource (quota) hard limit exceeded. The user's resource limit on the
+   * server has been exceeded.
+   */
+  public final static int NFS3ERR_DQUOT = 69;
+  
+  /**
+   * The file handle given in the arguments was invalid. The file referred to by
+   * that file handle no longer exists or access to it has been revoked.
+   */
+  public final static int NFS3ERR_STALE = 70;
+  
+  /**
+   * The file handle given in the arguments referred to a file on a non-local
+   * file system on the server.
+   */
+  public final static int NFS3ERR_REMOTE = 71;
+  
+  /** The file handle failed internal consistency checks */
+  public final static int NFS3ERR_BADHANDLE = 10001;
+  
+  /**
+   * Update synchronization mismatch was detected during a SETATTR operation.
+   */
+  public final static int NFS3ERR_NOT_SYNC = 10002;
+  
+  /** READDIR or READDIRPLUS cookie is stale */
+  public final static int NFS3ERR_BAD_COOKIE = 10003;
+  
+  /** Operation is not supported */
+  public final static int NFS3ERR_NOTSUPP = 10004;
+  
+  /** Buffer or request is too small */
+  public final static int NFS3ERR_TOOSMALL = 10005;
+  
+  /**
+   * An error occurred on the server which does not map to any of the legal NFS
+   * version 3 protocol error values. The client should translate this into an
+   * appropriate error. UNIX clients may choose to translate this to EIO.
+   */
+  public final static int NFS3ERR_SERVERFAULT = 10006;
+  
+  /**
+   * An attempt was made to create an object of a type not supported by the
+   * server.
+   */
+  public final static int NFS3ERR_BADTYPE = 10007;
+  
+  /**
+   * The server initiated the request, but was not able to complete it in a
+   * timely fashion. The client should wait and then try the request with a new
+   * RPC transaction ID. For example, this error should be returned from a
+   * server that supports hierarchical storage and receives a request to process
+   * a file that has been migrated. In this case, the server should start the
+   * immigration process and respond to client with this error.
+   */
+  public final static int NFS3ERR_JUKEBOX = 10008;
+}

+ 31 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/ACCESS3Request.java

@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * ACCESS3 Request
+ */
+public class ACCESS3Request extends RequestWithHandle {
+  public ACCESS3Request(XDR xdr) throws IOException {
+    super(xdr);
+  }
+}

+ 44 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/COMMIT3Request.java

@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * COMMIT3 Request
+ */
+public class COMMIT3Request extends RequestWithHandle {
+  private final long offset;
+  private final int count;
+
+  public COMMIT3Request(XDR xdr) throws IOException {
+    super(xdr);
+    offset = xdr.readHyper();
+    count = xdr.readInt();
+  }
+
+  public long getOffset() {
+    return this.offset;
+  }
+  
+  public int getCount() {
+    return this.count;
+  }
+}

+ 84 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/CREATE3Request.java

@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * CREATE3 Request
+ */
+public class CREATE3Request extends RequestWithHandle {
+  private final String name;
+  private final int mode;
+  private SetAttr3 objAttr = null;
+  private long verf;
+
+  public CREATE3Request(FileHandle handle, String name, int mode,
+      SetAttr3 objAttr, long verf) {
+    super(handle);
+    this.name = name;
+    this.mode = mode;
+    this.objAttr = objAttr;
+    this.verf = verf;
+  }
+  
+  public CREATE3Request(XDR xdr) throws IOException {
+    super(xdr);
+    name = xdr.readString();
+    mode = xdr.readInt();
+
+    objAttr = new SetAttr3();
+    if ((mode == Nfs3Constant.CREATE_UNCHECKED)
+        || (mode == Nfs3Constant.CREATE_GUARDED)) {
+      objAttr.deserialize(xdr);
+    } else if (mode == Nfs3Constant.CREATE_EXCLUSIVE) {
+      verf = xdr.readHyper();
+    } else {
+      throw new IOException("Wrong create mode:" + mode);
+    }
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public int getMode() {
+    return mode;
+  }
+
+  public SetAttr3 getObjAttr() {
+    return objAttr;
+  }
+
+  public long getVerf() {
+    return verf;
+  }
+  
+  @Override
+  public void serialize(XDR xdr) {
+    handle.serialize(xdr);
+    xdr.writeInt(name.length());
+    xdr.writeFixedOpaque(name.getBytes(), name.length());
+    xdr.writeInt(mode);
+    objAttr.serialize(xdr);
+  }
+}

+ 31 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/FSINFO3Request.java

@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * FSINFO3 Request
+ */
+public class FSINFO3Request extends RequestWithHandle {
+  public FSINFO3Request(XDR xdr) throws IOException {
+    super(xdr);
+  }
+}

+ 31 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/FSSTAT3Request.java

@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * FSSTAT3 Request
+ */
+public class FSSTAT3Request extends RequestWithHandle {
+  public FSSTAT3Request(XDR xdr) throws IOException {
+    super(xdr);
+  }
+}

+ 31 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/GETATTR3Request.java

@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * GETATTR3 Request
+ */
+public class GETATTR3Request extends RequestWithHandle {
+  public GETATTR3Request(XDR xdr) throws IOException {
+    super(xdr);
+  }
+}

+ 58 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/LOOKUP3Request.java

@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.oncrpc.XDR;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * LOOKUP3 Request
+ */
+public class LOOKUP3Request extends RequestWithHandle {
+  private String name;
+
+  public LOOKUP3Request(FileHandle handle, String name) {
+    super(handle);
+    this.name = name;
+  }
+  
+  public LOOKUP3Request(XDR xdr) throws IOException {
+    super(xdr);
+    name = xdr.readString();
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  @Override
+  @VisibleForTesting
+  public void serialize(XDR xdr) {
+    super.serialize(xdr);
+    xdr.writeInt(name.getBytes().length);
+    xdr.writeFixedOpaque(name.getBytes());
+  }
+}

+ 45 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/MKDIR3Request.java

@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * MKDIR3 Request
+ */
+public class MKDIR3Request extends RequestWithHandle {
+  private final String name;
+  private final SetAttr3 objAttr;
+
+  public MKDIR3Request(XDR xdr) throws IOException {
+    super(xdr);
+    name = xdr.readString();
+    objAttr = new SetAttr3();
+    objAttr.deserialize(xdr);
+  }
+  
+  public String getName() {
+    return name;
+  }
+
+  public SetAttr3 getObjAttr() {
+    return objAttr;
+  }
+}

+ 31 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/PATHCONF3Request.java

@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * PATHCONF3 Request
+ */
+public class PATHCONF3Request extends RequestWithHandle {
+  public PATHCONF3Request(XDR xdr) throws IOException {
+    super(xdr);
+  }
+}

+ 44 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/READ3Request.java

@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * READ3 Request
+ */
+public class READ3Request extends RequestWithHandle {
+  private final long offset;
+  private final int count;
+
+  public READ3Request(XDR xdr) throws IOException {
+    super(xdr);
+    offset = xdr.readHyper();
+    count = xdr.readInt();
+  }
+
+  public long getOffset() {
+    return this.offset;
+  }
+  
+  public int getCount() {
+    return this.count;
+  }
+}

+ 50 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/READDIR3Request.java

@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * READDIR3 Request
+ */
+public class READDIR3Request extends RequestWithHandle {
+  private final long cookie;
+  private final long cookieVerf;
+  private final int count;
+
+  public READDIR3Request(XDR xdr) throws IOException {
+    super(xdr);
+    cookie = xdr.readHyper();
+    cookieVerf = xdr.readHyper();
+    count = xdr.readInt();
+  }
+
+  public long getCookie() {
+    return this.cookie;
+  }
+
+  public long getCookieVerf() {
+    return this.cookieVerf;
+  }
+
+  public long getCount() {
+    return this.count;
+  }
+}

+ 56 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/READDIRPLUS3Request.java

@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * READDIRPLUS3 Request
+ */
+public class READDIRPLUS3Request extends RequestWithHandle {
+  private final long cookie;
+  private final long cookieVerf;
+  private final int dirCount;
+  private final int maxCount;
+
+  public READDIRPLUS3Request(XDR xdr) throws IOException {
+    super(xdr);
+    cookie = xdr.readHyper();
+    cookieVerf = xdr.readHyper();
+    dirCount = xdr.readInt();
+    maxCount = xdr.readInt();
+  }
+
+  public long getCookie() {
+    return this.cookie;
+  }
+
+  public long getCookieVerf() {
+    return this.cookieVerf;
+  }
+
+  public int getDirCount() {
+    return dirCount;
+  }
+
+  public int getMaxCount() {
+    return maxCount;
+  }
+}

+ 32 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/READLINK3Request.java

@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * READLINK3 Request
+ */
+public class READLINK3Request extends RequestWithHandle {
+
+  public READLINK3Request(XDR xdr) throws IOException {
+    super(xdr);
+  }
+}

+ 38 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/REMOVE3Request.java

@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * REMOVE3 Request
+ */
+public class REMOVE3Request extends RequestWithHandle {
+  private final String name;
+
+  public REMOVE3Request(XDR xdr) throws IOException {
+    super(xdr);
+    name = xdr.readString();
+  }
+
+  public String getName() {
+    return this.name;
+  }
+}

+ 62 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/RENAME3Request.java

@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * RENAME3 Request
+ */
+public class RENAME3Request {
+  private final FileHandle fromDirHandle;
+  private final String fromName;
+  private final FileHandle toDirHandle;
+  private final String toName;
+  
+  public RENAME3Request(XDR xdr) throws IOException {
+    fromDirHandle = new FileHandle();
+    if (!fromDirHandle.deserialize(xdr)) {
+      throw new IOException("can't deserialize file handle");
+    }
+    fromName = xdr.readString();
+    toDirHandle = new FileHandle();
+    if (!toDirHandle.deserialize(xdr)) {
+      throw new IOException("can't deserialize file handle");
+    }
+    toName = xdr.readString();
+  }
+  
+  public FileHandle getFromDirHandle() {
+    return fromDirHandle;
+  }
+
+  public String getFromName() {
+    return fromName;
+  }
+
+  public FileHandle getToDirHandle() {
+    return toDirHandle;
+  }
+
+  public String getToName() {
+    return toName;
+  }
+}

+ 38 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/RMDIR3Request.java

@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * RMDIR3 Request
+ */
+public class RMDIR3Request extends RequestWithHandle {
+  private final String name;
+
+  public RMDIR3Request(XDR xdr) throws IOException {
+    super(xdr);
+    name = xdr.readString();
+  }
+
+  public String getName() {
+    return this.name;
+  }
+}

+ 49 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/RequestWithHandle.java

@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * An NFS request that uses {@link FileHandle} to identify a file.
+ */
+public class RequestWithHandle {
+  protected final FileHandle handle;
+  
+  RequestWithHandle(FileHandle handle) {
+    this.handle = handle;
+  }
+  
+  RequestWithHandle(XDR xdr) throws IOException {
+    handle = new FileHandle();
+    if (!handle.deserialize(xdr)) {
+      throw new IOException("can't deserialize file handle");
+    }
+  }
+
+  public FileHandle getHandle() {
+    return this.handle;
+  }
+  
+  public void serialize(XDR xdr) {
+    handle.serialize(xdr);
+  }
+}

+ 64 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/SETATTR3Request.java

@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.nfs.NfsTime;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * SETATTR3 Request
+ */
+public class SETATTR3Request extends RequestWithHandle {
+  private final SetAttr3 attr;
+
+  /* A client may request that the server check that the object is in an
+   * expected state before performing the SETATTR operation. If guard.check is
+   * TRUE, the server must compare the value of ctime to the current ctime of
+   * the object. If the values are different, the server must preserve the
+   * object attributes and must return a status of NFS3ERR_NOT_SYNC. If check is
+   * FALSE, the server will not perform this check.
+   */
+  private final boolean check;
+  private final NfsTime ctime;
+  
+  public SETATTR3Request(XDR xdr) throws IOException {
+    super(xdr);
+    attr = new SetAttr3();
+    attr.deserialize(xdr);
+    check = xdr.readBoolean();
+    if (check) {
+      ctime = NfsTime.deserialize(xdr);
+    } else {
+      ctime = null;
+    }
+  }
+  
+  public SetAttr3 getAttr() {
+    return attr;
+  }
+
+  public boolean isCheck() {
+    return check;
+  }
+
+  public NfsTime getCtime() {
+    return ctime;
+  }
+}

+ 51 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/SYMLINK3Request.java

@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * SYMLINK3 Request
+ */
+public class SYMLINK3Request extends RequestWithHandle {
+  private final String name;
+  private final SetAttr3 symAttr;
+  private final String symData;
+  
+  public SYMLINK3Request(XDR xdr) throws IOException {
+    super(xdr);
+    name = xdr.readString();
+    symAttr = new SetAttr3();
+    symAttr.deserialize(xdr);
+    symData = xdr.readString();
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public SetAttr3 getSymAttr() {
+    return symAttr;
+  }
+
+  public String getSymData() {
+    return symData;
+  }
+}

+ 168 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/SetAttr3.java

@@ -0,0 +1,168 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.util.EnumSet;
+
+import org.apache.hadoop.nfs.NfsTime;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * SetAttr3 contains the file attributes that can be set from the client. The
+ * fields are the same as the similarly named fields in the NFS3Attributes
+ * structure.
+ */
+public class SetAttr3 {
+  // Options for time stamp change
+  public static final int TIME_DONT_CHANGE = 0;
+  public static final int TIME_SET_TO_SERVER_TIME = 1;
+  public static final int TIME_SET_TO_CLIENT_TIME = 2;
+
+  private int mode;
+  private int uid;
+  private int gid;
+  private long size;
+  private NfsTime atime;
+  private NfsTime mtime;
+  private EnumSet<SetAttrField> updateFields;
+
+  public static enum SetAttrField {
+    MODE, UID, GID, SIZE, ATIME, MTIME
+  };
+
+  public SetAttr3() {
+    mode = 0;
+    uid = 0;
+    gid = 0;
+    size = 0;
+    updateFields = EnumSet.noneOf(SetAttrField.class);
+  }
+
+  public int getMode() {
+    return mode;
+  }
+
+  public int getUid() {
+    return uid;
+  }
+
+  public int getGid() {
+    return gid;
+  }
+
+  public void setGid(int gid) {
+    this.gid = gid;  
+  }
+  
+  public long getSize() {
+    return size;
+  }
+
+  public NfsTime getAtime() {
+    return atime;
+  }
+
+  public NfsTime getMtime() {
+    return mtime;
+  }
+
+  public EnumSet<SetAttrField> getUpdateFields() {
+    return updateFields;
+  }
+
+  public void setUpdateFields(EnumSet<SetAttrField> updateFields) {
+    this.updateFields = updateFields;
+  }
+
+  public void serialize(XDR xdr) {
+    if (!updateFields.contains(SetAttrField.MODE)) {
+      xdr.writeBoolean(false);
+    } else {
+      xdr.writeBoolean(true);
+      xdr.writeInt(mode);
+    }
+    if (!updateFields.contains(SetAttrField.UID)) {
+      xdr.writeBoolean(false);
+    } else {
+      xdr.writeBoolean(true);
+      xdr.writeInt(uid);
+    }
+    if (!updateFields.contains(SetAttrField.GID)) {
+      xdr.writeBoolean(false);
+    } else {
+      xdr.writeBoolean(true);
+      xdr.writeInt(gid);
+    }
+    if (!updateFields.contains(SetAttrField.SIZE)) {
+      xdr.writeBoolean(false);
+    } else {
+      xdr.writeBoolean(true);
+      xdr.writeLongAsHyper(size);
+    }
+    if (!updateFields.contains(SetAttrField.ATIME)) {
+      xdr.writeBoolean(false);
+    } else {
+      xdr.writeBoolean(true);
+      atime.serialize(xdr);
+    }
+    if (!updateFields.contains(SetAttrField.MTIME)) {
+      xdr.writeBoolean(false);
+    } else {
+      xdr.writeBoolean(true);
+      mtime.serialize(xdr);
+    }
+  }
+  
+  public void deserialize(XDR xdr) {
+    if (xdr.readBoolean()) {
+      mode = xdr.readInt();
+      updateFields.add(SetAttrField.MODE);
+    }
+    if (xdr.readBoolean()) {
+      uid = xdr.readInt();
+      updateFields.add(SetAttrField.UID);
+    }
+    if (xdr.readBoolean()) {
+      gid = xdr.readInt();
+      updateFields.add(SetAttrField.GID);
+
+    }
+    if (xdr.readBoolean()) {
+      size = xdr.readHyper();
+      updateFields.add(SetAttrField.SIZE);
+
+    }
+    int timeSetHow = xdr.readInt();
+    if (timeSetHow == TIME_SET_TO_CLIENT_TIME) {
+      atime = NfsTime.deserialize(xdr);
+      updateFields.add(SetAttrField.ATIME);
+    } else if (timeSetHow == TIME_SET_TO_SERVER_TIME) {
+      atime = new NfsTime(System.currentTimeMillis());
+      updateFields.add(SetAttrField.ATIME);
+    }
+
+    timeSetHow = xdr.readInt();
+    if (timeSetHow == TIME_SET_TO_CLIENT_TIME) {
+      mtime = NfsTime.deserialize(xdr);
+      updateFields.add(SetAttrField.MTIME);
+    } else if (timeSetHow == TIME_SET_TO_SERVER_TIME) {
+      mtime = new NfsTime(System.currentTimeMillis());
+      updateFields.add(SetAttrField.MTIME);
+    }
+  }
+}

+ 78 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/WRITE3Request.java

@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.request;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant.WriteStableHow;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * WRITE3 Request
+ */
+public class WRITE3Request extends RequestWithHandle {
+  private final long offset;
+  private final int count;
+  private final WriteStableHow stableHow;
+  private final ByteBuffer data;
+
+  public WRITE3Request(XDR xdr) throws IOException {
+    super(xdr);
+    offset = xdr.readHyper();
+    count = xdr.readInt();
+    stableHow = WriteStableHow.fromValue(xdr.readInt());
+    data = ByteBuffer.wrap(xdr.readFixedOpaque(xdr.readInt()));
+  }
+
+  public WRITE3Request(FileHandle handle, final long offset, final int count,
+      final WriteStableHow stableHow, final ByteBuffer data) {
+    super(handle);
+    this.offset = offset;
+    this.count = count;
+    this.stableHow = stableHow;
+    this.data = data;
+  }
+
+  public long getOffset() {
+    return this.offset;
+  }
+
+  public int getCount() {
+    return this.count;
+  }
+
+  public WriteStableHow getStableHow() {
+    return this.stableHow;
+  }
+
+  public ByteBuffer getData() {
+    return this.data;
+  }
+
+  @Override
+  public void serialize(XDR xdr) {
+    handle.serialize(xdr);
+    xdr.writeLongAsHyper(offset);
+    xdr.writeInt(count);
+    xdr.writeInt(stableHow.getValue());
+    xdr.writeInt(count);
+    xdr.writeFixedOpaque(data.array(), count);
+  }
+}

+ 55 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/ACCESS3Response.java

@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * ACCESS3 Response 
+ */
+public class ACCESS3Response extends NFS3Response {
+  /*
+   * A bit mask of access permissions indicating access rights for the
+   * authentication credentials provided with the request.
+   */
+  private final int access;
+  private final Nfs3FileAttributes postOpAttr;
+  
+  public ACCESS3Response(int status) {
+    this(status, new Nfs3FileAttributes(), 0);
+  }
+  
+  public ACCESS3Response(int status, Nfs3FileAttributes postOpAttr, int access) {
+    super(status);
+    this.postOpAttr = postOpAttr;
+    this.access = access;
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    out.writeBoolean(true);
+    postOpAttr.serialize(out);
+    if (this.getStatus() == Nfs3Status.NFS3_OK) {
+      out.writeInt(access);
+    }
+    return out;
+  }
+}

+ 58 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/COMMIT3Response.java

@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * COMMIT3 Response
+ */
+public class COMMIT3Response extends NFS3Response {
+  private final WccData fileWcc;
+  private final long verf;
+
+  public COMMIT3Response(int status) {
+    this(status, new WccData(null, null), Nfs3Constant.WRITE_COMMIT_VERF);
+  }
+
+  public COMMIT3Response(int status, WccData fileWcc, long verf) {
+    super(status);
+    this.fileWcc = fileWcc;
+    this.verf = verf;
+  }
+  
+  public WccData getFileWcc() {
+    return fileWcc;
+  }
+
+  public long getVerf() {
+    return verf;
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    fileWcc.serialize(out);
+    if (getStatus() == Nfs3Status.NFS3_OK) {
+      out.writeLongAsHyper(verf);
+    }
+    return out;
+  }
+}

+ 73 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/CREATE3Response.java

@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * CREATE3 Response
+ */
+public class CREATE3Response extends NFS3Response {
+  private final FileHandle objHandle;
+  private final Nfs3FileAttributes postOpObjAttr;
+  private WccData dirWcc;
+
+  public CREATE3Response(int status) {
+    this(status, null, null, null);
+  }
+
+  public CREATE3Response(int status, FileHandle handle,
+      Nfs3FileAttributes postOpObjAttr, WccData dirWcc) {
+    super(status);
+    this.objHandle = handle;
+    this.postOpObjAttr = postOpObjAttr;
+    this.dirWcc = dirWcc;
+  }
+
+  public FileHandle getObjHandle() {
+    return objHandle;
+  }
+
+  public Nfs3FileAttributes getPostOpObjAttr() {
+    return postOpObjAttr;
+  }
+
+  public WccData getDirWcc() {
+    return dirWcc;
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    if (getStatus() == Nfs3Status.NFS3_OK) {
+      out.writeBoolean(true); // Handle follows
+      objHandle.serialize(out);
+      out.writeBoolean(true); // Attributes follow
+      postOpObjAttr.serialize(out);
+    }
+    if (dirWcc == null) {
+      dirWcc = new WccData(null, null);
+    }
+    dirWcc.serialize(out);
+
+    return out;
+  }
+}

+ 131 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/FSINFO3Response.java

@@ -0,0 +1,131 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.nfs.NfsTime;
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * FSINFO3 Response
+ */
+public class FSINFO3Response extends NFS3Response {
+  private final Nfs3FileAttributes postOpAttr;
+  /*
+   * The maximum size in bytes of a READ request supported by the server. Any
+   * READ with a number greater than rtmax will result in a short read of rtmax
+   * bytes or less.
+   */
+  private final int rtmax;
+  /*
+   * The preferred size of a READ request. This should be the same as rtmax
+   * unless there is a clear benefit in performance or efficiency.
+   */
+  private final int rtpref;
+  /* The suggested multiple for the size of a READ request. */
+  private final int rtmult;
+  /*
+   * The maximum size of a WRITE request supported by the server. In general,
+   * the client is limited by wtmax since there is no guarantee that a server
+   * can handle a larger write. Any WRITE with a count greater than wtmax will
+   * result in a short write of at most wtmax bytes.
+   */
+  private final int wtmax;
+  /*
+   * The preferred size of a WRITE request. This should be the same as wtmax
+   * unless there is a clear benefit in performance or efficiency.
+   */
+  private final int wtpref;
+  /*
+   * The suggested multiple for the size of a WRITE request.
+   */
+  private final int wtmult;
+  /* The preferred size of a READDIR request. */
+  private final int dtpref;
+  /* The maximum size of a file on the file system. */
+  private final long maxFileSize;
+  /*
+   * The server time granularity. When setting a file time using SETATTR, the
+   * server guarantees only to preserve times to this accuracy. If this is {0,
+   * 1}, the server can support nanosecond times, {0, 1000000} denotes
+   * millisecond precision, and {1, 0} indicates that times are accurate only to
+   * the nearest second.
+   */
+  private final NfsTime timeDelta;
+  /*
+   * A bit mask of file system properties. The following values are defined:
+   * 
+   * FSF_LINK If this bit is 1 (TRUE), the file system supports hard links.
+   * 
+   * FSF_SYMLINK If this bit is 1 (TRUE), the file system supports symbolic
+   * links.
+   * 
+   * FSF_HOMOGENEOUS If this bit is 1 (TRUE), the information returned by
+   * PATHCONF is identical for every file and directory in the file system. If
+   * it is 0 (FALSE), the client should retrieve PATHCONF information for each
+   * file and directory as required.
+   * 
+   * FSF_CANSETTIME If this bit is 1 (TRUE), the server will set the times for a
+   * file via SETATTR if requested (to the accuracy indicated by time_delta). If
+   * it is 0 (FALSE), the server cannot set times as requested.
+   */
+  private final int properties;
+
+  public FSINFO3Response(int status) {
+    this(status, new Nfs3FileAttributes(), 0, 0, 0, 0, 0, 0, 0, 0, null, 0);
+  }
+
+  public FSINFO3Response(int status, Nfs3FileAttributes postOpAttr, int rtmax,
+      int rtpref, int rtmult, int wtmax, int wtpref, int wtmult, int dtpref,
+      long maxFileSize, NfsTime timeDelta, int properties) {
+    super(status);
+    this.postOpAttr = postOpAttr;
+    this.rtmax = rtmax;
+    this.rtpref = rtpref;
+    this.rtmult = rtmult;
+    this.wtmax = wtmax;
+    this.wtpref = wtpref;
+    this.wtmult = wtmult;
+    this.dtpref = dtpref;
+    this.maxFileSize = maxFileSize;
+    this.timeDelta = timeDelta;
+    this.properties = properties;
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    out.writeBoolean(true);
+    postOpAttr.serialize(out);
+
+    if (getStatus() == Nfs3Status.NFS3_OK) {
+      out.writeInt(rtmax);
+      out.writeInt(rtpref);
+      out.writeInt(rtmult);
+      out.writeInt(wtmax);
+      out.writeInt(wtpref);
+      out.writeInt(wtmult);
+      out.writeInt(dtpref);
+      out.writeLongAsHyper(maxFileSize);
+      timeDelta.serialize(out);
+      out.writeInt(properties);
+   }
+    return out;
+  }
+}

+ 113 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/FSSTAT3Response.java

@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * FSSTAT3 Response
+ */
+public class FSSTAT3Response extends NFS3Response {
+  private Nfs3FileAttributes postOpAttr;
+
+  // The total size, in bytes, of the file system.
+  private final long tbytes;
+
+  // The amount of free space, in bytes, in the file system.
+  private final long fbytes;
+
+  /*
+   * The amount of free space, in bytes, available to the user identified by the
+   * authentication information in the RPC. (This reflects space that is
+   * reserved by the file system; it does not reflect any quota system
+   * implemented by the server.)
+   */
+  private final long abytes;
+
+  /*
+   * The total number of file slots in the file system. (On a UNIX server, this
+   * often corresponds to the number of inodes configured.)
+   */
+  private final long tfiles;
+
+  /* The number of free file slots in the file system. */
+  private final long ffiles;
+
+  /*
+   * The number of free file slots that are available to the user corresponding
+   * to the authentication information in the RPC. (This reflects slots that are
+   * reserved by the file system; it does not reflect any quota system
+   * implemented by the server.)
+   */
+  private final long afiles;
+
+  /*
+   * A measure of file system volatility: this is the number of seconds for
+   * which the file system is not expected to change. For a volatile, frequently
+   * updated file system, this will be 0. For an immutable file system, such as
+   * a CD-ROM, this would be the largest unsigned integer. For file systems that
+   * are infrequently modified, for example, one containing local executable
+   * programs and on-line documentation, a value corresponding to a few hours or
+   * days might be used. The client may use this as a hint in tuning its cache
+   * management. Note however, this measure is assumed to be dynamic and may
+   * change at any time.
+   */
+  private final int invarsec;
+
+  public FSSTAT3Response(int status) {
+    this(status, null, 0, 0, 0, 0, 0, 0, 0);
+  }
+
+  public FSSTAT3Response(int status, Nfs3FileAttributes postOpAttr,
+      long tbytes, long fbytes, long abytes, long tfiles, long ffiles,
+      long afiles, int invarsec) {
+    super(status);
+    this.postOpAttr = postOpAttr;
+    this.tbytes = tbytes;
+    this.fbytes = fbytes;
+    this.abytes = abytes;
+    this.tfiles = tfiles;
+    this.ffiles = ffiles;
+    this.afiles = afiles;
+    this.invarsec = invarsec;
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    out.writeBoolean(true);
+    if (postOpAttr == null) {
+      postOpAttr = new Nfs3FileAttributes();
+    }
+    postOpAttr.serialize(out);
+
+    if (getStatus() == Nfs3Status.NFS3_OK) {
+      out.writeLongAsHyper(tbytes);
+      out.writeLongAsHyper(fbytes);
+      out.writeLongAsHyper(abytes);
+      out.writeLongAsHyper(tfiles);
+      out.writeLongAsHyper(ffiles);
+      out.writeLongAsHyper(afiles);
+      out.writeInt(invarsec);
+    }
+    return out;
+
+  }
+}

+ 50 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/GETATTR3Response.java

@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * GETATTR3 Response
+ */
+public class GETATTR3Response extends NFS3Response {
+  private Nfs3FileAttributes postOpAttr;
+  public GETATTR3Response(int status) {
+    this(status, new Nfs3FileAttributes());
+  }
+  
+  public GETATTR3Response(int status, Nfs3FileAttributes attrs) {
+    super(status);
+    this.postOpAttr = attrs;
+  }
+  
+  public void setPostOpAttr(Nfs3FileAttributes postOpAttr) {
+    this.postOpAttr = postOpAttr;
+  }
+  
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    if (getStatus() == Nfs3Status.NFS3_OK) {
+      postOpAttr.serialize(out);
+    }
+    return out;
+  }
+}

+ 76 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/LOOKUP3Response.java

@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import java.io.IOException;
+
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * LOOKUP3 Response
+ */
+public class LOOKUP3Response extends NFS3Response {
+  private final FileHandle fileHandle;
+  private final Nfs3FileAttributes postOpObjAttr; // Can be null
+  private final Nfs3FileAttributes postOpDirAttr; // Can be null
+
+  public LOOKUP3Response(int status) {
+    this(status, null, new Nfs3FileAttributes(), new Nfs3FileAttributes());
+  }
+
+  public LOOKUP3Response(int status, FileHandle fileHandle,
+      Nfs3FileAttributes postOpObjAttr, Nfs3FileAttributes postOpDirAttributes) {
+    super(status);
+    this.fileHandle = fileHandle;
+    this.postOpObjAttr = postOpObjAttr;
+    this.postOpDirAttr = postOpDirAttributes;
+  }
+
+  public LOOKUP3Response(XDR xdr) throws IOException {
+    super(-1);
+    fileHandle = new FileHandle();
+    status = xdr.readInt();
+    Nfs3FileAttributes objAttr = null;
+    if (status == Nfs3Status.NFS3_OK) {
+      if (!fileHandle.deserialize(xdr)) {
+        throw new IOException("can't deserialize file handle");
+      }
+      objAttr = xdr.readBoolean() ? Nfs3FileAttributes.deserialize(xdr) : null;
+    }
+    postOpObjAttr = objAttr;
+    postOpDirAttr = xdr.readBoolean() ? Nfs3FileAttributes.deserialize(xdr)
+        : null;
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    if (this.status == Nfs3Status.NFS3_OK) {
+      fileHandle.serialize(out);
+      out.writeBoolean(true); // Attribute follows
+      postOpObjAttr.serialize(out);
+    }
+
+    out.writeBoolean(true); // Attribute follows
+    postOpDirAttr.serialize(out);
+    return out;
+  }
+}

+ 69 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/MKDIR3Response.java

@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * MKDIR3 Response
+ */
+public class MKDIR3Response extends NFS3Response {
+  private final FileHandle objFileHandle;
+  private final Nfs3FileAttributes objAttr;
+  private final WccData dirWcc;
+  
+  public MKDIR3Response(int status) {
+    this(status, null, null, new WccData(null, null));
+  }
+  
+  public MKDIR3Response(int status, FileHandle handle, Nfs3FileAttributes attr,
+      WccData dirWcc) {
+    super(status);
+    this.objFileHandle = handle;
+    this.objAttr = attr;
+    this.dirWcc = dirWcc;
+  }
+  
+  public FileHandle getObjFileHandle() {
+    return objFileHandle;
+  }
+
+  public Nfs3FileAttributes getObjAttr() {
+    return objAttr;
+  }
+
+  public WccData getDirWcc() {
+    return dirWcc;
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    if (getStatus() == Nfs3Status.NFS3_OK) {
+      out.writeBoolean(true); // Handle follows
+      objFileHandle.serialize(out);
+      out.writeBoolean(true); // Attributes follow
+      objAttr.serialize(out);
+    }
+    dirWcc.serialize(out);
+    return out;
+  }
+}

+ 46 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/NFS3Response.java

@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.oncrpc.RpcAcceptedReply;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * Abstract class for a NFSv3 response
+ */
+abstract public class NFS3Response {
+  protected int status;
+
+  public NFS3Response(int status) {
+    this.status = status;
+  }
+
+  public int getStatus() {
+    return this.status;
+  }
+
+  public void setStatus(int status) {
+    this.status = status;
+  }
+  
+  public XDR send(XDR out, int xid) {
+    RpcAcceptedReply.voidReply(out, xid);
+    out.writeInt(this.getStatus());
+    return out;
+  }
+}

+ 95 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/PATHCONF3Response.java

@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * PATHCONF3 Response
+ */
+public class PATHCONF3Response extends NFS3Response {
+  private final Nfs3FileAttributes postOpAttr;
+
+  /* The maximum number of hard links to an object. */
+  private final int linkMax;
+
+  /* The maximum length of a component of a filename. */
+  private final int nameMax;
+
+  /*
+   * If TRUE, the server will reject any request that includes a name longer
+   * than name_max with the error, NFS3ERR_NAMETOOLONG. If FALSE, any length
+   * name over name_max bytes will be silently truncated to name_max bytes.
+   */
+  private final boolean noTrunc;
+
+  /*
+   * If TRUE, the server will reject any request to change either the owner or
+   * the group associated with a file if the caller is not the privileged user.
+   * (Uid 0.)
+   */
+  private final boolean chownRestricted;
+
+  /*
+   * If TRUE, the server file system does not distinguish case when interpreting
+   * filenames.
+   */
+  private final boolean caseInsensitive;
+
+  /*
+   * If TRUE, the server file system will preserve the case of a name during a
+   * CREATE, MKDIR, MKNOD, SYMLINK, RENAME, or LINK operation.
+   */
+  private final boolean casePreserving;
+
+  public PATHCONF3Response(int status) {
+    this(status, new Nfs3FileAttributes(), 0, 0, false, false, false, false);
+  }
+
+  public PATHCONF3Response(int status, Nfs3FileAttributes postOpAttr,
+      int linkMax, int nameMax, boolean noTrunc, boolean chownRestricted,
+      boolean caseInsensitive, boolean casePreserving) {
+    super(status);
+    this.postOpAttr = postOpAttr;
+    this.linkMax = linkMax;
+    this.nameMax = nameMax;
+    this.noTrunc = noTrunc;
+    this.chownRestricted = chownRestricted;
+    this.caseInsensitive = caseInsensitive;
+    this.casePreserving = casePreserving;
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    out.writeBoolean(true);
+    postOpAttr.serialize(out);
+
+    if (getStatus() == Nfs3Status.NFS3_OK) {
+      out.writeInt(linkMax);
+      out.writeInt(nameMax);
+      out.writeBoolean(noTrunc);
+      out.writeBoolean(chownRestricted);
+      out.writeBoolean(caseInsensitive);
+      out.writeBoolean(casePreserving);
+    }
+    return out;
+  }
+}

+ 78 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READ3Response.java

@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * READ3 Response
+ */
+public class READ3Response extends NFS3Response {
+  private final Nfs3FileAttributes postOpAttr;
+  private final int count; // The real bytes of read data
+  private final boolean eof;
+  private final ByteBuffer data;
+
+  public READ3Response(int status) {
+    this(status, new Nfs3FileAttributes(), 0, false, null);
+  }
+  
+  public READ3Response(int status, Nfs3FileAttributes postOpAttr, int count,
+      boolean eof, ByteBuffer data) {
+    super(status);
+    this.postOpAttr = postOpAttr;
+    this.count = count;
+    this.eof = eof;
+    this.data = data;
+  }
+
+  public Nfs3FileAttributes getPostOpAttr() {
+    return postOpAttr;
+  }
+
+  public int getCount() {
+    return count;
+  }
+
+  public boolean isEof() {
+    return eof;
+  }
+
+  public ByteBuffer getData() {
+    return data;
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    out.writeBoolean(true); // Attribute follows
+    postOpAttr.serialize(out);
+
+    if (getStatus() == Nfs3Status.NFS3_OK) {
+      out.writeInt(count);
+      out.writeBoolean(eof);
+      out.writeInt(count);
+      out.writeFixedOpaque(data.array(), count);
+    }
+    return out;
+  }
+}

+ 117 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIR3Response.java

@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.oncrpc.XDR;
+
+import com.google.common.collect.ObjectArrays;
+
+/**
+ * READDIR3 Response
+ */
+public class READDIR3Response extends NFS3Response {
+  private final Nfs3FileAttributes postOpDirAttr;
+  private final long cookieVerf;
+  private final DirList3 dirList;
+
+  public static class Entry3 {
+    private final long fileId;
+    private final String name;
+    private final long cookie;
+    
+    public Entry3(long fileId, String name, long cookie) {
+      this.fileId = fileId;
+      this.name = name;
+      this.cookie = cookie;
+    }
+
+    long getFileId() {
+      return fileId;
+    }
+
+    String getName() {
+      return name;
+    }
+
+    long getCookie() {
+      return cookie;
+    }
+  }
+
+  public static class DirList3 {
+    final Entry3 entries[];
+    final boolean eof;
+    
+    public DirList3(Entry3[] entries, boolean eof) {
+      this.entries = ObjectArrays.newArray(entries, entries.length);
+      this.eof = eof;
+    }
+  }
+
+  public READDIR3Response(int status) {
+    this(status, new Nfs3FileAttributes());
+  }
+
+  public READDIR3Response(int status, Nfs3FileAttributes postOpAttr) {
+    this(status, postOpAttr, 0, null); 
+  }
+
+  public READDIR3Response(int status, Nfs3FileAttributes postOpAttr,
+      final long cookieVerf, final DirList3 dirList) {
+    super(status);
+    this.postOpDirAttr = postOpAttr;
+    this.cookieVerf = cookieVerf;
+    this.dirList = dirList;
+  }
+
+  public Nfs3FileAttributes getPostOpAttr() {
+    return postOpDirAttr;
+  }
+
+  public long getCookieVerf() {
+    return cookieVerf;
+  }
+
+  public DirList3 getDirList() {
+    return dirList;
+  }
+
+  @Override
+  public XDR send(XDR xdr, int xid) {
+    super.send(xdr, xid);
+    xdr.writeBoolean(true); // Attributes follow
+    postOpDirAttr.serialize(xdr);
+
+    if (getStatus() == Nfs3Status.NFS3_OK) {
+      xdr.writeLongAsHyper(cookieVerf);
+      Entry3[] f = dirList.entries;
+      for (int i = 0; i < f.length; i++) {
+        xdr.writeBoolean(true); // Value follows
+        xdr.writeLongAsHyper(f[i].getFileId());
+        xdr.writeString(f[i].getName());
+        xdr.writeLongAsHyper(f[i].getCookie());
+      }
+
+      xdr.writeBoolean(false);
+      xdr.writeBoolean(dirList.eof);
+    }
+    return xdr;
+  }
+}

+ 114 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIRPLUS3Response.java

@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.oncrpc.XDR;
+
+import com.google.common.collect.ObjectArrays;
+
+/**
+ * READDIRPLUS3 Response
+ */
+public class READDIRPLUS3Response  extends NFS3Response {
+  private Nfs3FileAttributes postOpDirAttr;
+  private final long cookieVerf;
+  private final DirListPlus3 dirListPlus;
+
+  public static class EntryPlus3 {
+    private final long fileId;
+    private final String name;
+    private final long cookie;
+    private final Nfs3FileAttributes nameAttr;
+    private final FileHandle objFileHandle;
+
+    public EntryPlus3(long fileId, String name, long cookie,
+        Nfs3FileAttributes nameAttr, FileHandle objFileHandle) {
+      this.fileId = fileId;
+      this.name = name;
+      this.cookie = cookie;
+      this.nameAttr = nameAttr;
+      this.objFileHandle = objFileHandle;
+    }
+
+    void seralize(XDR xdr) {
+      xdr.writeLongAsHyper(fileId);
+      xdr.writeString(name);
+      xdr.writeLongAsHyper(cookie);
+      xdr.writeBoolean(true);
+      nameAttr.serialize(xdr);
+      xdr.writeBoolean(true);
+      objFileHandle.serialize(xdr);
+    }
+  }
+
+  public static class DirListPlus3 {
+    EntryPlus3 entries[];
+    boolean eof;
+    
+    public DirListPlus3(EntryPlus3[] entries, boolean eof) {
+      this.entries = ObjectArrays.newArray(entries, entries.length);
+      this.eof = eof;
+    }
+
+    EntryPlus3[] getEntries() {
+      return entries;
+    }
+    
+    boolean getEof() {
+      return eof;
+    }
+  }
+
+  public READDIRPLUS3Response(int status) {
+    this(status, null, 0, null);
+  }
+
+  public READDIRPLUS3Response(int status, Nfs3FileAttributes postOpDirAttr,
+      final long cookieVerf, final DirListPlus3 dirListPlus) {
+    super(status);
+    this.postOpDirAttr = postOpDirAttr;
+    this.cookieVerf = cookieVerf;
+    this.dirListPlus = dirListPlus;
+  }
+  
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    out.writeBoolean(true); // attributes follow
+    if (postOpDirAttr == null) {
+      postOpDirAttr = new Nfs3FileAttributes();
+    }
+    postOpDirAttr.serialize(out);
+    
+    if (getStatus() == Nfs3Status.NFS3_OK) {
+      out.writeLongAsHyper(cookieVerf);
+      EntryPlus3[] f = dirListPlus.getEntries();
+      for (int i = 0; i < f.length; i++) {
+        out.writeBoolean(true); // next
+        f[i].seralize(out);
+      }
+
+      out.writeBoolean(false);
+      out.writeBoolean(dirListPlus.getEof());
+    }
+    return out;
+  }
+}

+ 53 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READLINK3Response.java

@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * READLINK3 Response
+ */
+public class READLINK3Response extends NFS3Response {
+  private final Nfs3FileAttributes postOpSymlinkAttr;
+  private final byte path[];
+
+  public READLINK3Response(int status) {
+    this(status, new Nfs3FileAttributes(), new byte[0]);
+  }
+
+  public READLINK3Response(int status, Nfs3FileAttributes postOpAttr,
+      byte path[]) {
+    super(status);
+    this.postOpSymlinkAttr = postOpAttr;
+    this.path = new byte[path.length];
+    System.arraycopy(path, 0, this.path, 0, path.length);
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    out.writeBoolean(true); // Attribute follows
+    postOpSymlinkAttr.serialize(out);
+    if (getStatus() == Nfs3Status.NFS3_OK) {
+      out.writeFixedOpaque(path, path.length);
+    }
+    return out;
+  }
+}

+ 46 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/REMOVE3Response.java

@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * REMOVE3 Response
+ */
+public class REMOVE3Response extends NFS3Response {
+  private WccData dirWcc;
+
+  public REMOVE3Response(int status) {
+    this(status, null);
+  }
+
+  public REMOVE3Response(int status, WccData dirWcc) {
+    super(status);
+    this.dirWcc = dirWcc;
+  }
+  
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    if (dirWcc == null) {
+      dirWcc = new WccData(null, null);
+    }
+    dirWcc.serialize(out);
+    return out;
+  }
+}

+ 54 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/RENAME3Response.java

@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * RENAME3 Response
+ */
+public class RENAME3Response extends NFS3Response {
+  private final WccData fromDirWcc;
+  private final WccData toDirWcc;
+
+  public RENAME3Response(int status) {
+    this(status, new WccData(null, null), new WccData(null, null));
+  }
+  
+  public RENAME3Response(int status, WccData fromWccData, WccData toWccData) {
+    super(status);
+    this.fromDirWcc = fromWccData;
+    this.toDirWcc = toWccData;
+  }
+
+  public WccData getFromDirWcc() {
+    return fromDirWcc;
+  }
+
+  public WccData getToDirWcc() {
+    return toDirWcc;
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    fromDirWcc.serialize(out);
+    toDirWcc.serialize(out);
+    return out;
+  }
+}

+ 47 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/RMDIR3Response.java

@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * RMDIR3 Response
+ */
+public class RMDIR3Response extends NFS3Response {
+  private final WccData dirWcc;
+
+  public RMDIR3Response(int status) {
+    this(status, new WccData(null, null));
+  }
+
+  public RMDIR3Response(int status, WccData wccData) {
+    super(status);
+    this.dirWcc = wccData;
+  }
+  
+  public WccData getDirWcc() {
+    return dirWcc;
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    dirWcc.serialize(out);
+    return out;
+  }
+}

+ 47 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/SETATTR3Response.java

@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * SETATTR3 Response
+ */
+public class SETATTR3Response extends NFS3Response {
+  private final WccData wccData;
+
+  public SETATTR3Response(int status) {
+    this(status, new WccData(null, null));
+  }
+  
+  public SETATTR3Response(int status, WccData wccData) {
+    super(status);
+    this.wccData = wccData;
+  }
+
+  public WccData getWccData() {
+    return wccData;
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    wccData.serialize(out);
+    return out;
+  }
+}

+ 70 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/SYMLINK3Response.java

@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * SYMLINK3 Response
+ */
+public class SYMLINK3Response extends NFS3Response {
+  private final FileHandle objFileHandle;
+  private final Nfs3FileAttributes objPostOpAttr;
+  private final WccData dirWcc;
+  
+  public SYMLINK3Response(int status) {
+    this(status, null, null, new WccData(null, null));
+  }
+  
+  public SYMLINK3Response(int status, FileHandle handle,
+      Nfs3FileAttributes attrs, WccData dirWcc) {
+    super(status);
+    this.objFileHandle = handle;
+    this.objPostOpAttr = attrs;
+    this.dirWcc = dirWcc;
+  }
+  
+  public FileHandle getObjFileHandle() {
+    return objFileHandle;
+  }
+
+  public Nfs3FileAttributes getObjPostOpAttr() {
+    return objPostOpAttr;
+  }
+
+  public WccData getDirWcc() {
+    return dirWcc;
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    if (this.getStatus() == Nfs3Status.NFS3_OK) {
+      out.writeBoolean(true);
+      objFileHandle.serialize(out);
+      objPostOpAttr.serialize(out);
+    }
+    out.writeBoolean(true);
+    dirWcc.serialize(out);
+    
+    return out;
+  }
+}

+ 37 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/VoidResponse.java

@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.oncrpc.RpcAcceptedReply;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * A void NFSv3 response
+ */
+public class VoidResponse extends NFS3Response {
+
+  public VoidResponse(int status) {
+    super(status);
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    RpcAcceptedReply.voidReply(out, xid);
+    return out;
+  }
+}

+ 71 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/WRITE3Response.java

@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant.WriteStableHow;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * WRITE3 Response
+ */
+public class WRITE3Response extends NFS3Response {
+  private final WccData fileWcc; // return on both success and failure
+  private final int count;
+  private final WriteStableHow stableHow;
+  private final long verifer;
+
+  public WRITE3Response(int status) {
+    this(status, new WccData(null, null), 0, WriteStableHow.UNSTABLE,
+        Nfs3Constant.WRITE_COMMIT_VERF);
+  }
+  
+  public WRITE3Response(int status, WccData fileWcc, int count,
+      WriteStableHow stableHow, long verifier) {
+    super(status);
+    this.fileWcc = fileWcc;
+    this.count = count;
+    this.stableHow = stableHow;
+    this.verifer = verifier;
+  }
+
+  public int getCount() {
+    return count;
+  }
+
+  public WriteStableHow getStableHow() {
+    return stableHow;
+  }
+
+  public long getVerifer() {
+    return verifer;
+  }
+
+  @Override
+  public XDR send(XDR out, int xid) {
+    super.send(out, xid);
+    fileWcc.serialize(out);
+    if (getStatus() == Nfs3Status.NFS3_OK) {
+      out.writeInt(count);
+      out.writeInt(stableHow.getValue());
+      out.writeLongAsHyper(verifer);
+    }
+    return out;
+  }
+}

+ 66 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/WccAttr.java

@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.nfs.NfsTime;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * WccAttr saves attributes used for weak cache consistency
+ */
+public class WccAttr {
+  long size;
+  NfsTime mtime; // in milliseconds
+  NfsTime ctime; // in milliseconds
+
+  public long getSize() {
+    return size;
+  }
+
+  public NfsTime getMtime() {
+    return mtime;
+  }
+
+  public NfsTime getCtime() {
+    return ctime;
+  }
+
+  public WccAttr() {
+    this.size = 0;
+    mtime = null;
+    ctime = null;
+  }
+
+  public WccAttr(long size, NfsTime mtime, NfsTime ctime) {
+    this.size = size;
+    this.mtime = mtime;
+    this.ctime = ctime;
+  }
+
+  public void serialize(XDR out) {
+    out.writeLongAsHyper(size);
+    if (mtime == null) {
+      mtime = new NfsTime(0);
+    }
+    mtime.serialize(out);
+    if (ctime == null) {
+      ctime = new NfsTime(0);
+    }
+    ctime.serialize(out);
+  }
+}

+ 58 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/WccData.java

@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3.response;
+
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.oncrpc.XDR;
+
+/**
+ * WccData saved information used by client for weak cache consistency
+ */
+public class WccData {
+  private WccAttr preOpAttr;
+  private Nfs3FileAttributes postOpAttr;
+
+  public WccAttr getPreOpAttr() {
+    return preOpAttr;
+  }
+
+  public void setPreOpAttr(WccAttr preOpAttr) {
+    this.preOpAttr = preOpAttr;
+  }
+
+  public Nfs3FileAttributes getPostOpAttr() {
+    return postOpAttr;
+  }
+
+  public void setPostOpAttr(Nfs3FileAttributes postOpAttr) {
+    this.postOpAttr = postOpAttr;
+  }
+
+  public WccData(WccAttr preOpAttr, Nfs3FileAttributes postOpAttr) {
+    this.preOpAttr = (preOpAttr == null) ? new WccAttr() : preOpAttr;
+    this.postOpAttr = (postOpAttr == null) ? new Nfs3FileAttributes()
+        : postOpAttr;
+  }
+
+  public void serialize(XDR out) {
+    out.writeBoolean(true); // attributes follow
+    preOpAttr.serialize(out);
+    out.writeBoolean(true); // attributes follow
+    postOpAttr.serialize(out);
+  }
+}

+ 47 - 0
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/TestNfsTime.java

@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.nfs.NfsTime;
+import org.apache.hadoop.oncrpc.XDR;
+import org.junit.Test;
+
+public class TestNfsTime {
+  @Test
+  public void testConstructor() {
+    NfsTime nfstime = new NfsTime(1001);
+    Assert.assertEquals(1, nfstime.getSeconds());
+    Assert.assertEquals(1000000, nfstime.getNseconds());
+  }
+  
+  @Test
+  public void testSerializeDeserialize() {
+    // Serialize NfsTime
+    NfsTime t1 = new NfsTime(1001);
+    XDR xdr = new XDR();
+    t1.serialize(xdr);
+    
+    // Deserialize it back
+    NfsTime t2 = NfsTime.deserialize(xdr);
+    
+    // Ensure the NfsTimes are equal
+    Assert.assertEquals(t1, t2);
+  }
+}

+ 39 - 0
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/nfs3/TestFileHandle.java

@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.nfs.nfs3;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.oncrpc.XDR;
+import org.junit.Test;
+
+public class TestFileHandle {
+  @Test
+  public void testConstructor() {
+    FileHandle handle = new FileHandle(1024);
+    XDR xdr = new XDR();
+    handle.serialize(xdr);
+    Assert.assertEquals(handle.getFileId(), 1024);
+
+    // Deserialize it back 
+    FileHandle handle2 = new FileHandle();
+    handle2.deserialize(xdr);
+    Assert.assertEquals(handle.getFileId(), 1024);
+  }
+}