浏览代码

HDFS-4762 Provide HDFS based NFSv3 and Mountd implementation. Contributed by Brandon Li

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1499029 13f79535-47bb-0310-9956-ffa450edef68
Brandon Li 12 年之前
父节点
当前提交
37f587563a
共有 24 个文件被更改,包括 4650 次插入0 次删除
  1. 4 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/README.txt
  2. 18 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/dev-support/findbugsExcludeFile.xml
  3. 268 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
  4. 54 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/Mountd.java
  5. 183 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
  6. 141 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/AsyncDataService.java
  7. 85 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java
  8. 60 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/LruCache.java
  9. 50 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java
  10. 166 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
  11. 72 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OffsetRange.java
  12. 775 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
  13. 1622 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
  14. 162 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java
  15. 284 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java
  16. 63 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestMountd.java
  17. 185 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java
  18. 142 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestPortmapRegister.java
  19. 132 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestUdpServer.java
  20. 57 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestDFSClientCache.java
  21. 58 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestOffsetRange.java
  22. 66 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java
  23. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  24. 1 0
      hadoop-hdfs-project/pom.xml

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/README.txt

@@ -0,0 +1,4 @@
+-----------------------------------------------------------------------------
+HDFS-NFS - NFS implementation for Hadoop HDFS
+
+-----------------------------------------------------------------------------

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/dev-support/findbugsExcludeFile.xml

@@ -0,0 +1,18 @@
+<!--
+   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.
+-->
+<FindBugsFilter>
+</FindBugsFilter>

+ 268 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml

@@ -0,0 +1,268 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project-dist</artifactId>
+    <version>3.0.0-SNAPSHOT</version>
+    <relativePath>../../hadoop-project-dist</relativePath>
+  </parent>
+  <groupId>org.apache.hadoop</groupId>
+  <artifactId>hadoop-hdfs-nfs</artifactId>
+  <version>3.0.0-SNAPSHOT</version>
+  <description>Apache Hadoop HDFS-NFS</description>
+  <name>Apache Hadoop HDFS-NFS</name>
+  <packaging>jar</packaging>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-annotations</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-auth</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-nfs</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty</artifactId>
+      <version>3.6.2.Final</version>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.zookeeper</groupId>
+      <artifactId>zookeeper</artifactId>
+      <version>3.4.2</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty-util</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-core</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-server</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-cli</groupId>
+      <artifactId>commons-cli</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-codec</groupId>
+      <artifactId>commons-codec</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-daemon</groupId>
+      <artifactId>commons-daemon</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>javax.servlet.jsp</groupId>
+      <artifactId>jsp-api</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>javax.servlet</groupId>
+      <artifactId>servlet-api</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-core-asl</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>tomcat</groupId>
+      <artifactId>jasper-runtime</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>xmlenc</groupId>
+      <artifactId>xmlenc</artifactId>
+      <scope>compile</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <systemPropertyVariables>
+            <startKdc>${startKdc}</startKdc>
+            <kdc.resource.dir>${kdc.resource.dir}</kdc.resource.dir>
+          </systemPropertyVariables>
+          <properties>
+            <property>
+              <name>listener</name>
+              <value>org.apache.hadoop.test.TimedOutTestsListener</value>
+            </property>
+          </properties>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>add-jsp-generated-sources-directory</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>${project.build.directory}/generated-sources/java</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <configuration>
+          <skipTests>false</skipTests>
+        </configuration>
+        <executions>
+          <execution>
+            <id>create-jsp-generated-sources-directory</id>
+            <phase>initialize</phase>
+            <goals>
+              <goal>run</goal>
+            </goals>
+            <configuration>
+              <target>
+                <mkdir dir="${project.build.directory}/generated-sources/java" />
+              </target>
+            </configuration>
+          </execution>
+          <execution>
+            <phase>pre-site</phase>
+            <goals>
+              <goal>run</goal>
+            </goals>
+            <configuration>
+              <tasks>
+                <copy file="src/main/resources/hdfs-nfs-default.xml" todir="src/site/resources"/>
+                <copy file="src/main/xsl/configuration.xsl" todir="src/site/resources"/>
+              </tasks>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>

+ 54 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/Mountd.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.hdfs.nfs.mount;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mount.MountdBase;
+
+/**
+ * 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.
+ */
+public class Mountd extends MountdBase {
+  /**
+   * Constructor
+   * @param exports
+   * @throws IOException 
+   */
+  public Mountd(List<String> exports) throws IOException {
+    super(exports, new RpcProgramMountd(exports));
+  }
+
+  public Mountd(List<String> exports, Configuration config) throws IOException {
+    super(exports, new RpcProgramMountd(exports, config));
+  }
+  
+  public static void main(String[] args) throws IOException {
+    List<String> exports = new ArrayList<String>();
+    exports.add("/");
+    Mountd mountd = new Mountd(exports);
+    mountd.start(true);
+  }
+}

+ 183 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java

@@ -0,0 +1,183 @@
+/**
+ * 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.hdfs.nfs.mount;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.mount.MountEntry;
+import org.apache.hadoop.mount.MountInterface;
+import org.apache.hadoop.mount.MountResponse;
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.oncrpc.RpcAcceptedReply;
+import org.apache.hadoop.oncrpc.RpcCall;
+import org.apache.hadoop.oncrpc.RpcProgram;
+import org.apache.hadoop.oncrpc.XDR;
+import org.jboss.netty.channel.Channel;
+
+/**
+ * RPC program corresponding to mountd daemon. See {@link Mountd}.
+ */
+public class RpcProgramMountd extends RpcProgram implements MountInterface {
+  private static final Log LOG = LogFactory.getLog(RpcProgramMountd.class);
+  public static final int PROGRAM = 100005;
+  public static final int VERSION_1 = 1;
+  public static final int VERSION_2 = 2;
+  public static final int VERSION_3 = 3;
+  public static final int PORT = 4242;
+
+  // Need DFSClient for branch-1 to get ExtendedHdfsFileStatus
+  private final DFSClient dfsClient;
+  
+  /** Synchronized list */
+  private final List<MountEntry> mounts;
+  
+  /** List that is unmodifiable */
+  private final List<String> exports;
+
+  public RpcProgramMountd() throws IOException {
+    this(new ArrayList<String>(0));
+  }
+
+  public RpcProgramMountd(List<String> exports) throws IOException {
+    this(exports, new Configuration());
+  }
+
+  public RpcProgramMountd(List<String> exports, Configuration config)
+      throws IOException {
+    // Note that RPC cache is not enabled
+    super("mountd", "localhost", PORT, PROGRAM, VERSION_1, VERSION_3, 0);
+    this.mounts = Collections.synchronizedList(new ArrayList<MountEntry>());
+    this.exports = Collections.unmodifiableList(exports);
+    this.dfsClient = new DFSClient(NameNode.getAddress(config), config);
+  }
+  
+  public XDR nullOp(XDR out, int xid, InetAddress client) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("MOUNT NULLOP : " + " client: " + client);
+    }
+    return  RpcAcceptedReply.voidReply(out, xid);
+  }
+
+  public XDR mnt(XDR xdr, XDR out, int xid, InetAddress client) {
+    String path = xdr.readString();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("MOUNT MNT path: " + path + " client: " + client);
+    }
+
+    String host = client.getHostName();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Got host: " + host + " path: " + path);
+    }
+    if (!exports.contains(path)) {
+      LOG.info("Path " + path + " is not shared.");
+      MountResponse.writeMNTResponse(Nfs3Status.NFS3ERR_NOENT, out, xid, null);
+      return out;
+    }
+
+    FileHandle handle = null;
+    try {
+      HdfsFileStatus exFileStatus = dfsClient.getFileInfo(path);
+      
+      handle = new FileHandle(exFileStatus.getFileId());
+    } catch (IOException e) {
+      LOG.error("Can't get handle for export:" + path + ", exception:" + e);
+      MountResponse.writeMNTResponse(Nfs3Status.NFS3ERR_NOENT, out, xid, null);
+      return out;
+    }
+
+    assert (handle != null);
+    LOG.info("Giving handle (fileId:" + handle.getFileId()
+        + ") to client for export " + path);
+    mounts.add(new MountEntry(host, path));
+
+    MountResponse.writeMNTResponse(Nfs3Status.NFS3_OK, out, xid,
+        handle.getContent());
+    return out;
+  }
+
+  public XDR dump(XDR out, int xid, InetAddress client) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("MOUNT NULLOP : " + " client: " + client);
+    }
+
+    List<MountEntry> copy = new ArrayList<MountEntry>(mounts);
+    MountResponse.writeMountList(out, xid, copy);
+    return out;
+  }
+
+  public XDR umnt(XDR xdr, XDR out, int xid, InetAddress client) {
+    String path = xdr.readString();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("MOUNT UMNT path: " + path + " client: " + client);
+    }
+    
+    String host = client.getHostName();
+    mounts.remove(new MountEntry(host, path));
+    RpcAcceptedReply.voidReply(out, xid);
+    return out;
+  }
+
+  public XDR umntall(XDR out, int xid, InetAddress client) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("MOUNT UMNTALL : " + " client: " + client);
+    }
+    mounts.clear();
+    return RpcAcceptedReply.voidReply(out, xid);
+  }
+
+  @Override
+  public XDR handleInternal(RpcCall rpcCall, XDR xdr, XDR out,
+      InetAddress client, Channel channel) {
+    int procedure = rpcCall.getProcedure();
+    int xid = rpcCall.getXid();
+    if (procedure == MNTPROC_NULL) {
+      out = nullOp(out, xid, client);
+    } else if (procedure == MNTPROC_MNT) {
+      out = mnt(xdr, out, xid, client);
+    } else if (procedure == MNTPROC_DUMP) {
+      out = dump(out, xid, client);
+    } else if (procedure == MNTPROC_UMNT) {      
+      out = umnt(xdr, out, xid, client);
+    } else if (procedure == MNTPROC_UMNTALL) {
+      umntall(out, xid, client);
+    } else if (procedure == MNTPROC_EXPORT) {
+      out = MountResponse.writeExportList(out, xid, exports);
+    } else {
+      // Invalid procedure
+      RpcAcceptedReply.voidReply(out, xid,
+          RpcAcceptedReply.AcceptState.PROC_UNAVAIL);    }  
+    return out;
+  }
+  
+  @Override
+  protected boolean isIdempotent(RpcCall call) {
+    // Not required, because cache is turned off
+    return false;
+  }
+}

+ 141 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/AsyncDataService.java

@@ -0,0 +1,141 @@
+/**
+ * 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.hdfs.nfs.nfs3;
+
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * This class is a thread pool to easily schedule async data operations.Current
+ * async data operation is write back operation. In the future, we could use it
+ * for readahead operations too.
+ */
+public class AsyncDataService {
+  static final Log LOG = LogFactory.getLog(AsyncDataService.class);
+
+  // ThreadPool core pool size
+  private static final int CORE_THREADS_PER_VOLUME = 1;
+  // ThreadPool maximum pool size
+  private static final int MAXIMUM_THREADS_PER_VOLUME = 4;
+  // ThreadPool keep-alive time for threads over core pool size
+  private static final long THREADS_KEEP_ALIVE_SECONDS = 60;
+  private final ThreadGroup threadGroup = new ThreadGroup("async data service");
+  private ThreadFactory threadFactory = null;
+  private ThreadPoolExecutor executor = null;
+
+  public AsyncDataService() {
+    threadFactory = new ThreadFactory() {
+      public Thread newThread(Runnable r) {
+        return new Thread(threadGroup, r);
+      }
+    };
+
+    executor = new ThreadPoolExecutor(CORE_THREADS_PER_VOLUME,
+        MAXIMUM_THREADS_PER_VOLUME, THREADS_KEEP_ALIVE_SECONDS,
+        TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(), threadFactory);
+
+    // This can reduce the number of running threads
+    executor.allowCoreThreadTimeOut(true);
+  }
+
+  /**
+   * Execute the task sometime in the future.
+   */
+  synchronized void execute(Runnable task) {
+    if (executor == null) {
+      throw new RuntimeException("AsyncDataService is already shutdown");
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Current active thread number: " + executor.getActiveCount()
+          + " queue size:" + executor.getQueue().size()
+          + " scheduled task number:" + executor.getTaskCount());
+    }
+    executor.execute(task);
+  }
+
+  /**
+   * Gracefully shut down the ThreadPool. Will wait for all data tasks to
+   * finish.
+   */
+  synchronized void shutdown() {
+    if (executor == null) {
+      LOG.warn("AsyncDataService has already shut down.");
+    } else {
+      LOG.info("Shutting down all async data service threads...");
+      executor.shutdown();
+
+      // clear the executor so that calling execute again will fail.
+      executor = null;
+      LOG.info("All async data service threads have been shut down");
+    }
+  }
+
+  /**
+   * Write the data to HDFS asynchronously
+   */
+  void writeAsync(OpenFileCtx openFileCtx) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Scheduling write back task for fileId: "
+          + openFileCtx.copyLatestAttr().getFileId());
+    }
+    WriteBackTask wbTask = new WriteBackTask(openFileCtx);
+    execute(wbTask);
+  }
+
+  /**
+   * A task for write data back to HDFS for a file. Since only one thread can
+   * write for a file, any time there should be only one task(in queue or
+   * executing) for one file existing, and this should be guaranteed by the
+   * caller.
+   */
+  static class WriteBackTask implements Runnable {
+
+    OpenFileCtx openFileCtx;
+
+    WriteBackTask(OpenFileCtx openFileCtx) {
+      this.openFileCtx = openFileCtx;
+    }
+
+    OpenFileCtx getOpenFileCtx() {
+      return openFileCtx;
+    }
+
+    @Override
+    public String toString() {
+      // Called in AsyncDataService.execute for displaying error messages.
+      return "write back data for fileId"
+          + openFileCtx.copyLatestAttr().getFileId() + " with nextOffset "
+          + openFileCtx.getNextOffset();
+    }
+
+    public void run() {
+      try {
+        openFileCtx.executeWriteBack();
+      } catch (Throwable t) {
+        LOG.error("Asyn data service got error:"
+            + ExceptionUtils.getFullStackTrace(t));
+      }
+    }
+  }
+}

+ 85 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java

@@ -0,0 +1,85 @@
+/**
+ * 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.hdfs.nfs.nfs3;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * A cache saves DFSClient objects for different users
+ */
+public class DFSClientCache {
+  static final Log LOG = LogFactory.getLog(DFSClientCache.class);
+  private final LruCache<String, DFSClient> lruTable;
+  private final Configuration config;
+
+  public DFSClientCache(Configuration config) {
+    // By default, keep 256 DFSClient instance for 256 active users
+    this(config, 256);
+  }
+
+  public DFSClientCache(Configuration config, int size) {
+    lruTable = new LruCache<String, DFSClient>(size);
+    this.config = config;
+  }
+
+  public void put(String uname, DFSClient client) {
+    lruTable.put(uname, client);
+  }
+
+  synchronized public DFSClient get(String uname) {
+    DFSClient client = lruTable.get(uname);
+    if (client != null) {
+      return client;
+    }
+
+    // Not in table, create one.
+    try {
+      UserGroupInformation ugi = UserGroupInformation.createRemoteUser(uname);
+      client = ugi.doAs(new PrivilegedExceptionAction<DFSClient>() {
+        public DFSClient run() throws IOException {
+          return new DFSClient(NameNode.getAddress(config), config);
+        }
+      });
+    } catch (IOException e) {
+      LOG.error("Create DFSClient failed for user:" + uname);
+      e.printStackTrace();
+
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+    // Add new entry
+    lruTable.put(uname, client);
+    return client;
+  }
+
+  public int usedSize() {
+    return lruTable.usedSize();
+  }
+
+  public boolean containsKey(String key) {
+    return lruTable.containsKey(key);
+  }
+}

+ 60 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/LruCache.java

@@ -0,0 +1,60 @@
+/**
+ * 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.hdfs.nfs.nfs3;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * A thread-safe LRU table.
+ */
+public class LruCache<K, V> {
+  private final int maxSize;
+  private final LinkedHashMap<K, V> map;
+  private static final float hashTableLoadFactor = 0.75f;
+
+  public LruCache(int maxSize) {
+    this.maxSize = maxSize;
+    int hashTableCapacity = (int) Math.ceil(maxSize / hashTableLoadFactor) + 1;
+    map = new LinkedHashMap<K, V>(hashTableCapacity, hashTableLoadFactor, true) {
+      private static final long serialVersionUID = 1L;
+
+      @Override
+      protected boolean removeEldestEntry(Map.Entry<K, V> eldest) {
+        return size() > LruCache.this.maxSize;
+      }
+    };
+  }
+
+  // The found entry becomes the most recently used.
+  public synchronized V get(K key) {
+    return map.get(key);
+  }
+
+  public synchronized void put(K key, V value) {
+    map.put(key, value);
+  }
+
+  public synchronized int usedSize() {
+    return map.size();
+  }
+
+  public synchronized boolean containsKey(K key) {
+    return map.containsKey(key);
+  }
+}

+ 50 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.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.hdfs.nfs.nfs3;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.nfs.mount.Mountd;
+import org.apache.hadoop.nfs.nfs3.Nfs3Base;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * Nfs server. Supports NFS v3 using {@link RpcProgramNfs3}.
+ * Currently Mountd program is also started inside this class.
+ * Only TCP server is supported and UDP is not supported.
+ */
+public class Nfs3 extends Nfs3Base {
+  public Nfs3(List<String> exports) throws IOException {
+    super(new Mountd(exports), new RpcProgramNfs3(exports));
+  }
+
+  public Nfs3(List<String> exports, Configuration config) throws IOException {
+    super(new Mountd(exports, config), new RpcProgramNfs3(exports, config));
+  }
+
+  public static void main(String[] args) throws IOException {
+    StringUtils.startupShutdownMessage(Nfs3.class, args, LOG);
+    List<String> exports = new ArrayList<String>();
+    exports.add("/");
+    final Nfs3 nfsServer = new Nfs3(exports);
+    nfsServer.start(true);
+  }
+}

+ 166 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java

@@ -0,0 +1,166 @@
+/**
+ * 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.hdfs.nfs.nfs3;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.nfs.NfsFileType;
+import org.apache.hadoop.nfs.NfsTime;
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.nfs.nfs3.IdUserGroup;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.response.WccAttr;
+import org.apache.hadoop.nfs.nfs3.response.WccData;
+import org.apache.hadoop.oncrpc.XDR;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.channel.Channel;
+
+/**
+ * Utility/helper methods related to NFS
+ */
+public class Nfs3Utils {
+  public final static String INODEID_PATH_PREFIX = "/.reserved/.inodes/";
+
+  public static String getFileIdPath(FileHandle handle) {
+    return getFileIdPath(handle.getFileId());
+  }
+
+  public static String getFileIdPath(long fileId) {
+    return INODEID_PATH_PREFIX + fileId;
+  }
+
+  public static HdfsFileStatus getFileStatus(DFSClient client, String fileIdPath)
+      throws IOException {
+    return client.getFileInfo(fileIdPath);
+  }
+
+  public static Nfs3FileAttributes getNfs3FileAttrFromFileStatus(
+      HdfsFileStatus fs, IdUserGroup iug) {
+    /**
+     * Some 32bit Linux client has problem with 64bit fileId: it seems the 32bit
+     * client takes only the lower 32bit of the fileId and treats it as signed
+     * int. When the 32th bit is 1, the client considers it invalid.
+     */
+    return new Nfs3FileAttributes(fs.isDir(), fs.getChildrenNum(), fs
+        .getPermission().toShort(), iug.getUidAllowingUnknown(fs.getOwner()),
+        iug.getGidAllowingUnknown(fs.getGroup()), fs.getLen(), 0 /* fsid */,
+        fs.getFileId(), fs.getModificationTime(), fs.getAccessTime());
+  }
+
+  public static Nfs3FileAttributes getFileAttr(DFSClient client,
+      String fileIdPath, IdUserGroup iug) throws IOException {
+    HdfsFileStatus fs = getFileStatus(client, fileIdPath);
+    return fs == null ? null : getNfs3FileAttrFromFileStatus(fs, iug);
+  }
+
+  public static WccAttr getWccAttr(DFSClient client, String fileIdPath)
+      throws IOException {
+    HdfsFileStatus fstat = getFileStatus(client, fileIdPath);
+    if (fstat == null) {
+      return null;
+    }
+
+    long size = fstat.isDir() ? Nfs3FileAttributes.getDirSize(fstat
+        .getChildrenNum()) : fstat.getLen();
+    return new WccAttr(size, new NfsTime(fstat.getModificationTime()),
+        new NfsTime(fstat.getModificationTime()));
+  }
+
+  public static WccAttr getWccAttr(Nfs3FileAttributes attr) {
+    return new WccAttr(attr.getSize(), attr.getMtime(), attr.getCtime());
+  }
+
+  public static WccData createWccData(final WccAttr preOpAttr,
+      DFSClient dfsClient, final String fileIdPath, final IdUserGroup iug)
+      throws IOException {
+    Nfs3FileAttributes postOpDirAttr = getFileAttr(dfsClient, fileIdPath, iug);
+    return new WccData(preOpAttr, postOpDirAttr);
+  }
+
+  /**
+   * Send a write response to the netty network socket channel
+   */
+  public static void writeChannel(Channel channel, XDR out) {
+    ChannelBuffer outBuf = XDR.writeMessageTcp(out, true);
+    channel.write(outBuf);
+  }
+
+  private static boolean isSet(int access, int bits) {
+    return (access & bits) == bits;
+  }
+
+  public static int getAccessRights(int mode, int type) {
+    int rtn = 0;
+    if (isSet(mode, Nfs3Constant.ACCESS_MODE_READ)) {
+      rtn |= Nfs3Constant.ACCESS3_READ;
+      // LOOKUP is only meaningful for dir
+      if (type == NfsFileType.NFSDIR.toValue()) {
+        rtn |= Nfs3Constant.ACCESS3_LOOKUP;
+      }
+    }
+    if (isSet(mode, Nfs3Constant.ACCESS_MODE_WRITE)) {
+      rtn |= Nfs3Constant.ACCESS3_MODIFY;
+      rtn |= Nfs3Constant.ACCESS3_EXTEND;
+      // Set delete bit, UNIX may ignore it for regular file since it's up to
+      // parent dir op permission
+      rtn |= Nfs3Constant.ACCESS3_DELETE;
+    }
+    if (isSet(mode, Nfs3Constant.ACCESS_MODE_EXECUTE)) {
+      if (type == NfsFileType.NFSREG.toValue()) {
+        rtn |= Nfs3Constant.ACCESS3_EXECUTE;
+      }
+    }
+    return rtn;
+  }
+
+  public static int getAccessRightsForUserGroup(int uid, int gid,
+      Nfs3FileAttributes attr) {
+    int mode = attr.getMode();
+    if (uid == attr.getUid()) {
+      return getAccessRights(mode >> 6, attr.getType());
+    }
+    if (gid == attr.getGid()) {
+      return getAccessRights(mode >> 3, attr.getType());
+    }
+    return getAccessRights(mode, attr.getType());
+  }
+  
+  public static long bytesToLong(byte[] data) {
+    long n = 0xffL & data[0];
+    for (int i = 1; i < 8; i++) {
+      n = (n << 8) | (0xffL & data[i]);
+    }
+    return n;
+  }
+
+  public static byte[] longToByte(long v) {
+    byte[] data = new byte[8];
+    data[0] = (byte) (v >>> 56);
+    data[1] = (byte) (v >>> 48);
+    data[2] = (byte) (v >>> 40);
+    data[3] = (byte) (v >>> 32);
+    data[4] = (byte) (v >>> 24);
+    data[5] = (byte) (v >>> 16);
+    data[6] = (byte) (v >>> 8);
+    data[7] = (byte) (v >>> 0);
+    return data;
+  }
+}

+ 72 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OffsetRange.java

@@ -0,0 +1,72 @@
+/**
+ * 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.hdfs.nfs.nfs3;
+
+/**
+ * OffsetRange is the range of read/write request. A single point (e.g.,[5,5])
+ * is not a valid range.
+ */
+public class OffsetRange implements Comparable<OffsetRange> {
+  private final long min;
+  private final long max;
+
+  OffsetRange(long min, long max) {
+    if ((min >= max) || (min < 0) || (max < 0)) {
+      throw new IllegalArgumentException("Wrong offset range: (" + min + ","
+          + max + ")");
+    }
+    this.min = min;
+    this.max = max;
+  }
+
+  long getMin() {
+    return min;
+  }
+
+  long getMax() {
+    return max;
+  }
+
+  @Override
+  public int hashCode() {
+    return (int) (min ^ max);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    assert (o instanceof OffsetRange);
+    OffsetRange range = (OffsetRange) o;
+    return (min == range.getMin()) && (max == range.getMax());
+  }
+
+  private static int compareTo(long left, long right) {
+    if (left < right) {
+      return -1;
+    } else if (left > right) {
+      return 1;
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  public int compareTo(OffsetRange other) {
+    final int d = compareTo(min, other.getMin());
+    return d != 0 ? d : compareTo(max, other.getMax());
+  }
+}

+ 775 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java

@@ -0,0 +1,775 @@
+/**
+ * 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.hdfs.nfs.nfs3;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.security.InvalidParameterException;
+import java.util.EnumSet;
+import java.util.Iterator;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
+import org.apache.hadoop.io.BytesWritable.Comparator;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.nfs.nfs3.IdUserGroup;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant.WriteStableHow;
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.nfs.nfs3.request.WRITE3Request;
+import org.apache.hadoop.nfs.nfs3.response.WRITE3Response;
+import org.apache.hadoop.nfs.nfs3.response.WccAttr;
+import org.apache.hadoop.nfs.nfs3.response.WccData;
+import org.apache.hadoop.oncrpc.XDR;
+import org.jboss.netty.channel.Channel;
+
+/**
+ * OpenFileCtx saves the context of one HDFS file output stream. Access to it is
+ * synchronized by its member lock.
+ */
+class OpenFileCtx {
+  public static final Log LOG = LogFactory.getLog(OpenFileCtx.class);
+  
+  /**
+   * Lock to synchronize OpenFileCtx changes. Thread should get this lock before
+   * any read/write operation to an OpenFileCtx object
+   */
+  private final ReentrantLock ctxLock;
+
+  // The stream status. False means the stream is closed.
+  private boolean activeState;
+  // The stream write-back status. True means one thread is doing write back.
+  private boolean asyncStatus;
+
+  private final FSDataOutputStream fos;
+  private final Nfs3FileAttributes latestAttr;
+  private long nextOffset;
+
+  private final SortedMap<OffsetRange, WriteCtx> pendingWrites;
+  
+  // The last write, commit request or write-back event. Updating time to keep
+  // output steam alive.
+  private long lastAccessTime;
+  
+  // Pending writes water mark for dump, 1MB
+  private static int DUMP_WRITE_WATER_MARK = 1024 * 1024; 
+  private FileOutputStream dumpOut;
+  private long nonSequentialWriteInMemory;
+  private boolean enabledDump;
+  private RandomAccessFile raf;
+  private final String dumpFilePath;
+  
+  private void updateLastAccessTime() {
+    lastAccessTime = System.currentTimeMillis();
+  }
+
+  private boolean checkStreamTimeout(long streamTimeout) {
+    return System.currentTimeMillis() - lastAccessTime > streamTimeout;
+  }
+  
+  // Increase or decrease the memory occupation of non-sequential writes
+  private long updateNonSequentialWriteInMemory(long count) {
+    nonSequentialWriteInMemory += count;
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Update nonSequentialWriteInMemory by " + count + " new value:"
+          + nonSequentialWriteInMemory);
+    }
+
+    if (nonSequentialWriteInMemory < 0) {
+      LOG.error("nonSequentialWriteInMemory is negative after update with count "
+          + count);
+      throw new IllegalArgumentException(
+          "nonSequentialWriteInMemory is negative after update with count "
+              + count);
+    }
+    return nonSequentialWriteInMemory;
+  }
+  
+  OpenFileCtx(FSDataOutputStream fos, Nfs3FileAttributes latestAttr,
+      String dumpFilePath) {
+    this.fos = fos;
+    this.latestAttr = latestAttr;
+    pendingWrites = new TreeMap<OffsetRange, WriteCtx>();
+    updateLastAccessTime();
+    activeState = true;
+    asyncStatus = false;
+    dumpOut = null;
+    raf = null;
+    nonSequentialWriteInMemory = 0;
+    this.dumpFilePath = dumpFilePath;  
+    enabledDump = dumpFilePath == null ? false: true;
+    ctxLock = new ReentrantLock(true);
+  }
+
+  private void lockCtx() {
+    if (LOG.isTraceEnabled()) {
+      StackTraceElement[] stacktrace = Thread.currentThread().getStackTrace();
+      StackTraceElement e = stacktrace[2];
+      String methodName = e.getMethodName();
+      LOG.trace("lock ctx, caller:" + methodName);
+    }
+    ctxLock.lock();
+  }
+
+  private void unlockCtx() {
+    ctxLock.unlock();
+    if (LOG.isTraceEnabled()) {
+      StackTraceElement[] stacktrace = Thread.currentThread().getStackTrace();
+      StackTraceElement e = stacktrace[2];
+      String methodName = e.getMethodName();
+      LOG.info("unlock ctx, caller:" + methodName);
+    }
+  }
+  
+  // Make a copy of the latestAttr
+  public Nfs3FileAttributes copyLatestAttr() {
+    Nfs3FileAttributes ret;
+    lockCtx();
+    try {
+      ret = new Nfs3FileAttributes(latestAttr);
+    } finally {
+      unlockCtx();
+    }
+    return ret;
+  }
+  
+  private long getNextOffsetUnprotected() {
+    assert(ctxLock.isLocked());
+    return nextOffset;
+  }
+
+  public long getNextOffset() {
+    long ret;
+    lockCtx();
+    try {
+      ret = getNextOffsetUnprotected();
+    } finally {
+      unlockCtx();
+    }
+    return ret;
+  }
+  
+  // Get flushed offset. Note that flushed data may not be persisted.
+  private long getFlushedOffset() {
+    return fos.getPos();
+  }
+  
+  // Check if need to dump the new writes
+  private void checkDump(long count) {
+    assert (ctxLock.isLocked());
+
+    // Always update the in memory count
+    updateNonSequentialWriteInMemory(count);
+
+    if (!enabledDump) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Do nothing, dump is disabled.");
+      }
+      return;
+    }
+
+    if (nonSequentialWriteInMemory < DUMP_WRITE_WATER_MARK) {
+      return;
+    }
+
+    // Create dump outputstream for the first time
+    if (dumpOut == null) {
+      LOG.info("Create dump file:" + dumpFilePath);
+      File dumpFile = new File(dumpFilePath);
+      try {
+        if (dumpFile.exists()) {
+          LOG.fatal("The dump file should not exist:" + dumpFilePath);
+          throw new RuntimeException("The dump file should not exist:"
+              + dumpFilePath);
+        }
+        dumpOut = new FileOutputStream(dumpFile);
+      } catch (IOException e) {
+        LOG.error("Got failure when creating dump stream " + dumpFilePath
+            + " with error:" + e);
+        enabledDump = false;
+        IOUtils.cleanup(LOG, dumpOut);
+        return;
+      }
+    }
+    // Get raf for the first dump
+    if (raf == null) {
+      try {
+        raf = new RandomAccessFile(dumpFilePath, "r");
+      } catch (FileNotFoundException e) {
+        LOG.error("Can't get random access to file " + dumpFilePath);
+        // Disable dump
+        enabledDump = false;
+        return;
+      }
+    }
+    
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Start dump, current write number:" + pendingWrites.size());
+    }
+    Iterator<OffsetRange> it = pendingWrites.keySet().iterator();
+    while (it.hasNext()) {
+      OffsetRange key = it.next();
+      WriteCtx writeCtx = pendingWrites.get(key);
+      try {
+        long dumpedDataSize = writeCtx.dumpData(dumpOut, raf);
+        if (dumpedDataSize > 0) {
+          updateNonSequentialWriteInMemory(-dumpedDataSize);
+        }
+      } catch (IOException e) {
+        LOG.error("Dump data failed:" + writeCtx + " with error:" + e);
+        // Disable dump
+        enabledDump = false;
+        return;
+      }
+    }
+    if (nonSequentialWriteInMemory != 0) {
+      LOG.fatal("After dump, nonSequentialWriteInMemory is not zero: "
+          + nonSequentialWriteInMemory);
+      throw new RuntimeException(
+          "After dump, nonSequentialWriteInMemory is not zero: "
+              + nonSequentialWriteInMemory);
+    }
+  }
+  
+  private WriteCtx checkRepeatedWriteRequest(WRITE3Request request,
+      Channel channel, int xid) {
+    OffsetRange range = new OffsetRange(request.getOffset(),
+        request.getOffset() + request.getCount());
+    WriteCtx writeCtx = pendingWrites.get(range);
+    if (writeCtx== null) {
+      return null;
+    } else {
+      if (xid != writeCtx.getXid()) {
+        LOG.warn("Got a repeated request, same range, with a different xid:"
+            + xid + " xid in old request:" + writeCtx.getXid());
+        //TODO: better handling.
+      }
+      return writeCtx;  
+    }
+  }
+  
+  public void receivedNewWrite(DFSClient dfsClient, WRITE3Request request,
+      Channel channel, int xid, AsyncDataService asyncDataService,
+      IdUserGroup iug) {
+
+    lockCtx();
+    try {
+      if (!activeState) {
+        LOG.info("OpenFileCtx is inactive, fileId:"
+            + request.getHandle().getFileId());
+        WccData fileWcc = new WccData(latestAttr.getWccAttr(), latestAttr);
+        WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_IO,
+            fileWcc, 0, request.getStableHow(), Nfs3Constant.WRITE_COMMIT_VERF);
+        Nfs3Utils.writeChannel(channel, response.send(new XDR(), xid));
+      } else {
+        // Handle repeated write requests(same xid or not).
+        // If already replied, send reply again. If not replied, drop the
+        // repeated request.
+        WriteCtx existantWriteCtx = checkRepeatedWriteRequest(request, channel,
+            xid);
+        if (existantWriteCtx != null) {
+          if (!existantWriteCtx.getReplied()) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Repeated write request which hasn't be served: xid="
+                  + xid + ", drop it.");
+            }
+          } else {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Repeated write request which is already served: xid="
+                  + xid + ", resend response.");
+            }
+            WccData fileWcc = new WccData(latestAttr.getWccAttr(), latestAttr);
+            WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK,
+                fileWcc, request.getCount(), request.getStableHow(),
+                Nfs3Constant.WRITE_COMMIT_VERF);
+            Nfs3Utils.writeChannel(channel, response.send(new XDR(), xid));
+          }
+          updateLastAccessTime();
+          
+        } else {
+          receivedNewWriteInternal(dfsClient, request, channel, xid,
+              asyncDataService, iug);
+        }
+      }
+
+    } finally {
+      unlockCtx();
+    }
+  }
+
+  private void receivedNewWriteInternal(DFSClient dfsClient,
+      WRITE3Request request, Channel channel, int xid,
+      AsyncDataService asyncDataService, IdUserGroup iug) {
+    long offset = request.getOffset();
+    int count = request.getCount();
+    WriteStableHow stableHow = request.getStableHow();
+
+    // Get file length, fail non-append call
+    WccAttr preOpAttr = latestAttr.getWccAttr();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("requesed offset=" + offset + " and current filesize="
+          + preOpAttr.getSize());
+    }
+
+    long nextOffset = getNextOffsetUnprotected();
+    if (offset == nextOffset) {
+      LOG.info("Add to the list, update nextOffset and notify the writer,"
+          + " nextOffset:" + nextOffset);
+      WriteCtx writeCtx = new WriteCtx(request.getHandle(),
+          request.getOffset(), request.getCount(), request.getStableHow(),
+          request.getData().array(), channel, xid, false, WriteCtx.NO_DUMP);
+      addWrite(writeCtx);
+      
+      // Create an async task and change openFileCtx status to indicate async
+      // task pending
+      if (!asyncStatus) {
+        asyncStatus = true;
+        asyncDataService.execute(new AsyncDataService.WriteBackTask(this));
+      }
+      
+      // Update the write time first
+      updateLastAccessTime();
+      Nfs3FileAttributes postOpAttr = new Nfs3FileAttributes(latestAttr);
+
+      // Send response immediately for unstable write
+      if (request.getStableHow() == WriteStableHow.UNSTABLE) {
+        WccData fileWcc = new WccData(preOpAttr, postOpAttr);
+        WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK,
+            fileWcc, count, stableHow, Nfs3Constant.WRITE_COMMIT_VERF);
+        Nfs3Utils.writeChannel(channel, response.send(new XDR(), xid));
+        writeCtx.setReplied(true);
+      }
+
+    } else if (offset > nextOffset) {
+      LOG.info("Add new write to the list but not update nextOffset:"
+          + nextOffset);
+      WriteCtx writeCtx = new WriteCtx(request.getHandle(),
+          request.getOffset(), request.getCount(), request.getStableHow(),
+          request.getData().array(), channel, xid, false, WriteCtx.ALLOW_DUMP);
+      addWrite(writeCtx);
+
+      // Check if need to dump some pending requests to file
+      checkDump(request.getCount());
+      updateLastAccessTime();
+      Nfs3FileAttributes postOpAttr = new Nfs3FileAttributes(latestAttr);
+      
+      // In test, noticed some Linux client sends a batch (e.g., 1MB)
+      // of reordered writes and won't send more writes until it gets
+      // responses of the previous batch. So here send response immediately for
+      // unstable non-sequential write
+      if (request.getStableHow() == WriteStableHow.UNSTABLE) {
+        WccData fileWcc = new WccData(preOpAttr, postOpAttr);
+        WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK,
+            fileWcc, count, stableHow, Nfs3Constant.WRITE_COMMIT_VERF);
+        Nfs3Utils.writeChannel(channel, response.send(new XDR(), xid));
+        writeCtx.setReplied(true);
+      }
+
+    } else {
+      // offset < nextOffset
+      LOG.warn("(offset,count,nextOffset):" + "(" + offset + "," + count + ","
+          + nextOffset + ")");
+      WccData wccData = new WccData(preOpAttr, null);
+      WRITE3Response response;
+
+      if (offset + count > nextOffset) {
+        LOG.warn("Haven't noticed any partial overwrite out of a sequential file"
+            + "write requests, so treat it as a real random write, no support.");
+        response = new WRITE3Response(Nfs3Status.NFS3ERR_INVAL, wccData, 0,
+            WriteStableHow.UNSTABLE, 0);
+      } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Process perfectOverWrite");
+        }
+        response = processPerfectOverWrite(dfsClient, offset, count, stableHow,
+            request.getData().array(),
+            Nfs3Utils.getFileIdPath(request.getHandle()), wccData, iug);
+      }
+      
+      updateLastAccessTime();
+      Nfs3Utils.writeChannel(channel, response.send(new XDR(), xid));
+    }
+  }
+  
+  /**
+   * Honor 2 kinds of overwrites: 1). support some application like touch(write
+   * the same content back to change mtime), 2) client somehow sends the same
+   * write again in a different RPC.
+   */
+  private WRITE3Response processPerfectOverWrite(DFSClient dfsClient,
+      long offset, int count, WriteStableHow stableHow, byte[] data,
+      String path, WccData wccData, IdUserGroup iug) {
+    assert (ctxLock.isLocked());
+    WRITE3Response response = null;
+
+    // Read the content back
+    byte[] readbuffer = new byte[count];
+
+    int readCount = 0;
+    FSDataInputStream fis = null;
+    try {
+      // Sync file data and length to avoid partial read failure
+      ((HdfsDataOutputStream) fos).hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
+      
+      fis = new FSDataInputStream(dfsClient.open(path));
+      readCount = fis.read(offset, readbuffer, 0, count);
+      if (readCount < count) {
+        LOG.error("Can't read back " + count + " bytes, partial read size:"
+            + readCount);
+        return response = new WRITE3Response(Nfs3Status.NFS3ERR_IO, wccData, 0,
+            stableHow, Nfs3Constant.WRITE_COMMIT_VERF);
+      }
+
+    } catch (IOException e) {
+      LOG.info("Read failed when processing possible perfect overwrite, path="
+          + path + " error:" + e);
+      return response = new WRITE3Response(Nfs3Status.NFS3ERR_IO, wccData, 0,
+          stableHow, Nfs3Constant.WRITE_COMMIT_VERF);
+    } finally {
+      IOUtils.cleanup(LOG, fis);
+    }
+
+    // Compare with the request
+    Comparator comparator = new Comparator();
+    if (comparator.compare(readbuffer, 0, readCount, data, 0, count) != 0) {
+      LOG.info("Perfect overwrite has different content");
+      response = new WRITE3Response(Nfs3Status.NFS3ERR_INVAL, wccData, 0,
+          stableHow, 0);
+    } else {
+      LOG.info("Perfect overwrite has same content,"
+          + " updating the mtime, then return success");
+      Nfs3FileAttributes postOpAttr = null;
+      try {
+        dfsClient.setTimes(path, System.currentTimeMillis(), -1);
+        postOpAttr = Nfs3Utils.getFileAttr(dfsClient, path, iug);
+      } catch (IOException e) {
+        LOG.info("Got error when processing perfect overwrite, path=" + path
+            + " error:" + e);
+        return new WRITE3Response(Nfs3Status.NFS3ERR_IO, wccData, 0, stableHow,
+            0);
+      }
+
+      wccData.setPostOpAttr(postOpAttr);
+      response = new WRITE3Response(Nfs3Status.NFS3_OK, wccData, count,
+          stableHow, 0);
+    }
+    return response;
+  }
+  
+  public final static int COMMIT_FINISHED = 0;
+  public final static int COMMIT_WAIT = 1;
+  public final static int COMMIT_INACTIVE_CTX = 2;
+  public final static int COMMIT_ERROR = 3;
+
+  /**
+   * return one commit status: COMMIT_FINISHED, COMMIT_WAIT,
+   * COMMIT_INACTIVE_CTX, COMMIT_ERROR
+   */
+  public int checkCommit(long commitOffset) {
+    int ret = COMMIT_WAIT;
+
+    lockCtx();
+    try {
+      if (!activeState) {
+        ret = COMMIT_INACTIVE_CTX;
+      } else {
+        ret = checkCommitInternal(commitOffset);
+      }
+    } finally {
+      unlockCtx();
+    }
+    return ret;
+  }
+  
+  private int checkCommitInternal(long commitOffset) {
+    if (commitOffset == 0) {
+      // Commit whole file
+      commitOffset = getNextOffsetUnprotected();
+    }
+
+    long flushed = getFlushedOffset();
+    LOG.info("getFlushedOffset=" + flushed + " commitOffset=" + commitOffset);
+    if (flushed < commitOffset) {
+      // Keep stream active
+      updateLastAccessTime();
+      return COMMIT_WAIT;
+    }
+
+    int ret = COMMIT_WAIT;
+    try {
+      // Sync file data and length
+      ((HdfsDataOutputStream) fos).hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
+      // Nothing to do for metadata since attr related change is pass-through
+      ret = COMMIT_FINISHED;
+    } catch (IOException e) {
+      LOG.error("Got stream error during data sync:" + e);
+      // Do nothing. Stream will be closed eventually by StreamMonitor.
+      ret = COMMIT_ERROR;
+    }
+
+    // Keep stream active
+    updateLastAccessTime();
+    return ret;
+  }
+  
+  private void addWrite(WriteCtx writeCtx) {
+    assert (ctxLock.isLocked());
+    long offset = writeCtx.getOffset();
+    int count = writeCtx.getCount();
+    pendingWrites.put(new OffsetRange(offset, offset + count), writeCtx);
+  }
+  
+  
+  /**
+   * Check stream status to decide if it should be closed
+   * @return true, remove stream; false, keep stream
+   */
+  public boolean streamCleanup(long fileId, long streamTimeout) {
+    if (streamTimeout < WriteManager.MINIMIUM_STREAM_TIMEOUT) {
+      throw new InvalidParameterException("StreamTimeout" + streamTimeout
+          + "ms is less than MINIMIUM_STREAM_TIMEOUT "
+          + WriteManager.MINIMIUM_STREAM_TIMEOUT + "ms");
+    }
+    
+    boolean flag = false;
+    if (!ctxLock.tryLock()) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Another thread is working on it" + ctxLock.toString());
+      }
+      return flag;
+    }
+    
+    try {
+      // Check the stream timeout
+      if (checkStreamTimeout(streamTimeout)) {
+        LOG.info("closing stream for fileId:" + fileId);
+        cleanup();
+        flag = true;
+      }
+    } finally {
+      unlockCtx();
+    }
+    return flag;
+  }
+  
+  // Invoked by AsynDataService to do the write back
+  public void executeWriteBack() {
+    long nextOffset;
+    OffsetRange key;
+    WriteCtx writeCtx;
+
+    try {
+      // Don't lock OpenFileCtx for all writes to reduce the timeout of other
+      // client request to the same file
+      while (true) {
+        lockCtx();
+        if (!asyncStatus) {
+          // This should never happen. There should be only one thread working
+          // on one OpenFileCtx anytime.
+          LOG.fatal("The openFileCtx has false async status");
+          throw new RuntimeException("The openFileCtx has false async status");
+        }
+        // Any single write failure can change activeState to false, so do the
+        // check each loop.
+        if (pendingWrites.isEmpty()) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("The asyn write task has no pendding writes, fileId: "
+                + latestAttr.getFileId());
+          }
+          break;
+        }
+        if (!activeState) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("The openFileCtx is not active anymore, fileId: "
+                + latestAttr.getFileId());
+          }
+          break;
+        }
+
+        // Get the next sequential write
+        nextOffset = getNextOffsetUnprotected();
+        key = pendingWrites.firstKey();
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("key.getMin()=" + key.getMin() + " nextOffset="
+              + nextOffset);
+        }
+
+        if (key.getMin() > nextOffset) {
+          if (LOG.isDebugEnabled()) {
+            LOG.info("The next sequencial write has not arrived yet");
+          }
+          break;
+
+        } else if (key.getMin() < nextOffset && key.getMax() > nextOffset) {
+          // Can't handle overlapping write. Didn't see it in tests yet.
+          LOG.fatal("Got a overlapping write (" + key.getMin() + ","
+              + key.getMax() + "), nextOffset=" + nextOffset);
+          throw new RuntimeException("Got a overlapping write (" + key.getMin()
+              + "," + key.getMax() + "), nextOffset=" + nextOffset);
+
+        } else {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Remove write(" + key.getMin() + "-" + key.getMax()
+                + ") from the list");
+          }
+          writeCtx = pendingWrites.remove(key);
+          // Do the write
+          doSingleWrite(writeCtx);
+          updateLastAccessTime();
+        }
+        
+        unlockCtx();
+      }
+
+    } finally {
+      // Always reset the async status so another async task can be created
+      // for this file
+      asyncStatus = false;
+      if (ctxLock.isHeldByCurrentThread()) {
+        unlockCtx();
+      }
+    }
+  }
+
+  private void doSingleWrite(final WriteCtx writeCtx) {
+    assert(ctxLock.isLocked());
+    Channel channel = writeCtx.getChannel();
+    int xid = writeCtx.getXid();
+
+    long offset = writeCtx.getOffset();
+    int count = writeCtx.getCount();
+    WriteStableHow stableHow = writeCtx.getStableHow();
+    byte[] data = null;
+    try {
+      data = writeCtx.getData();
+    } catch (IOException e1) {
+      LOG.error("Failed to get request data offset:" + offset + " count:"
+          + count + " error:" + e1);
+      // Cleanup everything
+      cleanup();
+      return;
+    }
+    assert (data.length == count);
+
+    FileHandle handle = writeCtx.getHandle();
+    LOG.info("do write, fileId: " + handle.getFileId() + " offset: " + offset
+        + " length:" + count + " stableHow:" + stableHow.getValue());
+
+    try {
+      fos.write(data, 0, count);
+
+      if (fos.getPos() != (offset + count)) {
+        throw new IOException("output stream is out of sync, pos="
+            + fos.getPos() + " and nextOffset should be" + (offset + count));
+      }
+      nextOffset = fos.getPos();
+
+      // Reduce memory occupation size if request was allowed dumped
+      if (writeCtx.getDataState() == WriteCtx.ALLOW_DUMP) {
+        updateNonSequentialWriteInMemory(-count);
+      }
+      
+      if (!writeCtx.getReplied()) {
+        WccAttr preOpAttr = latestAttr.getWccAttr();
+        WccData fileWcc = new WccData(preOpAttr, latestAttr);
+        WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK,
+            fileWcc, count, stableHow, Nfs3Constant.WRITE_COMMIT_VERF);
+        Nfs3Utils.writeChannel(channel, response.send(new XDR(), xid));
+      }
+
+    } catch (IOException e) {
+      LOG.error("Error writing to fileId " + handle.getFileId() + " at offset "
+          + offset + " and length " + data.length, e);
+      if (!writeCtx.getReplied()) {
+        WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_IO);
+        Nfs3Utils.writeChannel(channel, response.send(new XDR(), xid));
+        // Keep stream open. Either client retries or SteamMonitor closes it.
+      }
+
+      LOG.info("Clean up open file context for fileId: "
+          + latestAttr.getFileid());
+      cleanup();
+    }
+  }
+
+  private void cleanup() {
+    assert(ctxLock.isLocked());
+    activeState = false;
+    
+    // Close stream
+    try {
+      if (fos != null) {
+        fos.close();
+      }
+    } catch (IOException e) {
+      LOG.info("Can't close stream for fileId:" + latestAttr.getFileid()
+          + ", error:" + e);
+    }
+    
+    // Reply error for pending writes
+    LOG.info("There are " + pendingWrites.size() + " pending writes.");
+    WccAttr preOpAttr = latestAttr.getWccAttr();
+    while (!pendingWrites.isEmpty()) {
+      OffsetRange key = pendingWrites.firstKey();
+      LOG.info("Fail pending write: (" + key.getMin() + "," + key.getMax()
+          + "), nextOffset=" + getNextOffsetUnprotected());
+      
+      WriteCtx writeCtx = pendingWrites.remove(key);
+      if (!writeCtx.getReplied()) {
+        WccData fileWcc = new WccData(preOpAttr, latestAttr);
+        WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_IO,
+            fileWcc, 0, writeCtx.getStableHow(), Nfs3Constant.WRITE_COMMIT_VERF);
+        Nfs3Utils.writeChannel(writeCtx.getChannel(),
+            response.send(new XDR(), writeCtx.getXid()));
+      }
+    }
+    
+    // Cleanup dump file
+    if (dumpOut!=null){
+      try {
+        dumpOut.close();
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+    }
+    if (raf!=null) {
+      try {
+        raf.close();
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+    }
+    File dumpFile = new File(dumpFilePath);
+    if (dumpFile.delete()) {
+      LOG.error("Failed to delete dumpfile: "+ dumpFile);
+    }
+  }
+}

+ 1622 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java

@@ -0,0 +1,1622 @@
+/**
+ * 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.hdfs.nfs.nfs3;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.EnumSet;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FsStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSInputStream;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.nfs.NfsTime;
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.nfs.nfs3.IdUserGroup;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant.WriteStableHow;
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Interface;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.nfs.nfs3.request.ACCESS3Request;
+import org.apache.hadoop.nfs.nfs3.request.COMMIT3Request;
+import org.apache.hadoop.nfs.nfs3.request.CREATE3Request;
+import org.apache.hadoop.nfs.nfs3.request.FSINFO3Request;
+import org.apache.hadoop.nfs.nfs3.request.FSSTAT3Request;
+import org.apache.hadoop.nfs.nfs3.request.GETATTR3Request;
+import org.apache.hadoop.nfs.nfs3.request.LOOKUP3Request;
+import org.apache.hadoop.nfs.nfs3.request.MKDIR3Request;
+import org.apache.hadoop.nfs.nfs3.request.PATHCONF3Request;
+import org.apache.hadoop.nfs.nfs3.request.READ3Request;
+import org.apache.hadoop.nfs.nfs3.request.READDIR3Request;
+import org.apache.hadoop.nfs.nfs3.request.READDIRPLUS3Request;
+import org.apache.hadoop.nfs.nfs3.request.REMOVE3Request;
+import org.apache.hadoop.nfs.nfs3.request.RENAME3Request;
+import org.apache.hadoop.nfs.nfs3.request.RMDIR3Request;
+import org.apache.hadoop.nfs.nfs3.request.SETATTR3Request;
+import org.apache.hadoop.nfs.nfs3.request.SetAttr3;
+import org.apache.hadoop.nfs.nfs3.request.SetAttr3.SetAttrField;
+import org.apache.hadoop.nfs.nfs3.request.WRITE3Request;
+import org.apache.hadoop.nfs.nfs3.response.ACCESS3Response;
+import org.apache.hadoop.nfs.nfs3.response.COMMIT3Response;
+import org.apache.hadoop.nfs.nfs3.response.CREATE3Response;
+import org.apache.hadoop.nfs.nfs3.response.FSINFO3Response;
+import org.apache.hadoop.nfs.nfs3.response.FSSTAT3Response;
+import org.apache.hadoop.nfs.nfs3.response.GETATTR3Response;
+import org.apache.hadoop.nfs.nfs3.response.LOOKUP3Response;
+import org.apache.hadoop.nfs.nfs3.response.MKDIR3Response;
+import org.apache.hadoop.nfs.nfs3.response.NFS3Response;
+import org.apache.hadoop.nfs.nfs3.response.PATHCONF3Response;
+import org.apache.hadoop.nfs.nfs3.response.READ3Response;
+import org.apache.hadoop.nfs.nfs3.response.READDIR3Response;
+import org.apache.hadoop.nfs.nfs3.response.READDIR3Response.DirList3;
+import org.apache.hadoop.nfs.nfs3.response.READDIR3Response.Entry3;
+import org.apache.hadoop.nfs.nfs3.response.READDIRPLUS3Response;
+import org.apache.hadoop.nfs.nfs3.response.READDIRPLUS3Response.DirListPlus3;
+import org.apache.hadoop.nfs.nfs3.response.READLINK3Response;
+import org.apache.hadoop.nfs.nfs3.response.REMOVE3Response;
+import org.apache.hadoop.nfs.nfs3.response.RENAME3Response;
+import org.apache.hadoop.nfs.nfs3.response.RMDIR3Response;
+import org.apache.hadoop.nfs.nfs3.response.SETATTR3Response;
+import org.apache.hadoop.nfs.nfs3.response.SYMLINK3Response;
+import org.apache.hadoop.nfs.nfs3.response.VoidResponse;
+import org.apache.hadoop.nfs.nfs3.response.WRITE3Response;
+import org.apache.hadoop.nfs.nfs3.response.WccAttr;
+import org.apache.hadoop.nfs.nfs3.response.WccData;
+import org.apache.hadoop.oncrpc.RpcAcceptedReply;
+import org.apache.hadoop.oncrpc.RpcAuthInfo.AuthFlavor;
+import org.apache.hadoop.oncrpc.RpcAuthSys;
+import org.apache.hadoop.oncrpc.RpcCall;
+import org.apache.hadoop.oncrpc.RpcDeniedReply;
+import org.apache.hadoop.oncrpc.RpcProgram;
+import org.apache.hadoop.oncrpc.RpcReply;
+import org.apache.hadoop.oncrpc.XDR;
+import org.apache.hadoop.security.AccessControlException;
+import org.jboss.netty.channel.Channel;
+
+/**
+ * RPC program corresponding to nfs daemon. See {@link Nfs3}.
+ */
+public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
+  public static final int DEFAULT_UMASK = 0022;
+  public static final FsPermission umask = new FsPermission(
+      (short) DEFAULT_UMASK);
+  
+  private static final Log LOG = LogFactory.getLog(RpcProgramNfs3.class);
+  private static final int MAX_READ_TRANSFER_SIZE = 64 * 1024;
+  private static final int MAX_WRITE_TRANSFER_SIZE = 64 * 1024;
+  private static final int MAX_READDIR_TRANSFER_SIZE = 64 * 1024;
+
+  private final Configuration config = new Configuration();
+  private final WriteManager writeManager;
+  private final IdUserGroup iug;// = new IdUserGroup();
+  private final DFSClientCache clientCache;
+
+  /**
+   * superUserClient should always impersonate HDFS file system owner to send
+   * requests which requires supergroup privilege. This requires the same user
+   * to start HDFS and NFS.
+   */
+  private final DFSClient superUserClient;
+  
+  private final short replication;
+  private final long blockSize;
+  private final int bufferSize;
+  private Statistics statistics;
+  private String writeDumpDir; // The dir save dump files
+  
+  public RpcProgramNfs3(List<String> exports) throws IOException {
+    this(exports, new Configuration());
+  }
+
+  public RpcProgramNfs3(List<String> exports, Configuration config)
+      throws IOException {
+    super("NFS3", "localhost", Nfs3Constant.PORT, Nfs3Constant.PROGRAM,
+        Nfs3Constant.VERSION, Nfs3Constant.VERSION, 100);
+   
+    config.set(FsPermission.UMASK_LABEL, "000");
+    iug = new IdUserGroup();
+    writeManager = new WriteManager(iug, config);
+    clientCache = new DFSClientCache(config);
+    superUserClient = new DFSClient(NameNode.getAddress(config), config);
+    replication = (short) config.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
+        DFSConfigKeys.DFS_REPLICATION_DEFAULT);
+    blockSize = config.getLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY,
+        DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT);
+    bufferSize = config.getInt("io.file.buffer.size", 4096);
+    
+    writeDumpDir = config.get("dfs.nfs3.dump.dir", "/tmp/.hdfs-nfs");    
+    boolean enableDump = config.getBoolean("dfs.nfs3.enableDump", true);
+    if (!enableDump) {
+      writeDumpDir = null;
+    } else {
+      clearDirectory(writeDumpDir);
+    }
+  }
+
+  private void clearDirectory(String writeDumpDir) throws IOException {
+    File dumpDir = new File(writeDumpDir);
+    if (dumpDir.exists()) {
+      LOG.info("Delete current dump directory " + writeDumpDir);
+      if (!(FileUtil.fullyDelete(dumpDir))) {
+        throw new IOException("Cannot remove current dump directory: "
+            + dumpDir);
+      }
+    }
+    LOG.info("Create new dump directory " + writeDumpDir);
+    if (!dumpDir.mkdirs()) {
+      throw new IOException("Cannot create dump directory " + dumpDir);
+    }
+  }
+  
+  /******************************************************
+   * RPC call handlers
+   ******************************************************/
+  
+  public NFS3Response nullProcedure() {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS NULL");
+    }
+    return new VoidResponse(Nfs3Status.NFS3_OK);
+  }
+
+  public GETATTR3Response getattr(XDR xdr, RpcAuthSys authSys) {
+    GETATTR3Response response = new GETATTR3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    GETATTR3Request request = null;
+    try {
+      request = new GETATTR3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid GETATTR request");
+      response.setStatus(Nfs3Status.NFS3ERR_INVAL);
+      return response;
+    }
+
+    FileHandle handle = request.getHandle();
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("GETATTR for fileId: " + handle.getFileId());
+    }
+
+    Nfs3FileAttributes attrs = null;
+    try {
+      attrs = writeManager.getFileAttr(dfsClient, handle, iug);
+    } catch (IOException e) {
+      LOG.info("Can't get file attribute, fileId=" + handle.getFileId());
+      response.setStatus(Nfs3Status.NFS3ERR_IO);
+      return response;
+    }
+    if (attrs == null) {
+      LOG.error("Can't get path for fileId:" + handle.getFileId());
+      response.setStatus(Nfs3Status.NFS3ERR_STALE);
+      return response;
+    }
+    response.setPostOpAttr(attrs);
+    return response;
+  }
+
+  // Set attribute, don't support setting "size". For file/dir creation, mode is
+  // set during creation and setMode should be false here.
+  private void setattrInternal(DFSClient dfsClient, String fileIdPath,
+      SetAttr3 newAttr, boolean setMode) throws IOException {
+    EnumSet<SetAttrField> updateFields = newAttr.getUpdateFields();
+    
+    if (setMode && updateFields.contains(SetAttrField.MODE)) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("set new mode:" + newAttr.getMode());
+      }
+      dfsClient.setPermission(fileIdPath,
+          new FsPermission((short) (newAttr.getMode())));
+    }
+    if (updateFields.contains(SetAttrField.UID)
+        || updateFields.contains(SetAttrField.GID)) {
+      String uname = updateFields.contains(SetAttrField.UID) ? iug.getUserName(
+          newAttr.getUid(), UNKNOWN_USER) : null;
+      String gname = updateFields.contains(SetAttrField.GID) ? iug
+          .getGroupName(newAttr.getGid(), UNKNOWN_GROUP) : null;
+      dfsClient.setOwner(fileIdPath, uname, gname);
+    }
+
+    long atime = updateFields.contains(SetAttrField.ATIME) ? newAttr.getAtime()
+        .getMilliSeconds() : -1;
+    long mtime = updateFields.contains(SetAttrField.MTIME) ? newAttr.getMtime()
+        .getMilliSeconds() : -1;
+    if (atime != -1 || mtime != -1) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("set atime:" + +atime + " mtime:" + mtime);
+      }
+      dfsClient.setTimes(fileIdPath, mtime, atime);
+    }
+  }
+
+  public SETATTR3Response setattr(XDR xdr, RpcAuthSys authSys) {
+    SETATTR3Response response = new SETATTR3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    SETATTR3Request request = null;
+    try {
+      request = new SETATTR3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid SETATTR request");
+      response.setStatus(Nfs3Status.NFS3ERR_INVAL);
+      return response;
+    }
+
+    FileHandle handle = request.getHandle();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS SETATTR fileId: " + handle.getFileId());
+    }
+
+    if (request.getAttr().getUpdateFields().contains(SetAttrField.SIZE)) {
+      LOG.error("Setting file size is not supported when setattr, fileId: "
+          + handle.getFileId());
+      response.setStatus(Nfs3Status.NFS3ERR_INVAL);
+      return response;
+    }
+
+    String fileIdPath = Nfs3Utils.getFileIdPath(handle);
+    WccAttr preOpAttr = null;
+    try {
+      preOpAttr = Nfs3Utils.getWccAttr(dfsClient, fileIdPath);
+      if (preOpAttr == null) {
+        LOG.info("Can't get path for fileId:" + handle.getFileId());
+        response.setStatus(Nfs3Status.NFS3ERR_STALE);
+        return response;
+      }
+      if (request.isCheck()) {
+        if (!preOpAttr.getCtime().equals(request.getCtime())) {
+          WccData wccData = Nfs3Utils.createWccData(preOpAttr, dfsClient,
+              fileIdPath, iug);
+          return new SETATTR3Response(Nfs3Status.NFS3ERR_NOT_SYNC, wccData);
+        }
+      }
+
+      setattrInternal(dfsClient, fileIdPath, request.getAttr(), true);
+      Nfs3FileAttributes postOpAttr = Nfs3Utils.getFileAttr(dfsClient,
+          fileIdPath, iug);
+      WccData wccData = new WccData(preOpAttr, postOpAttr);
+      return new SETATTR3Response(Nfs3Status.NFS3_OK, wccData);
+
+    } catch (IOException e) {
+      LOG.warn("Exception ", e);
+      WccData wccData = null;
+      try {
+        wccData = Nfs3Utils
+            .createWccData(preOpAttr, dfsClient, fileIdPath, iug);
+      } catch (IOException e1) {
+        LOG.info("Can't get postOpAttr for fileIdPath: " + fileIdPath);
+      }
+      if (e instanceof AccessControlException) {
+        return new SETATTR3Response(Nfs3Status.NFS3ERR_ACCES, wccData);
+      } else {
+        return new SETATTR3Response(Nfs3Status.NFS3ERR_IO, wccData);
+      }
+    }
+  }
+
+  public LOOKUP3Response lookup(XDR xdr, RpcAuthSys authSys) {
+    LOOKUP3Response response = new LOOKUP3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    LOOKUP3Request request = null;
+    try {
+      request = new LOOKUP3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid LOOKUP request");
+      return new LOOKUP3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+
+    FileHandle dirHandle = request.getHandle();
+    String fileName = request.getName();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS LOOKUP dir fileId: " + dirHandle.getFileId() + " name: "
+          + fileName);
+    }
+
+    try {
+      String dirFileIdPath = Nfs3Utils.getFileIdPath(dirHandle);
+      Nfs3FileAttributes postOpObjAttr = writeManager.getFileAttr(dfsClient,
+          dirHandle, fileName);
+      if (postOpObjAttr == null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("NFS LOOKUP fileId: " + dirHandle.getFileId() + " name:"
+              + fileName + " does not exist");
+        }
+        Nfs3FileAttributes postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient,
+            dirFileIdPath, iug);
+        return new LOOKUP3Response(Nfs3Status.NFS3ERR_NOENT, null, null,
+            postOpDirAttr);
+      }
+
+      Nfs3FileAttributes postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient,
+          dirFileIdPath, iug);
+      if (postOpDirAttr == null) {
+        LOG.info("Can't get path for dir fileId:" + dirHandle.getFileId());
+        return new LOOKUP3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+      FileHandle fileHandle = new FileHandle(postOpObjAttr.getFileId());
+      return new LOOKUP3Response(Nfs3Status.NFS3_OK, fileHandle, postOpObjAttr,
+          postOpDirAttr);
+
+    } catch (IOException e) {
+      LOG.warn("Exception ", e);
+      return new LOOKUP3Response(Nfs3Status.NFS3ERR_IO);
+    }
+  }
+  
+  public ACCESS3Response access(XDR xdr, RpcAuthSys authSys) {
+    ACCESS3Response response = new ACCESS3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    ACCESS3Request request = null;
+    try {
+      request = new ACCESS3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid ACCESS request");
+      return new ACCESS3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+
+    FileHandle handle = request.getHandle();
+    Nfs3FileAttributes attrs;
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS ACCESS fileId: " + handle.getFileId());
+    } 
+
+    try {
+      // Use superUserClient to get file attr since we don't know whether the
+      // NFS client user has access permission to the file
+      attrs = writeManager.getFileAttr(superUserClient, handle, iug);
+      if (attrs == null) {
+        LOG.error("Can't get path for fileId:" + handle.getFileId());
+        return new ACCESS3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+      int access = Nfs3Utils.getAccessRightsForUserGroup(authSys.getUid(),
+          authSys.getGid(), attrs);
+      
+      return new ACCESS3Response(Nfs3Status.NFS3_OK, attrs, access);
+    } catch (IOException e) {
+      LOG.warn("Exception ", e);
+      return new ACCESS3Response(Nfs3Status.NFS3ERR_IO);
+    }
+  }
+
+  public READLINK3Response readlink(XDR xdr, RpcAuthSys authSys) {
+    return new READLINK3Response(Nfs3Status.NFS3ERR_NOTSUPP);
+  }
+
+  public READ3Response read(XDR xdr, RpcAuthSys authSys) {
+    READ3Response response = new READ3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    READ3Request request = null;
+
+    try {
+      request = new READ3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid READ request");
+      return new READ3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+
+    long offset = request.getOffset();
+    int count = request.getCount();
+
+    
+    FileHandle handle = request.getHandle();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS READ fileId: " + handle.getFileId() + " offset: " + offset
+          + " count: " + count);
+    }
+
+    Nfs3FileAttributes attrs;
+    boolean eof;
+    if (count == 0) {
+      // Only do access check.
+      try {
+        // Don't read from cache. Client may not have read permission.
+        attrs = Nfs3Utils.getFileAttr(superUserClient,
+            Nfs3Utils.getFileIdPath(handle), iug);
+      } catch (IOException e) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Get error accessing file, fileId:" + handle.getFileId());
+        }
+        return new READ3Response(Nfs3Status.NFS3ERR_IO);
+      }
+      if (attrs == null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Can't get path for fileId:" + handle.getFileId());
+        }
+        return new READ3Response(Nfs3Status.NFS3ERR_NOENT);
+      }
+      int access = Nfs3Utils.getAccessRightsForUserGroup(authSys.getUid(),
+          authSys.getGid(), attrs);
+      if ((access & Nfs3Constant.ACCESS3_READ) != 0) {
+        eof = offset < attrs.getSize() ? false : true;
+        return new READ3Response(Nfs3Status.NFS3_OK, attrs, 0, eof,
+            ByteBuffer.wrap(new byte[0]));
+      } else {
+        return new READ3Response(Nfs3Status.NFS3ERR_ACCES);
+      }
+    }
+    
+    try {
+      int buffSize = Math.min(MAX_READ_TRANSFER_SIZE, count);
+      byte[] readbuffer = new byte[buffSize];
+
+      DFSInputStream is = dfsClient.open(Nfs3Utils.getFileIdPath(handle));
+      FSDataInputStream fis = new FSDataInputStream(is);
+      
+      int readCount = fis.read(offset, readbuffer, 0, count);
+      fis.close();
+
+      attrs = Nfs3Utils.getFileAttr(dfsClient, Nfs3Utils.getFileIdPath(handle),
+          iug);
+      if (readCount < count) {
+        LOG.info("Partical read. Asked offset:" + offset + " count:" + count
+            + " and read back:" + readCount + "file size:" + attrs.getSize());
+      }
+      // HDFS returns -1 for read beyond file size.
+      if (readCount < 0) {
+        readCount = 0;
+      }
+      eof = (offset + readCount) < attrs.getSize() ? false : true;
+      return new READ3Response(Nfs3Status.NFS3_OK, attrs, readCount, eof,
+          ByteBuffer.wrap(readbuffer));
+
+    } catch (IOException e) {
+      LOG.warn("Read error: " + e.getClass() + " offset: " + offset
+          + " count: " + count, e);
+      return new READ3Response(Nfs3Status.NFS3ERR_IO);
+    }
+  }
+
+  public WRITE3Response write(XDR xdr, Channel channel, int xid,
+      RpcAuthSys authSys) {
+    WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    WRITE3Request request = null;
+
+    try {
+      request = new WRITE3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid WRITE request");
+      return new WRITE3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+
+    long offset = request.getOffset();
+    int count = request.getCount();
+    WriteStableHow stableHow = request.getStableHow();
+    byte[] data = request.getData().array();
+    if (data.length < count) {
+      LOG.error("Invalid argument, data size is less than count in request");
+      return new WRITE3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+
+    FileHandle handle = request.getHandle();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS WRITE fileId: " + handle.getFileId() + " offset: "
+          + offset + " length:" + count + " stableHow:" + stableHow.getValue()
+          + " xid:" + xid);
+    }
+
+    Nfs3FileAttributes preOpAttr = null;
+    try {
+      preOpAttr = writeManager.getFileAttr(dfsClient, handle, iug);
+      if (preOpAttr == null) {
+        LOG.error("Can't get path for fileId:" + handle.getFileId());
+        return new WRITE3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("requesed offset=" + offset + " and current filesize="
+            + preOpAttr.getSize());
+      }
+
+      writeManager.handleWrite(dfsClient, request, channel, xid, preOpAttr);
+
+    } catch (IOException e) {
+      LOG.info("Error writing to fileId " + handle.getFileId() + " at offset "
+          + offset + " and length " + data.length, e);
+      // Try to return WccData
+      Nfs3FileAttributes postOpAttr = null;
+      try {
+        postOpAttr = writeManager.getFileAttr(dfsClient, handle, iug);
+      } catch (IOException e1) {
+        LOG.info("Can't get postOpAttr for fileId: " + handle.getFileId());
+      }
+      WccAttr attr = preOpAttr == null ? null : Nfs3Utils.getWccAttr(preOpAttr);
+      WccData fileWcc = new WccData(attr, postOpAttr);
+      return new WRITE3Response(Nfs3Status.NFS3ERR_IO, fileWcc, 0,
+          request.getStableHow(), Nfs3Constant.WRITE_COMMIT_VERF);
+    }
+
+    return null;
+  }
+
+  public CREATE3Response create(XDR xdr, RpcAuthSys authSys) {
+    CREATE3Response response = new CREATE3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    CREATE3Request request = null;
+
+    try {
+      request = new CREATE3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid CREATE request");
+      return new CREATE3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+
+    FileHandle dirHandle = request.getHandle();
+    String fileName = request.getName();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS CREATE dir fileId: " + dirHandle.getFileId()
+          + " filename: " + fileName);
+    }
+
+    int createMode = request.getMode();
+    if ((createMode != Nfs3Constant.CREATE_EXCLUSIVE)
+        && request.getObjAttr().getUpdateFields().contains(SetAttrField.SIZE)) {
+      LOG.error("Setting file size is not supported when creating file: "
+          + fileName + " dir fileId:" + dirHandle.getFileId());
+      return new CREATE3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+
+    FSDataOutputStream fos = null;
+    String dirFileIdPath = Nfs3Utils.getFileIdPath(dirHandle);
+    WccAttr preOpDirAttr = null;
+    Nfs3FileAttributes postOpObjAttr = null;
+    FileHandle fileHandle = null;
+    WccData dirWcc = null;
+    try {
+      preOpDirAttr = Nfs3Utils.getWccAttr(dfsClient, dirFileIdPath);
+      if (preOpDirAttr == null) {
+        LOG.error("Can't get path for dirHandle:" + dirHandle);
+        return new CREATE3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+
+      String fileIdPath = Nfs3Utils.getFileIdPath(dirHandle) + "/" + fileName;
+      SetAttr3 setAttr3 = request.getObjAttr();
+      assert (setAttr3 != null);
+      FsPermission permission = setAttr3.getUpdateFields().contains(
+          SetAttrField.MODE) ? new FsPermission((short) setAttr3.getMode())
+          : FsPermission.getDefault().applyUMask(umask);
+          
+      EnumSet<CreateFlag> flag = (createMode != Nfs3Constant.CREATE_EXCLUSIVE) ? EnumSet
+          .of(CreateFlag.CREATE, CreateFlag.OVERWRITE) : EnumSet
+          .of(CreateFlag.CREATE);
+      fos = new FSDataOutputStream(dfsClient.create(fileIdPath, permission,
+          flag, false, replication, blockSize, null, bufferSize, null),
+          statistics);
+      
+      if ((createMode == Nfs3Constant.CREATE_UNCHECKED)
+          || (createMode == Nfs3Constant.CREATE_GUARDED)) {
+        // Set group if it's not specified in the request.
+        if (!setAttr3.getUpdateFields().contains(SetAttrField.GID)) {
+          setAttr3.getUpdateFields().add(SetAttrField.GID);
+          setAttr3.setGid(authSys.getGid());
+        }
+        setattrInternal(dfsClient, fileIdPath, setAttr3, false);
+      }
+
+      postOpObjAttr = Nfs3Utils.getFileAttr(dfsClient, fileIdPath, iug);
+      dirWcc = Nfs3Utils.createWccData(preOpDirAttr, dfsClient, dirFileIdPath,
+          iug);
+    } catch (IOException e) {
+      LOG.error("Exception", e);
+      if (fos != null) {
+        try {
+          fos.close();
+        } catch (IOException e1) {
+          LOG.error("Can't close stream for dirFileId:" + dirHandle.getFileId()
+              + " filename: " + fileName);
+        }
+      }
+      if (dirWcc == null) {
+        try {
+          dirWcc = Nfs3Utils.createWccData(preOpDirAttr, dfsClient,
+              dirFileIdPath, iug);
+        } catch (IOException e1) {
+          LOG.error("Can't get postOpDirAttr for dirFileId:"
+              + dirHandle.getFileId());
+        }
+      }
+      if (e instanceof AccessControlException) {
+        return new CREATE3Response(Nfs3Status.NFS3ERR_ACCES, fileHandle,
+            postOpObjAttr, dirWcc);
+      } else {
+        return new CREATE3Response(Nfs3Status.NFS3ERR_IO, fileHandle,
+            postOpObjAttr, dirWcc);
+      }
+    }
+    
+    // Add open stream
+    OpenFileCtx openFileCtx = new OpenFileCtx(fos, postOpObjAttr, writeDumpDir
+        + "/" + postOpObjAttr.getFileId());
+    fileHandle = new FileHandle(postOpObjAttr.getFileId());
+    writeManager.addOpenFileStream(fileHandle, openFileCtx);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("open stream for file:" + fileName + ", fileId:"
+          + fileHandle.getFileId());
+    }
+    
+    return new CREATE3Response(Nfs3Status.NFS3_OK, fileHandle, postOpObjAttr,
+        dirWcc);
+  }
+
+  public MKDIR3Response mkdir(XDR xdr, RpcAuthSys authSys) {
+    MKDIR3Response response = new MKDIR3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    MKDIR3Request request = null;
+
+    try {
+      request = new MKDIR3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid MKDIR request");
+      return new MKDIR3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+    FileHandle dirHandle = request.getHandle();
+    String fileName = request.getName();
+
+    if (request.getObjAttr().getUpdateFields().contains(SetAttrField.SIZE)) {
+      LOG.error("Setting file size is not supported when mkdir: " + fileName
+          + " in dirHandle" + dirHandle);
+      return new MKDIR3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+
+    String dirFileIdPath = Nfs3Utils.getFileIdPath(dirHandle);
+    WccAttr preOpDirAttr = null;
+    Nfs3FileAttributes postOpDirAttr = null;
+    Nfs3FileAttributes postOpObjAttr = null;
+    FileHandle objFileHandle = null;
+    try {
+      preOpDirAttr = Nfs3Utils.getWccAttr(dfsClient, dirFileIdPath);
+      if (preOpDirAttr == null) {
+        LOG.info("Can't get path for dir fileId:" + dirHandle.getFileId());
+        return new MKDIR3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+
+      final String fileIdPath = dirFileIdPath + "/" + fileName;
+      SetAttr3 setAttr3 = request.getObjAttr();
+      FsPermission permission = setAttr3.getUpdateFields().contains(
+          SetAttrField.MODE) ? new FsPermission((short) setAttr3.getMode())
+          : FsPermission.getDefault().applyUMask(umask);
+
+      if (!dfsClient.mkdirs(fileIdPath, permission, false)) {
+        WccData dirWcc = Nfs3Utils.createWccData(preOpDirAttr, dfsClient,
+            dirFileIdPath, iug);
+        return new MKDIR3Response(Nfs3Status.NFS3ERR_IO, null, null, dirWcc);
+      }
+
+      // Set group if it's not specified in the request.
+      if (!setAttr3.getUpdateFields().contains(SetAttrField.GID)) {
+        setAttr3.getUpdateFields().add(SetAttrField.GID);
+        setAttr3.setGid(authSys.getGid());
+      }
+      setattrInternal(dfsClient, fileIdPath, setAttr3, false);
+      
+      postOpObjAttr = Nfs3Utils.getFileAttr(dfsClient, fileIdPath, iug);
+      objFileHandle = new FileHandle(postOpObjAttr.getFileId());
+      WccData dirWcc = Nfs3Utils.createWccData(preOpDirAttr, dfsClient,
+          dirFileIdPath, iug);
+      return new MKDIR3Response(Nfs3Status.NFS3_OK, new FileHandle(
+          postOpObjAttr.getFileId()), postOpObjAttr, dirWcc);
+    } catch (IOException e) {
+      LOG.warn("Exception ", e);
+      // Try to return correct WccData
+      if (postOpDirAttr == null) {
+        try {
+          postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
+        } catch (IOException e1) {
+          LOG.info("Can't get postOpDirAttr for " + dirFileIdPath);
+        }
+      }
+      WccData dirWcc = new WccData(preOpDirAttr, postOpDirAttr);
+      if (e instanceof AccessControlException) {
+        return new MKDIR3Response(Nfs3Status.NFS3ERR_PERM, objFileHandle,
+            postOpObjAttr, dirWcc);
+      } else {
+        return new MKDIR3Response(Nfs3Status.NFS3ERR_IO, objFileHandle,
+            postOpObjAttr, dirWcc);
+      }
+    }
+  }
+
+
+  public READDIR3Response mknod(XDR xdr, RpcAuthSys authSys) {
+    return new READDIR3Response(Nfs3Status.NFS3ERR_NOTSUPP);
+  }
+  
+  public REMOVE3Response remove(XDR xdr, RpcAuthSys authSys) {
+    REMOVE3Response response = new REMOVE3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    REMOVE3Request request = null;
+    try {
+      request = new REMOVE3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid REMOVE request");
+      return new REMOVE3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+    FileHandle dirHandle = request.getHandle();
+    String fileName = request.getName();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS REMOVE dir fileId: " + dirHandle.getFileId()
+          + " fileName: " + fileName);
+    }
+
+    String dirFileIdPath = Nfs3Utils.getFileIdPath(dirHandle);
+    WccAttr preOpDirAttr = null;
+    Nfs3FileAttributes postOpDirAttr = null;
+    try {
+      preOpDirAttr = Nfs3Utils.getWccAttr(dfsClient, dirFileIdPath);
+      if (preOpDirAttr == null) {
+        LOG.info("Can't get path for dir fileId:" + dirHandle.getFileId());
+        return new REMOVE3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+
+      String fileIdPath = dirFileIdPath + "/" + fileName;
+      HdfsFileStatus fstat = Nfs3Utils.getFileStatus(dfsClient,
+          fileIdPath);
+      if (fstat == null) {
+        WccData dirWcc = Nfs3Utils.createWccData(preOpDirAttr, dfsClient,
+            dirFileIdPath, iug);
+        return new REMOVE3Response(Nfs3Status.NFS3ERR_NOENT, dirWcc);
+      }
+      if (fstat.isDir()) {
+        WccData dirWcc = Nfs3Utils.createWccData(preOpDirAttr, dfsClient,
+            dirFileIdPath, iug);
+        return new REMOVE3Response(Nfs3Status.NFS3ERR_ISDIR, dirWcc);
+      }
+
+      if (dfsClient.delete(fileIdPath, false) == false) {
+        WccData dirWcc = Nfs3Utils.createWccData(preOpDirAttr, dfsClient,
+            dirFileIdPath, iug);
+        return new REMOVE3Response(Nfs3Status.NFS3ERR_ACCES, dirWcc);
+      }
+
+      WccData dirWcc = Nfs3Utils.createWccData(preOpDirAttr, dfsClient,
+          dirFileIdPath, iug);
+      return new REMOVE3Response(Nfs3Status.NFS3_OK, dirWcc);
+    } catch (IOException e) {
+      LOG.warn("Exception ", e);
+      // Try to return correct WccData
+      if (postOpDirAttr == null) {
+        try {
+          postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
+        } catch (IOException e1) {
+          LOG.info("Can't get postOpDirAttr for " + dirFileIdPath);
+        }
+      }
+      WccData dirWcc = new WccData(preOpDirAttr, postOpDirAttr);
+      if (e instanceof AccessControlException) {
+        return new REMOVE3Response(Nfs3Status.NFS3ERR_PERM, dirWcc);
+      } else {
+        return new REMOVE3Response(Nfs3Status.NFS3ERR_IO, dirWcc);
+      }
+    }
+  }
+
+  public RMDIR3Response rmdir(XDR xdr, RpcAuthSys authSys) {
+    RMDIR3Response response = new RMDIR3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    RMDIR3Request request = null;
+    try {
+      request = new RMDIR3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid RMDIR request");
+      return new RMDIR3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+    FileHandle dirHandle = request.getHandle();
+    String fileName = request.getName();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS RMDIR dir fileId: " + dirHandle.getFileId()
+          + " fileName: " + fileName);
+    }
+
+    String dirFileIdPath = Nfs3Utils.getFileIdPath(dirHandle);
+    WccAttr preOpDirAttr = null;
+    Nfs3FileAttributes postOpDirAttr = null;
+    try {
+      preOpDirAttr = Nfs3Utils.getWccAttr(dfsClient, dirFileIdPath);
+      if (preOpDirAttr == null) {
+        LOG.info("Can't get path for dir fileId:" + dirHandle.getFileId());
+        return new RMDIR3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+
+      String fileIdPath = dirFileIdPath + "/" + fileName;
+      HdfsFileStatus fstat = Nfs3Utils.getFileStatus(dfsClient,
+          fileIdPath);
+      if (fstat == null) {
+        WccData dirWcc = Nfs3Utils.createWccData(preOpDirAttr, dfsClient,
+            dirFileIdPath, iug);
+        return new RMDIR3Response(Nfs3Status.NFS3ERR_NOENT, dirWcc);
+      }
+      if (!fstat.isDir()) {
+        WccData dirWcc = Nfs3Utils.createWccData(preOpDirAttr, dfsClient,
+            dirFileIdPath, iug);
+        return new RMDIR3Response(Nfs3Status.NFS3ERR_NOTDIR, dirWcc);
+      }
+      
+      if (fstat.getChildrenNum() > 0) {
+        WccData dirWcc = Nfs3Utils.createWccData(preOpDirAttr, dfsClient,
+            dirFileIdPath, iug);
+        return new RMDIR3Response(Nfs3Status.NFS3ERR_NOTEMPTY, dirWcc);
+      }
+
+      if (dfsClient.delete(fileIdPath, false) == false) {
+        WccData dirWcc = Nfs3Utils.createWccData(preOpDirAttr, dfsClient,
+            dirFileIdPath, iug);
+        return new RMDIR3Response(Nfs3Status.NFS3ERR_ACCES, dirWcc);
+      }
+
+      postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
+      WccData wccData = new WccData(preOpDirAttr, postOpDirAttr);
+      return new RMDIR3Response(Nfs3Status.NFS3_OK, wccData);
+
+    } catch (IOException e) {
+      LOG.warn("Exception ", e);
+      // Try to return correct WccData
+      if (postOpDirAttr == null) {
+        try {
+          postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
+        } catch (IOException e1) {
+          LOG.info("Can't get postOpDirAttr for " + dirFileIdPath);
+        }
+      }
+      WccData dirWcc = new WccData(preOpDirAttr, postOpDirAttr);
+      if (e instanceof AccessControlException) {
+        return new RMDIR3Response(Nfs3Status.NFS3ERR_PERM, dirWcc);
+      } else {
+        return new RMDIR3Response(Nfs3Status.NFS3ERR_IO, dirWcc);
+      }
+    }
+  }
+
+  public RENAME3Response rename(XDR xdr, RpcAuthSys authSys) {
+    RENAME3Response response = new RENAME3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    RENAME3Request request = null;
+    try {
+      request = new RENAME3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid RENAME request");
+      return new RENAME3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+
+    FileHandle fromHandle = request.getFromDirHandle();
+    String fromName = request.getFromName();
+    FileHandle toHandle = request.getToDirHandle();
+    String toName = request.getToName();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS RENAME from: " + fromHandle.getFileId() + "/" + fromName
+          + " to: " + toHandle.getFileId() + "/" + toName);
+    }
+
+    String fromDirFileIdPath = Nfs3Utils.getFileIdPath(fromHandle);
+    String toDirFileIdPath = Nfs3Utils.getFileIdPath(toHandle);
+    WccAttr fromPreOpAttr = null;
+    WccAttr toPreOpAttr = null;
+    WccData fromDirWcc = null;
+    WccData toDirWcc = null;
+    try {
+      fromPreOpAttr = Nfs3Utils.getWccAttr(dfsClient, fromDirFileIdPath);
+      if (fromPreOpAttr == null) {
+        LOG.info("Can't get path for fromHandle fileId:"
+            + fromHandle.getFileId());
+        return new RENAME3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+
+      toPreOpAttr = Nfs3Utils.getWccAttr(dfsClient, toDirFileIdPath);
+      if (toPreOpAttr == null) {
+        LOG.info("Can't get path for toHandle fileId:" + toHandle.getFileId());
+        return new RENAME3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+
+      String src = fromDirFileIdPath + "/" + fromName;
+      String dst = toDirFileIdPath + "/" + toName;
+
+      dfsClient.rename(src, dst, Options.Rename.NONE);
+
+      // Assemble the reply
+      fromDirWcc = Nfs3Utils.createWccData(fromPreOpAttr, dfsClient,
+          fromDirFileIdPath, iug);
+      toDirWcc = Nfs3Utils.createWccData(toPreOpAttr, dfsClient,
+          toDirFileIdPath, iug);
+      return new RENAME3Response(Nfs3Status.NFS3_OK, fromDirWcc, toDirWcc);
+
+    } catch (IOException e) {
+      LOG.warn("Exception ", e);
+      // Try to return correct WccData      
+      try {
+        fromDirWcc = Nfs3Utils.createWccData(fromPreOpAttr, dfsClient,
+            fromDirFileIdPath, iug);
+        toDirWcc = Nfs3Utils.createWccData(toPreOpAttr, dfsClient,
+            toDirFileIdPath, iug);
+      } catch (IOException e1) {
+        LOG.info("Can't get postOpDirAttr for " + fromDirFileIdPath + " or"
+            + toDirFileIdPath);
+      }
+      if (e instanceof AccessControlException) {
+        return new RENAME3Response(Nfs3Status.NFS3ERR_PERM, fromDirWcc,
+            toDirWcc);
+      } else {
+        return new RENAME3Response(Nfs3Status.NFS3ERR_IO, fromDirWcc, toDirWcc);
+      }
+    }
+  }
+
+  public SYMLINK3Response symlink(XDR xdr, RpcAuthSys authSys) {
+    return new SYMLINK3Response(Nfs3Status.NFS3ERR_NOTSUPP);
+  }
+
+  public READDIR3Response link(XDR xdr, RpcAuthSys authSys) {
+    return new READDIR3Response(Nfs3Status.NFS3ERR_NOTSUPP);
+  }
+
+  public READDIR3Response readdir(XDR xdr, RpcAuthSys authSys) {
+    READDIR3Response response = new READDIR3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    READDIR3Request request = null;
+    try {
+      request = new READDIR3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid READDIR request");
+      return new READDIR3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+    FileHandle handle = request.getHandle();
+    long cookie = request.getCookie();
+    if (cookie < 0) {
+      LOG.error("Invalid READDIR request, with negitve cookie:" + cookie);
+      return new READDIR3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+    long count = request.getCount();
+    if (count <= 0) {
+      LOG.info("Nonpositive count in invalid READDIR request:" + count);
+      return new READDIR3Response(Nfs3Status.NFS3_OK);
+    }
+    
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS READDIR fileId: " + handle.getFileId() + " cookie: "
+          + cookie + " count: " + count);
+    }
+
+    HdfsFileStatus dirStatus;
+    DirectoryListing dlisting = null;
+    Nfs3FileAttributes postOpAttr = null;
+    long dotdotFileId = 0;
+    try {
+      String dirFileIdPath = Nfs3Utils.getFileIdPath(handle);
+      dirStatus = dfsClient.getFileInfo(dirFileIdPath);
+      if (dirStatus == null) {
+        LOG.info("Can't get path for fileId:" + handle.getFileId());
+        return new READDIR3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+      if (!dirStatus.isDir()) {
+        LOG.error("Can't readdir for regular file, fileId:"
+            + handle.getFileId());
+        return new READDIR3Response(Nfs3Status.NFS3ERR_NOTDIR);
+      }
+      long cookieVerf = request.getCookieVerf();
+      if ((cookieVerf != 0) && (cookieVerf != dirStatus.getModificationTime())) {
+        LOG.error("CookierVerf mismatch. request cookierVerf:" + cookieVerf
+            + " dir cookieVerf:" + dirStatus.getModificationTime());
+        return new READDIR3Response(Nfs3Status.NFS3ERR_BAD_COOKIE);
+      }
+
+      if (cookie == 0) {
+        // Get dotdot fileId
+        String dotdotFileIdPath = dirFileIdPath + "/..";
+        HdfsFileStatus dotdotStatus = dfsClient.getFileInfo(dotdotFileIdPath);
+
+        if (dotdotStatus == null) {
+          // This should not happen
+          throw new IOException("Can't get path for handle path:"
+              + dotdotFileIdPath);
+        }
+        dotdotFileId = dotdotStatus.getFileId();
+      }
+
+      // Get the list from the resume point
+      byte[] startAfter;
+      if(cookie == 0 ) {
+        startAfter = HdfsFileStatus.EMPTY_NAME;
+      } else {
+        String inodeIdPath = Nfs3Utils.getFileIdPath(cookie);
+        startAfter = inodeIdPath.getBytes();
+      }
+      dlisting = dfsClient.listPaths(dirFileIdPath, startAfter);
+
+      postOpAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
+      if (postOpAttr == null) {
+        LOG.error("Can't get path for fileId:" + handle.getFileId());
+        return new READDIR3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+    } catch (IOException e) {
+      LOG.warn("Exception ", e);
+      return new READDIR3Response(Nfs3Status.NFS3ERR_IO);
+    }
+
+    /**
+     * Set up the dirents in the response. fileId is used as the cookie with one
+     * exception. Linux client can either be stuck with "ls" command (on REHL)
+     * or report "Too many levels of symbolic links" (Ubuntu).
+     * 
+     * The problem is that, only two items returned, "." and ".." when the
+     * namespace is empty. Both of them are "/" with the same cookie(root
+     * fileId). Linux client doesn't think such a directory is a real directory.
+     * Even though NFS protocol specifies cookie is an opaque data, Linux client
+     * somehow doesn't like an empty dir returns same cookie for both "." and
+     * "..".
+     * 
+     * The workaround is to use 0 as the cookie for "." and always return "." as
+     * the first entry in readdir/readdirplus response.
+     */
+    HdfsFileStatus[] fstatus = dlisting.getPartialListing();    
+    int n = (int) Math.min(fstatus.length, count-2);
+    boolean eof = (n < fstatus.length) ? false : (dlisting
+        .getRemainingEntries() == 0);
+    
+    Entry3[] entries;
+    if (cookie == 0) {
+      entries = new Entry3[n + 2];
+      entries[0] = new READDIR3Response.Entry3(postOpAttr.getFileId(), ".", 0);
+      entries[1] = new READDIR3Response.Entry3(dotdotFileId, "..", dotdotFileId);
+
+      for (int i = 2; i < n + 2; i++) {
+        entries[i] = new READDIR3Response.Entry3(fstatus[i - 2].getFileId(),
+            fstatus[i - 2].getLocalName(), fstatus[i - 2].getFileId());
+      }
+    } else {
+      // Resume from last readdirplus. If the cookie is "..", the result
+      // list is up the directory content since HDFS uses name as resume point.    
+      entries = new Entry3[n];    
+      for (int i = 0; i < n; i++) {
+        entries[i] = new READDIR3Response.Entry3(fstatus[i].getFileId(),
+            fstatus[i].getLocalName(), fstatus[i].getFileId());
+      }
+    }
+    
+    DirList3 dirList = new READDIR3Response.DirList3(entries, eof);
+    return new READDIR3Response(Nfs3Status.NFS3_OK, postOpAttr,
+        dirStatus.getModificationTime(), dirList);
+  }
+
+  public READDIRPLUS3Response readdirplus(XDR xdr, RpcAuthSys authSys) {   
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_SERVERFAULT);
+    }
+    
+    READDIRPLUS3Request request = null;
+    try {
+      request = new READDIRPLUS3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid READDIRPLUS request");
+      return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+
+    FileHandle handle = request.getHandle();
+    long cookie = request.getCookie();
+    if (cookie < 0) {
+      LOG.error("Invalid READDIRPLUS request, with negitve cookie:" + cookie);
+      return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+    long dirCount = request.getDirCount();
+    if (dirCount <= 0) {
+      LOG.info("Nonpositive count in invalid READDIRPLUS request:" + dirCount);
+      return new READDIRPLUS3Response(Nfs3Status.NFS3_OK);
+    }
+    int maxCount = request.getMaxCount();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS READDIRPLUS fileId: " + handle.getFileId() + " cookie: "
+          + cookie + " dirCount: " + dirCount + " maxCount: " + maxCount);
+    }
+
+    HdfsFileStatus dirStatus;
+    DirectoryListing dlisting = null;
+    Nfs3FileAttributes postOpDirAttr = null;
+    long dotdotFileId = 0;
+    try {
+      String dirFileIdPath = Nfs3Utils.getFileIdPath(handle);
+      dirStatus = dfsClient.getFileInfo(dirFileIdPath);
+      if (dirStatus == null) {
+        LOG.info("Can't get path for fileId:" + handle.getFileId());
+        return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+      if (!dirStatus.isDir()) {
+        LOG.error("Can't readdirplus for regular file, fileId:"
+            + handle.getFileId());
+        return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_NOTDIR);
+      }
+      long cookieVerf = request.getCookieVerf();
+      if ((cookieVerf != 0) && (cookieVerf != dirStatus.getModificationTime())) {
+        LOG.error("CookierVerf mismatch. request cookierVerf:" + cookieVerf
+            + " dir cookieVerf:" + dirStatus.getModificationTime());
+        return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_BAD_COOKIE);
+      }
+
+      if (cookie == 0) {
+        // Get dotdot fileId
+        String dotdotFileIdPath = dirFileIdPath + "/..";
+        HdfsFileStatus dotdotStatus = dfsClient.getFileInfo(dotdotFileIdPath);
+
+        if (dotdotStatus == null) {
+          // This should not happen
+          throw new IOException("Can't get path for handle path:"
+              + dotdotFileIdPath);
+        }
+        dotdotFileId = dotdotStatus.getFileId();
+      }
+
+      // Get the list from the resume point
+      byte[] startAfter;
+      if (cookie == 0) {
+        startAfter = HdfsFileStatus.EMPTY_NAME;
+      } else {
+        String inodeIdPath = Nfs3Utils.getFileIdPath(cookie);
+        startAfter = inodeIdPath.getBytes();
+      }
+      dlisting = dfsClient.listPaths(dirFileIdPath, startAfter);
+
+      postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
+      if (postOpDirAttr == null) {
+        LOG.info("Can't get path for fileId:" + handle.getFileId());
+        return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+    } catch (IOException e) {
+      LOG.warn("Exception ", e);
+      return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_IO);
+    }
+    
+    // Set up the dirents in the response
+    HdfsFileStatus[] fstatus = dlisting.getPartialListing();
+    int n = (int) Math.min(fstatus.length, dirCount-2);
+    boolean eof = (n < fstatus.length) ? false : (dlisting
+        .getRemainingEntries() == 0);
+    
+    READDIRPLUS3Response.EntryPlus3[] entries;
+    if (cookie == 0) {
+      entries = new READDIRPLUS3Response.EntryPlus3[n+2];
+      
+      entries[0] = new READDIRPLUS3Response.EntryPlus3(
+          postOpDirAttr.getFileId(), ".", 0, postOpDirAttr, new FileHandle(
+              postOpDirAttr.getFileid()));
+      entries[1] = new READDIRPLUS3Response.EntryPlus3(dotdotFileId, "..",
+          dotdotFileId, postOpDirAttr, new FileHandle(dotdotFileId));
+
+      for (int i = 2; i < n + 2; i++) {
+        long fileId = fstatus[i - 2].getFileId();
+        FileHandle childHandle = new FileHandle(fileId);
+        Nfs3FileAttributes attr;
+        try {
+          attr = writeManager.getFileAttr(dfsClient, childHandle, iug);
+        } catch (IOException e) {
+          LOG.error("Can't get file attributes for fileId:" + fileId
+              + " error:" + e);
+          continue;
+        }
+        entries[i] = new READDIRPLUS3Response.EntryPlus3(fileId,
+            fstatus[i - 2].getLocalName(), fileId, attr, childHandle);
+      }
+    } else {
+      // Resume from last readdirplus. If the cookie is "..", the result
+      // list is up the directory content since HDFS uses name as resume point.
+      entries = new READDIRPLUS3Response.EntryPlus3[n]; 
+      for (int i = 0; i < n; i++) {
+        long fileId = fstatus[i].getFileId();
+        FileHandle childHandle = new FileHandle(fileId);
+        Nfs3FileAttributes attr;
+        try {
+          attr = writeManager.getFileAttr(dfsClient, childHandle, iug);
+        } catch (IOException e) {
+          LOG.error("Can't get file attributes for fileId:" + fileId
+              + " error:" + e);
+          continue;
+        }
+        entries[i] = new READDIRPLUS3Response.EntryPlus3(fileId,
+            fstatus[i].getLocalName(), fileId, attr, childHandle);
+      }
+    }
+
+    DirListPlus3 dirListPlus = new READDIRPLUS3Response.DirListPlus3(entries,
+        eof);
+    return new READDIRPLUS3Response(Nfs3Status.NFS3_OK, postOpDirAttr,
+        dirStatus.getModificationTime(), dirListPlus);
+  }
+  
+  public FSSTAT3Response fsstat(XDR xdr, RpcAuthSys authSys) {
+    FSSTAT3Response response = new FSSTAT3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    FSSTAT3Request request = null;
+    try {
+      request = new FSSTAT3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid FSSTAT request");
+      return new FSSTAT3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+
+    FileHandle handle = request.getHandle();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS FSSTAT fileId: " + handle.getFileId());
+    }
+
+    try {
+      // Use superUserClient to get file system status
+      FsStatus fsStatus = superUserClient.getDiskStatus();
+      long totalBytes = fsStatus.getCapacity();
+      long freeBytes = fsStatus.getRemaining();
+      
+      Nfs3FileAttributes attrs = writeManager.getFileAttr(dfsClient, handle,
+          iug);
+      if (attrs == null) {
+        LOG.info("Can't get path for fileId:" + handle.getFileId());
+        return new FSSTAT3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+      
+      long maxFsObjects = config.getLong("dfs.max.objects", 0);
+      if (maxFsObjects == 0) {
+        // A value of zero in HDFS indicates no limit to the number
+        // of objects that dfs supports. Using Integer.MAX_VALUE instead of
+        // Long.MAX_VALUE so 32bit client won't complain.
+        maxFsObjects = Integer.MAX_VALUE;
+      }
+      
+      return new FSSTAT3Response(Nfs3Status.NFS3_OK, attrs, totalBytes,
+          freeBytes, freeBytes, maxFsObjects, maxFsObjects, maxFsObjects, 0);
+    } catch (IOException e) {
+      LOG.warn("Exception ", e);
+      return new FSSTAT3Response(Nfs3Status.NFS3ERR_IO);
+    }
+  }
+
+  public FSINFO3Response fsinfo(XDR xdr, RpcAuthSys authSys) {
+    FSINFO3Response response = new FSINFO3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    FSINFO3Request request = null;
+    try {
+      request = new FSINFO3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid FSINFO request");
+      return new FSINFO3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+
+    FileHandle handle = request.getHandle();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS FSINFO fileId: " + handle.getFileId());
+    }
+
+    try {
+      int rtmax = MAX_READ_TRANSFER_SIZE;
+      int wtmax = MAX_WRITE_TRANSFER_SIZE;
+      int dtperf = MAX_READDIR_TRANSFER_SIZE;
+
+      Nfs3FileAttributes attrs = Nfs3Utils.getFileAttr(dfsClient,
+          Nfs3Utils.getFileIdPath(handle), iug);
+      if (attrs == null) {
+        LOG.info("Can't get path for fileId:" + handle.getFileId());
+        return new FSINFO3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+      
+      int fsProperty = Nfs3Constant.FSF3_CANSETTIME
+          | Nfs3Constant.FSF3_HOMOGENEOUS;
+
+      return new FSINFO3Response(Nfs3Status.NFS3_OK, attrs, rtmax, rtmax, 1,
+          wtmax, wtmax, 1, dtperf, Long.MAX_VALUE, new NfsTime(1), fsProperty);
+    } catch (IOException e) {
+      LOG.warn("Exception ", e);
+      return new FSINFO3Response(Nfs3Status.NFS3ERR_IO);
+    }
+  }
+
+  public PATHCONF3Response pathconf(XDR xdr, RpcAuthSys authSys) {
+    PATHCONF3Response response = new PATHCONF3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    PATHCONF3Request request = null;
+    try {
+      request = new PATHCONF3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid PATHCONF request");
+      return new PATHCONF3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+
+    FileHandle handle = request.getHandle();
+    Nfs3FileAttributes attrs;
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS PATHCONF fileId: " + handle.getFileId());
+    }
+
+    try {
+      attrs = Nfs3Utils.getFileAttr(dfsClient, Nfs3Utils.getFileIdPath(handle),
+          iug);
+      if (attrs == null) {
+        LOG.info("Can't get path for fileId:" + handle.getFileId());
+        return new PATHCONF3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+
+      return new PATHCONF3Response(Nfs3Status.NFS3_OK, attrs, 0,
+          HdfsConstants.MAX_PATH_LENGTH, true, false, false, true);
+    } catch (IOException e) {
+      LOG.warn("Exception ", e);
+      return new PATHCONF3Response(Nfs3Status.NFS3ERR_IO);
+    }
+  }
+
+  public COMMIT3Response commit(XDR xdr, RpcAuthSys authSys) {
+    COMMIT3Response response = new COMMIT3Response(Nfs3Status.NFS3_OK);
+    String uname = authSysCheck(authSys);
+    DFSClient dfsClient = clientCache.get(uname);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+    
+    COMMIT3Request request = null;
+    try {
+      request = new COMMIT3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid COMMIT request");
+      response.setStatus(Nfs3Status.NFS3ERR_INVAL);
+      return response;
+    }
+
+    FileHandle handle = request.getHandle();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS COMMIT fileId: " + handle.getFileId() + " offset="
+          + request.getOffset() + " count=" + request.getCount());
+    }
+
+    String fileIdPath = Nfs3Utils.getFileIdPath(handle);
+    WccAttr preOpAttr = null;
+    try {
+      preOpAttr = Nfs3Utils.getWccAttr(dfsClient, fileIdPath);
+      if (preOpAttr == null) {
+        LOG.info("Can't get path for fileId:" + handle.getFileId());
+        return new COMMIT3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+      long commitOffset = (request.getCount() == 0) ? 0
+          : (request.getOffset() + request.getCount());
+      
+      int status;
+      if (writeManager.handleCommit(handle, commitOffset)) {
+        status = Nfs3Status.NFS3_OK;
+      } else {
+        status = Nfs3Status.NFS3ERR_IO;
+      }
+      Nfs3FileAttributes postOpAttr = writeManager.getFileAttr(dfsClient,
+          handle, iug);
+      WccData fileWcc = new WccData(preOpAttr, postOpAttr);
+      return new COMMIT3Response(status, fileWcc,
+          Nfs3Constant.WRITE_COMMIT_VERF);
+
+    } catch (IOException e) {
+      LOG.warn("Exception ", e);
+      Nfs3FileAttributes postOpAttr = null;
+      try {
+        postOpAttr = writeManager.getFileAttr(dfsClient, handle, iug);
+      } catch (IOException e1) {
+        LOG.info("Can't get postOpAttr for fileId: " + handle.getFileId());
+      }
+      WccData fileWcc = new WccData(preOpAttr, postOpAttr);
+      return new COMMIT3Response(Nfs3Status.NFS3ERR_IO, fileWcc,
+          Nfs3Constant.WRITE_COMMIT_VERF);
+    }
+  }
+  
+  private final static String UNKNOWN_USER = "nobody";
+  private final static String UNKNOWN_GROUP = "nobody";
+
+  private String authSysCheck(RpcAuthSys authSys) {
+    return iug.getUserName(authSys.getUid(), UNKNOWN_USER);
+  }
+  
+  @Override
+  public XDR handleInternal(RpcCall rpcCall, final XDR xdr, XDR out,
+      InetAddress client, Channel channel) {
+    int procedure = rpcCall.getProcedure();
+    int xid = rpcCall.getXid();
+    RpcAuthSys authSys = null;
+    
+    // Ignore auth only for NFSPROC3_NULL, especially for Linux clients.
+    if (procedure != Nfs3Constant.NFSPROC3_NULL) {
+      if (rpcCall.getCredential().getFlavor() != AuthFlavor.AUTH_SYS) {
+        LOG.info("Wrong RPC AUTH flavor, "
+            + rpcCall.getCredential().getFlavor() + " is not AUTH_SYS.");
+        XDR reply = new XDR();
+        reply = RpcDeniedReply.voidReply(reply, xid,
+            RpcReply.ReplyState.MSG_ACCEPTED,
+            RpcDeniedReply.RejectState.AUTH_ERROR);
+        return reply;
+      }
+      authSys = RpcAuthSys.from(rpcCall.getCredential().getBody());
+    }
+    
+    NFS3Response response = null;
+    if (procedure == Nfs3Constant.NFSPROC3_NULL) {
+      response = nullProcedure();
+    } else if (procedure == Nfs3Constant.NFSPROC3_GETATTR) {
+      response = getattr(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_SETATTR) {
+      response = setattr(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_LOOKUP) {
+      response = lookup(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_ACCESS) {
+      response = access(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_READLINK) {
+      response = readlink(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_READ) {
+      response = read(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_WRITE) {
+      response = write(xdr, channel, xid, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_CREATE) {
+      response = create(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_MKDIR) {      
+      response = mkdir(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_SYMLINK) {
+      response = symlink(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_MKNOD) {
+      response = mknod(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_REMOVE) {
+      response = remove(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_RMDIR) {
+      response = rmdir(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_RENAME) {
+      response = rename(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_LINK) {
+      response = link(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_READDIR) {
+      response = readdir(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_READDIRPLUS) {
+      response = readdirplus(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_FSSTAT) {
+      response = fsstat(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_FSINFO) {
+      response = fsinfo(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_PATHCONF) {
+      response = pathconf(xdr, authSys);
+    } else if (procedure == Nfs3Constant.NFSPROC3_COMMIT) {
+      response = commit(xdr, authSys);
+    } else {
+      // Invalid procedure
+      RpcAcceptedReply.voidReply(out, xid,
+          RpcAcceptedReply.AcceptState.PROC_UNAVAIL);
+    }
+    if (response != null) {
+      out = response.send(out, xid);
+    }
+    return out;
+  }
+  
+  @Override
+  protected boolean isIdempotent(RpcCall call) {
+    return isIdempotent(call.getProcedure());
+  }
+  
+  public static boolean isIdempotent(int procedure) {
+    return !(procedure == Nfs3Constant.NFSPROC3_CREATE
+        || procedure == Nfs3Constant.NFSPROC3_REMOVE
+        || procedure == Nfs3Constant.NFSPROC3_MKDIR
+        || procedure == Nfs3Constant.NFSPROC3_MKNOD
+        || procedure == Nfs3Constant.NFSPROC3_LINK
+        || procedure == Nfs3Constant.NFSPROC3_RMDIR
+        || procedure == Nfs3Constant.NFSPROC3_SYMLINK
+        || procedure == Nfs3Constant.NFSPROC3_RENAME);
+  }
+}

+ 162 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteCtx.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.hdfs.nfs.nfs3;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant.WriteStableHow;
+import org.jboss.netty.channel.Channel;
+
+/**
+ * WriteCtx saves the context of one write request, such as request, channel,
+ * xid and reply status.
+ */
+class WriteCtx {
+  public static final Log LOG = LogFactory.getLog(WriteCtx.class);
+
+  private final FileHandle handle;
+  private final long offset;
+  private final int count;
+  private final WriteStableHow stableHow;
+  private byte[] data;
+  
+  private final Channel channel;
+  private final int xid;
+  private boolean replied;
+  
+  /**
+   * In memory write data has 3 states. ALLOW_DUMP: not sequential write, still
+   * wait for prerequisit writes. NO_DUMP: sequential write, no need to dump
+   * since it will be written to HDFS soon. DUMPED: already dumped to a file.
+   */
+  public final static int ALLOW_DUMP = 0;
+  public final static int NO_DUMP = 1;
+  public final static int DUMPED = 2;
+  private int dataState;
+
+  public int getDataState() {
+    return dataState;
+  }
+
+  public void setDataState(int dataState) {
+    this.dataState = dataState;
+  }
+
+  private RandomAccessFile raf;
+  private long dumpFileOffset;
+  
+  // Return the dumped data size
+  public long dumpData(FileOutputStream dumpOut, RandomAccessFile raf)
+      throws IOException {
+    if (dataState != ALLOW_DUMP) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("No need to dump with status(replied,dataState):" + "("
+            + replied + "," + dataState + ")");
+      }
+      return 0;
+    }
+    this.raf = raf;
+    dumpFileOffset = dumpOut.getChannel().position();
+    dumpOut.write(data, 0, count);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("After dump, new dumpFileOffset:" + dumpFileOffset);
+    }
+    data = null;
+    dataState = DUMPED;
+    return count;
+  }
+
+  public FileHandle getHandle() {
+    return handle;
+  }
+  
+  public long getOffset() {
+    return offset;
+  }
+
+  public int getCount() {
+    return count;
+  }
+
+  public WriteStableHow getStableHow() {
+    return stableHow;
+  }
+
+  public byte[] getData() throws IOException {
+    if (dataState != DUMPED) {
+      if (data == null) {
+        throw new IOException("Data is not dumpted but has null:" + this);
+      }
+    } else {
+      // read back
+      if (data != null) {
+        throw new IOException("Data is dumpted but not null");
+      }
+      data = new byte[count];
+      raf.seek(dumpFileOffset);
+      int size = raf.read(data, 0, count);
+      if (size != count) {
+        throw new IOException("Data count is " + count + ", but read back "
+            + size + "bytes");
+      }
+    }
+    return data;
+  }
+
+  Channel getChannel() {
+    return channel;
+  }
+
+  int getXid() {
+    return xid;
+  }
+
+  boolean getReplied() {
+    return replied;
+  }
+  
+  void setReplied(boolean replied) {
+    this.replied = replied;
+  }
+  
+  WriteCtx(FileHandle handle, long offset, int count, WriteStableHow stableHow,
+      byte[] data, Channel channel, int xid, boolean replied, int dataState) {
+    this.handle = handle;
+    this.offset = offset;
+    this.count = count;
+    this.stableHow = stableHow;
+    this.data = data;
+    this.channel = channel;
+    this.xid = xid;
+    this.replied = replied;
+    this.dataState = dataState;
+    raf = null;
+  }
+  
+  @Override
+  public String toString() {
+    return "Id:" + handle.getFileId() + " offset:" + offset + " count:" + count
+        + " stableHow:" + stableHow + " replied:" + replied + " dataState:"
+        + dataState + " xid:" + xid;
+  }
+}

+ 284 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java

@@ -0,0 +1,284 @@
+/**
+ * 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.hdfs.nfs.nfs3;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.nfs.NfsFileType;
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.nfs.nfs3.IdUserGroup;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant.WriteStableHow;
+import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.nfs.nfs3.request.WRITE3Request;
+import org.apache.hadoop.nfs.nfs3.response.WRITE3Response;
+import org.apache.hadoop.nfs.nfs3.response.WccData;
+import org.apache.hadoop.oncrpc.XDR;
+import org.apache.hadoop.util.Daemon;
+import org.jboss.netty.channel.Channel;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Manage the writes and responds asynchronously.
+ */
+public class WriteManager {
+  public static final Log LOG = LogFactory.getLog(WriteManager.class);
+
+  private final IdUserGroup iug;
+  private final ConcurrentMap<FileHandle, OpenFileCtx> openFileMap = Maps
+      .newConcurrentMap();
+
+  private AsyncDataService asyncDataService;
+  private boolean asyncDataServiceStarted = false;
+
+  private final StreamMonitor streamMonitor;
+  
+  /**
+   * The time limit to wait for accumulate reordered sequential writes to the
+   * same file before the write is considered done.
+   */
+  private long streamTimeout;
+  
+  public static final long DEFAULT_STREAM_TIMEOUT = 10 * 1000; // 10 second
+  public static final long MINIMIUM_STREAM_TIMEOUT = 1 * 1000; // 1 second
+  
+  void addOpenFileStream(FileHandle h, OpenFileCtx ctx) {
+    openFileMap.put(h, ctx);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("After add the new stream " + h.getFileId()
+          + ", the stream number:" + openFileMap.size());
+    }
+  }
+
+  WriteManager(IdUserGroup iug, final Configuration config) {
+    this.iug = iug;
+    
+    streamTimeout = config.getLong("dfs.nfs3.stream.timeout",
+        DEFAULT_STREAM_TIMEOUT);
+    LOG.info("Stream timeout is " + streamTimeout + "ms.");
+    if (streamTimeout < MINIMIUM_STREAM_TIMEOUT) {
+      LOG.info("Reset stream timeout to minimum value "
+          + MINIMIUM_STREAM_TIMEOUT + "ms.");
+      streamTimeout = MINIMIUM_STREAM_TIMEOUT;
+    }
+    
+    this.streamMonitor = new StreamMonitor();
+  }
+
+  private void startAsyncDataSerivce() {
+    streamMonitor.start();
+    this.asyncDataService = new AsyncDataService();
+    asyncDataServiceStarted = true;
+  }
+
+  private void shutdownAsyncDataService() {
+    asyncDataService.shutdown();
+    asyncDataServiceStarted = false;
+    streamMonitor.interrupt();
+  }
+
+  void handleWrite(DFSClient dfsClient, WRITE3Request request, Channel channel,
+      int xid, Nfs3FileAttributes preOpAttr) throws IOException {
+    // First write request starts the async data service
+    if (!asyncDataServiceStarted) {
+      startAsyncDataSerivce();
+    }
+
+    long offset = request.getOffset();
+    int count = request.getCount();
+    WriteStableHow stableHow = request.getStableHow();
+    byte[] data = request.getData().array();
+    if (data.length < count) {
+      WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_INVAL);
+      Nfs3Utils.writeChannel(channel, response.send(new XDR(), xid));
+      return;
+    }
+
+    FileHandle handle = request.getHandle();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("handleWrite fileId: " + handle.getFileId() + " offset: "
+          + offset + " length:" + count + " stableHow:" + stableHow.getValue());
+    }
+
+    // Check if there is a stream to write
+    FileHandle fileHandle = request.getHandle();
+    OpenFileCtx openFileCtx = openFileMap.get(fileHandle);
+    if (openFileCtx == null) {
+      LOG.info("No opened stream for fileId:" + fileHandle.getFileId());
+      WccData fileWcc = new WccData(Nfs3Utils.getWccAttr(preOpAttr), preOpAttr);
+      WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_IO,
+          fileWcc, count, request.getStableHow(),
+          Nfs3Constant.WRITE_COMMIT_VERF);
+      Nfs3Utils.writeChannel(channel, response.send(new XDR(), xid));
+      return;
+    }
+
+    // Add write into the async job queue
+    openFileCtx.receivedNewWrite(dfsClient, request, channel, xid,
+        asyncDataService, iug);
+    // Block stable write
+    if (request.getStableHow() != WriteStableHow.UNSTABLE) {
+      if (handleCommit(fileHandle, offset + count)) {
+        Nfs3FileAttributes postOpAttr = getFileAttr(dfsClient, handle, iug);
+        WccData fileWcc = new WccData(Nfs3Utils.getWccAttr(preOpAttr),
+            postOpAttr);
+        WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK,
+            fileWcc, count, request.getStableHow(),
+            Nfs3Constant.WRITE_COMMIT_VERF);
+        Nfs3Utils.writeChannel(channel, response.send(new XDR(), xid));
+      } else {
+        WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_IO);
+        Nfs3Utils.writeChannel(channel, response.send(new XDR(), xid));
+      }
+    }
+
+    return;
+  }
+
+  boolean handleCommit(FileHandle fileHandle, long commitOffset) {
+    OpenFileCtx openFileCtx = openFileMap.get(fileHandle);
+    if (openFileCtx == null) {
+      LOG.info("No opened stream for fileId:" + fileHandle.getFileId()
+          + " commitOffset=" + commitOffset);
+      return true;
+    }
+    long timeout = 30 * 1000; // 30 seconds
+    long startCommit = System.currentTimeMillis();
+    while (true) {
+      int ret = openFileCtx.checkCommit(commitOffset);
+      if (ret == OpenFileCtx.COMMIT_FINISHED) {
+        // Committed
+        return true;
+      } else if (ret == OpenFileCtx.COMMIT_INACTIVE_CTX) {
+        LOG.info("Inactive stream, fileId=" + fileHandle.getFileId()
+            + " commitOffset=" + commitOffset);
+        return true;
+      }
+      assert (ret == OpenFileCtx.COMMIT_WAIT || ret == OpenFileCtx.COMMIT_ERROR);
+      if (ret == OpenFileCtx.COMMIT_ERROR) {
+        return false;
+      }
+      
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Not committed yet, wait., fileId=" + fileHandle.getFileId()
+            + " commitOffset=" + commitOffset);
+      }
+      if (System.currentTimeMillis() - startCommit > timeout) {
+        // Commit took too long, return error
+        return false;
+      }
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException e) {
+        LOG.info("Commit is interrupted, fileId=" + fileHandle.getFileId()
+            + " commitOffset=" + commitOffset);
+        return false;
+      }
+    }// while
+  }
+
+  /**
+   * If the file is in cache, update the size based on the cached data size
+   */
+  Nfs3FileAttributes getFileAttr(DFSClient client, FileHandle fileHandle,
+      IdUserGroup iug) throws IOException {
+    String fileIdPath = Nfs3Utils.getFileIdPath(fileHandle);
+    Nfs3FileAttributes attr = Nfs3Utils.getFileAttr(client, fileIdPath, iug);
+    if (attr != null) {
+      OpenFileCtx openFileCtx = openFileMap.get(fileHandle);
+      if (openFileCtx != null) {
+        attr.setSize(openFileCtx.getNextOffset());
+        attr.setUsed(openFileCtx.getNextOffset());
+      }
+    }
+    return attr;
+  }
+
+  Nfs3FileAttributes getFileAttr(DFSClient client, FileHandle dirHandle,
+      String fileName) throws IOException {
+    String fileIdPath = Nfs3Utils.getFileIdPath(dirHandle) + "/" + fileName;
+    Nfs3FileAttributes attr = Nfs3Utils.getFileAttr(client, fileIdPath, iug);
+
+    if ((attr != null) && (attr.getType() == NfsFileType.NFSREG.toValue())) {
+      OpenFileCtx openFileCtx = openFileMap
+          .get(new FileHandle(attr.getFileId()));
+
+      if (openFileCtx != null) {
+        attr.setSize(openFileCtx.getNextOffset());
+        attr.setUsed(openFileCtx.getNextOffset());
+      }
+    }
+    return attr;
+  }
+  
+  /**
+   * StreamMonitor wakes up periodically to find and closes idle streams.
+   */
+  class StreamMonitor extends Daemon {
+    private int rotation = 5 * 1000; // 5 seconds
+    private long lastWakeupTime = 0;
+
+    @Override
+    public void run() {
+      while (true) {
+        Iterator<Entry<FileHandle, OpenFileCtx>> it = openFileMap.entrySet()
+            .iterator();
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("openFileMap size:" + openFileMap.size());
+        }
+        while (it.hasNext()) {
+          Entry<FileHandle, OpenFileCtx> pairs = it.next();
+          OpenFileCtx ctx = pairs.getValue();
+          if (ctx.streamCleanup((pairs.getKey()).getFileId(), streamTimeout)) {
+            it.remove();
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("After remove stream " + pairs.getKey().getFileId()
+                  + ", the stream number:" + openFileMap.size());
+            }
+          }
+        }
+
+        // Check if it can sleep
+        try {
+          long workedTime = System.currentTimeMillis() - lastWakeupTime;
+          if (workedTime < rotation) {
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("StreamMonitor can still have a sleep:"
+                  + ((rotation - workedTime) / 1000));
+            }
+            Thread.sleep(rotation - workedTime);
+          }
+          lastWakeupTime = System.currentTimeMillis();
+
+        } catch (InterruptedException e) {
+          LOG.info("StreamMonitor got interrupted");
+          return;
+        }
+      }
+    }
+  }
+}

+ 63 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestMountd.java

@@ -0,0 +1,63 @@
+/**
+ * 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.hdfs.nfs;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.nfs.mount.RpcProgramMountd;
+import org.apache.hadoop.hdfs.nfs.nfs3.Nfs3;
+import org.apache.hadoop.hdfs.nfs.nfs3.RpcProgramNfs3;
+import org.apache.hadoop.oncrpc.XDR;
+import org.junit.Test;
+
+public class TestMountd {
+
+  public static final Log LOG = LogFactory.getLog(TestMountd.class);
+
+  @Test
+  public void testStart() throws IOException {
+    // Start minicluster
+    Configuration config = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(config).numDataNodes(1)
+        .manageNameDfsDirs(false).build();
+    cluster.waitActive();
+    
+    // Start nfs
+    List<String> exports = new ArrayList<String>();
+    exports.add("/");
+    Nfs3 nfs3 = new Nfs3(exports, config);
+    nfs3.start(false);
+
+    RpcProgramMountd mountd = (RpcProgramMountd) nfs3.getMountBase()
+        .getRpcProgram();
+    mountd.nullOp(new XDR(), 1234, InetAddress.getByName("localhost"));
+    
+    RpcProgramNfs3 nfsd = (RpcProgramNfs3) nfs3.getRpcProgram();
+    nfsd.nullProcedure();
+    
+    cluster.shutdown();
+  }
+}

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

@@ -0,0 +1,185 @@
+/**
+ * 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.hdfs.nfs;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.nfs.nfs3.Nfs3Utils;
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant.WriteStableHow;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.nfs.nfs3.request.CREATE3Request;
+import org.apache.hadoop.nfs.nfs3.request.SetAttr3;
+import org.apache.hadoop.nfs.nfs3.request.WRITE3Request;
+import org.apache.hadoop.oncrpc.RegistrationClient;
+import org.apache.hadoop.oncrpc.RpcCall;
+import org.apache.hadoop.oncrpc.RpcFrameDecoder;
+import org.apache.hadoop.oncrpc.RpcReply;
+import org.apache.hadoop.oncrpc.SimpleTcpClient;
+import org.apache.hadoop.oncrpc.SimpleTcpClientHandler;
+import org.apache.hadoop.oncrpc.XDR;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.Channels;
+import org.jboss.netty.channel.MessageEvent;
+
+public class TestOutOfOrderWrite {
+  public final static Log LOG = LogFactory.getLog(TestOutOfOrderWrite.class);
+
+  static FileHandle handle = null;
+  static Channel channel;
+
+  static byte[] data1 = new byte[1000];
+  static byte[] data2 = new byte[1000];
+  static byte[] data3 = new byte[1000];
+
+  static XDR create() {
+    XDR request = new XDR();
+    RpcCall.write(request, 0x8000004c, Nfs3Constant.PROGRAM,
+        Nfs3Constant.VERSION, Nfs3Constant.NFSPROC3_CREATE);
+
+    // credentials
+    request.writeInt(0); // auth null
+    request.writeInt(0); // length zero
+    // verifier
+    request.writeInt(0); // auth null
+    request.writeInt(0); // length zero
+
+    SetAttr3 objAttr = new SetAttr3();
+    CREATE3Request createReq = new CREATE3Request(new FileHandle("/"),
+        "out-of-order-write" + System.currentTimeMillis(), 0, objAttr, 0);
+    createReq.serialize(request);
+    return request;
+  }
+
+  static XDR write(FileHandle handle, int xid, long offset, int count,
+      byte[] data) {
+    XDR request = new XDR();
+    RpcCall.write(request, xid, Nfs3Constant.PROGRAM, Nfs3Constant.VERSION,
+        Nfs3Constant.NFSPROC3_WRITE);
+
+    // credentials
+    request.writeInt(0); // auth null
+    request.writeInt(0); // length zero
+    // verifier
+    request.writeInt(0); // auth null
+    request.writeInt(0); // length zero
+    WRITE3Request write1 = new WRITE3Request(handle, offset, count,
+        WriteStableHow.UNSTABLE, ByteBuffer.wrap(data));
+    write1.serialize(request);
+    return request;
+  }
+
+  static void testRequest(XDR request) {
+    RegistrationClient registrationClient = new RegistrationClient("localhost",
+        Nfs3Constant.SUN_RPCBIND, request);
+    registrationClient.run();
+  }
+
+  static class WriteHandler extends SimpleTcpClientHandler {
+
+    public WriteHandler(XDR request) {
+      super(request);
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
+      // Get handle from create response
+      ChannelBuffer buf = (ChannelBuffer) e.getMessage();
+      XDR rsp = new XDR(buf.array());
+      if (rsp.getBytes().length == 0) {
+        LOG.info("rsp length is zero, why?");
+        return;
+      }
+      LOG.info("rsp length=" + rsp.getBytes().length);
+
+      RpcReply reply = RpcReply.read(rsp);
+      int xid = reply.getXid();
+      // Only process the create response
+      if (xid != 0x8000004c) {
+        return;
+      }
+      int status = rsp.readInt();
+      if (status != Nfs3Status.NFS3_OK) {
+        LOG.error("Create failed, status =" + status);
+        return;
+      }
+      LOG.info("Create succeeded");
+      rsp.readBoolean(); // value follow
+      handle = new FileHandle();
+      handle.deserialize(rsp);
+      channel = e.getChannel();
+    }
+  }
+
+  static class WriteClient extends SimpleTcpClient {
+
+    public WriteClient(String host, int port, XDR request, Boolean oneShot) {
+      super(host, port, request, oneShot);
+    }
+
+    @Override
+    protected ChannelPipelineFactory setPipelineFactory() {
+      this.pipelineFactory = new ChannelPipelineFactory() {
+        public ChannelPipeline getPipeline() {
+          return Channels.pipeline(new RpcFrameDecoder(), new WriteHandler(
+              request));
+        }
+      };
+      return this.pipelineFactory;
+    }
+
+  }
+
+  public static void main(String[] args) throws InterruptedException {
+
+    Arrays.fill(data1, (byte) 7);
+    Arrays.fill(data2, (byte) 8);
+    Arrays.fill(data3, (byte) 9);
+
+    // NFS3 Create request
+    WriteClient client = new WriteClient("localhost", Nfs3Constant.PORT,
+        create(), false);
+    client.run();
+
+    while (handle == null) {
+      Thread.sleep(1000);
+      System.out.println("handle is still null...");
+    }
+    LOG.info("Send write1 request");
+
+    XDR writeReq;
+
+    writeReq = write(handle, 0x8000005c, 2000, 1000, data3);
+    Nfs3Utils.writeChannel(channel, writeReq);
+    writeReq = write(handle, 0x8000005d, 1000, 1000, data2);
+    Nfs3Utils.writeChannel(channel, writeReq);
+    writeReq = write(handle, 0x8000005e, 0, 1000, data1);
+    Nfs3Utils.writeChannel(channel, writeReq);
+
+    // TODO: convert to Junit test, and validate result automatically
+  }
+}

+ 142 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestPortmapRegister.java

@@ -0,0 +1,142 @@
+/**
+ * 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.hdfs.nfs;
+
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.nfs.mount.RpcProgramMountd;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+import org.apache.hadoop.oncrpc.RegistrationClient;
+import org.apache.hadoop.oncrpc.RpcCall;
+import org.apache.hadoop.oncrpc.XDR;
+import org.apache.hadoop.portmap.PortmapMapping;
+import org.apache.hadoop.portmap.PortmapRequest;
+
+public class TestPortmapRegister {
+  
+  public static final Log LOG = LogFactory.getLog(TestPortmapRegister.class);
+  
+  static void testRequest(XDR request, XDR request2) {
+    RegistrationClient registrationClient = new RegistrationClient(
+        "localhost", Nfs3Constant.SUN_RPCBIND, request);
+    registrationClient.run();
+  }
+ 
+  public static void main(String[] args) throws InterruptedException {
+    PortmapMapping mapEntry = new PortmapMapping(RpcProgramMountd.PROGRAM,
+        RpcProgramMountd.VERSION_1, PortmapMapping.TRANSPORT_UDP,
+        RpcProgramMountd.PORT);
+    XDR mappingRequest = PortmapRequest.create(mapEntry);
+    RegistrationClient registrationClient = new RegistrationClient(
+        "localhost", Nfs3Constant.SUN_RPCBIND, mappingRequest);
+    registrationClient.run();
+        
+    Thread t1 = new Runtest1();
+    //Thread t2 = testa.new Runtest2();
+    t1.start();
+    //t2.start();
+    t1.join();
+    //t2.join();
+    //testDump();
+  }
+  
+  static class Runtest1 extends Thread {
+    @Override
+    public void run() {
+      //testGetportMount();
+      PortmapMapping mapEntry = new PortmapMapping(RpcProgramMountd.PROGRAM,
+          RpcProgramMountd.VERSION_1, PortmapMapping.TRANSPORT_UDP,
+          RpcProgramMountd.PORT);
+      XDR req = PortmapRequest.create(mapEntry);
+      testRequest(req, req);
+    }
+  }
+  
+  static class Runtest2 extends Thread {
+    @Override
+    public void run() {
+      testDump();
+    }
+  }
+  
+  static void createPortmapXDRheader(XDR xdr_out, int procedure) {
+    // TODO: Move this to RpcRequest
+    RpcCall.write(xdr_out, 0, 100000, 2, procedure);
+    xdr_out.writeInt(0); //no auth
+    xdr_out.writeInt(0);
+    xdr_out.writeInt(0);
+    xdr_out.writeInt(0);
+    
+    /*
+    xdr_out.putInt(1); //unix auth
+    xdr_out.putVariableOpaque(new byte[20]);
+    xdr_out.putInt(0);
+    xdr_out.putInt(0);
+*/
+  }
+ 
+  static void testGetportMount() {
+    XDR xdr_out = new XDR();
+
+    createPortmapXDRheader(xdr_out, 3);
+
+    xdr_out.writeInt(100005);
+    xdr_out.writeInt(1);
+    xdr_out.writeInt(6);
+    xdr_out.writeInt(0);
+
+    XDR request2 = new XDR();
+
+    createPortmapXDRheader(xdr_out, 3);
+    request2.writeInt(100005);
+    request2.writeInt(1);
+    request2.writeInt(6);
+    request2.writeInt(0);
+
+    testRequest(xdr_out, request2);
+  }
+  
+  static void testGetport() {
+    XDR xdr_out = new XDR();
+
+    createPortmapXDRheader(xdr_out, 3);
+
+    xdr_out.writeInt(100003);
+    xdr_out.writeInt(3);
+    xdr_out.writeInt(6);
+    xdr_out.writeInt(0);
+
+    XDR request2 = new XDR();
+
+    createPortmapXDRheader(xdr_out, 3);
+    request2.writeInt(100003);
+    request2.writeInt(3);
+    request2.writeInt(6);
+    request2.writeInt(0);
+
+    testRequest(xdr_out, request2);
+  }
+  
+  static void testDump() {
+    XDR xdr_out = new XDR();
+    createPortmapXDRheader(xdr_out, 4);
+    testRequest(xdr_out, xdr_out);
+  }
+}

+ 132 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestUdpServer.java

@@ -0,0 +1,132 @@
+/**
+ * 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.hdfs.nfs;
+
+import java.io.IOException;
+import java.net.DatagramPacket;
+import java.net.DatagramSocket;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+import org.apache.hadoop.oncrpc.RpcCall;
+import org.apache.hadoop.oncrpc.XDR;
+
+// TODO: convert this to Junit
+public class TestUdpServer {
+  static void testRequest(XDR request, XDR request2) {
+    try {
+      DatagramSocket clientSocket = new DatagramSocket();
+      InetAddress IPAddress = InetAddress.getByName("localhost");
+      byte[] sendData = request.getBytes();
+      byte[] receiveData = new byte[65535];
+
+      DatagramPacket sendPacket = new DatagramPacket(sendData, sendData.length,
+          IPAddress, Nfs3Constant.SUN_RPCBIND);
+      clientSocket.send(sendPacket);
+      DatagramPacket receivePacket = new DatagramPacket(receiveData,
+          receiveData.length);
+      clientSocket.receive(receivePacket);
+      clientSocket.close();
+
+    } catch (UnknownHostException e) {
+      System.err.println("Don't know about host: localhost.");
+      System.exit(1);
+    } catch (IOException e) {
+      System.err.println("Couldn't get I/O for "
+          + "the connection to: localhost.");
+      System.exit(1);
+    }
+  }
+ 
+  public static void main(String[] args) throws InterruptedException {
+    Thread t1 = new Runtest1();
+    // TODO: cleanup
+    //Thread t2 = new Runtest2();
+    t1.start();
+    //t2.start();
+    t1.join();
+    //t2.join();
+    //testDump();
+  }
+  
+  static class Runtest1 extends Thread {
+    @Override
+    public void run() {
+      testGetportMount();
+    }
+  }
+  
+  static class Runtest2 extends Thread {
+    @Override
+    public void run() {
+      testDump();
+    }
+  }
+  
+  static void createPortmapXDRheader(XDR xdr_out, int procedure) {
+    // Make this a method
+    RpcCall.write(xdr_out, 0, 100000, 2, procedure);
+  }
+ 
+  static void testGetportMount() {
+    XDR xdr_out = new XDR();
+    createPortmapXDRheader(xdr_out, 3);
+    xdr_out.writeInt(100005);
+    xdr_out.writeInt(1);
+    xdr_out.writeInt(6);
+    xdr_out.writeInt(0);
+
+    XDR request2 = new XDR();
+    createPortmapXDRheader(xdr_out, 3);
+    request2.writeInt(100005);
+    request2.writeInt(1);
+    request2.writeInt(6);
+    request2.writeInt(0);
+
+    testRequest(xdr_out, request2);
+  }
+  
+  static void testGetport() {
+    XDR xdr_out = new XDR();
+
+    createPortmapXDRheader(xdr_out, 3);
+
+    xdr_out.writeInt(100003);
+    xdr_out.writeInt(3);
+    xdr_out.writeInt(6);
+    xdr_out.writeInt(0);
+
+    XDR request2 = new XDR();
+
+    createPortmapXDRheader(xdr_out, 3);
+    request2.writeInt(100003);
+    request2.writeInt(3);
+    request2.writeInt(6);
+    request2.writeInt(0);
+
+    testRequest(xdr_out, request2);
+  }
+  
+  static void testDump() {
+    XDR xdr_out = new XDR();
+    createPortmapXDRheader(xdr_out, 4);
+    testRequest(xdr_out, xdr_out);
+  }
+}

+ 57 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestDFSClientCache.java

@@ -0,0 +1,57 @@
+/**
+ * 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.hdfs.nfs.nfs3;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestDFSClientCache {
+  @Test
+  public void testLruTable() throws IOException {
+    DFSClientCache cache = new DFSClientCache(new Configuration(), 3);
+    DFSClient client = Mockito.mock(DFSClient.class);
+    cache.put("a", client);
+    assertTrue(cache.containsKey("a"));
+
+    cache.put("b", client);
+    cache.put("c", client);
+    cache.put("d", client);
+    assertTrue(cache.usedSize() == 3);
+    assertFalse(cache.containsKey("a"));
+
+    // Cache should have d,c,b in LRU order
+    assertTrue(cache.containsKey("b"));
+    // Do a lookup to make b the most recently used
+    assertTrue(cache.get("b") != null);
+
+    cache.put("e", client);
+    assertTrue(cache.usedSize() == 3);
+    // c should be replaced with e, and cache has e,b,d
+    assertFalse(cache.containsKey("c"));
+    assertTrue(cache.containsKey("e"));
+    assertTrue(cache.containsKey("b"));
+    assertTrue(cache.containsKey("d"));
+  }
+}

+ 58 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestOffsetRange.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.hdfs.nfs.nfs3;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+public class TestOffsetRange {
+  @Test(expected = IllegalArgumentException.class)
+  public void testConstructor1() throws IOException {
+    new OffsetRange(0, 0);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testConstructor2() throws IOException {
+    new OffsetRange(-1, 0);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testConstructor3() throws IOException {
+    new OffsetRange(-3, -1);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testConstructor4() throws IOException {
+    new OffsetRange(-3, 100);
+  }
+
+  @Test
+  public void testCompare() throws IOException {
+    OffsetRange r1 = new OffsetRange(0, 1);
+    OffsetRange r2 = new OffsetRange(1, 3);
+    OffsetRange r3 = new OffsetRange(1, 3);
+    OffsetRange r4 = new OffsetRange(3, 4);
+
+    assertTrue(r2.compareTo(r3) == 0);
+    assertTrue(r2.compareTo(r1) == 1);
+    assertTrue(r2.compareTo(r4) == -1);
+  }
+}

+ 66 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.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.hdfs.nfs.nfs3;
+
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+/**
+ * Tests for {@link RpcProgramNfs3}
+ */
+public class TestRpcProgramNfs3 {
+  @Test(timeout=1000)
+  public void testIdempotent() {
+    int[][] procedures = {
+        { Nfs3Constant.NFSPROC3_NULL, 1 },
+        { Nfs3Constant.NFSPROC3_GETATTR, 1 },
+        { Nfs3Constant.NFSPROC3_SETATTR, 1 },
+        { Nfs3Constant.NFSPROC3_LOOKUP, 1 },
+        { Nfs3Constant.NFSPROC3_ACCESS, 1 },
+        { Nfs3Constant.NFSPROC3_READLINK, 1 },
+        { Nfs3Constant.NFSPROC3_READ, 1 },
+        { Nfs3Constant.NFSPROC3_WRITE, 1 },
+        { Nfs3Constant.NFSPROC3_CREATE, 0 },
+        { Nfs3Constant.NFSPROC3_MKDIR, 0 },
+        { Nfs3Constant.NFSPROC3_SYMLINK, 0 },
+        { Nfs3Constant.NFSPROC3_MKNOD, 0 },
+        { Nfs3Constant.NFSPROC3_REMOVE, 0 },
+        { Nfs3Constant.NFSPROC3_RMDIR, 0 },
+        { Nfs3Constant.NFSPROC3_RENAME, 0 },
+        { Nfs3Constant.NFSPROC3_LINK, 0 },
+        { Nfs3Constant.NFSPROC3_READDIR, 1 },
+        { Nfs3Constant.NFSPROC3_READDIRPLUS, 1 },
+        { Nfs3Constant.NFSPROC3_FSSTAT, 1 },
+        { Nfs3Constant.NFSPROC3_FSINFO, 1 },
+        { Nfs3Constant.NFSPROC3_PATHCONF, 1 },
+        { Nfs3Constant.NFSPROC3_COMMIT, 1 } };
+    for (int[] procedure : procedures) {
+      boolean idempotent = procedure[1] == 1;
+      int proc = procedure[0];
+      if (idempotent) {
+        Assert.assertTrue(("Procedure " + proc + " should be idempotent"),
+            RpcProgramNfs3.isIdempotent(proc));
+      } else {
+        Assert.assertFalse(("Procedure " + proc + " should be non-idempotent"),
+            RpcProgramNfs3.isIdempotent(proc));
+      }
+    }
+  }
+}

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

@@ -12,6 +12,8 @@ Trunk (Unreleased)
 
     HDFS-4659 Support setting execution bit for regular files (Brandon Li via sanjay)
 
+    HDFS-4762 Provide HDFS based NFSv3 and Mountd implementation (brandonli)
+
   IMPROVEMENTS
 
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.

+ 1 - 0
hadoop-hdfs-project/pom.xml

@@ -34,6 +34,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <module>hadoop-hdfs</module>
     <module>hadoop-hdfs-httpfs</module>
     <module>hadoop-hdfs/src/contrib/bkjournal</module>
+    <module>hadoop-hdfs-nfs</module>
   </modules>
 
   <build>