You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by br...@apache.org on 2014/05/31 01:53:03 UTC

svn commit: r1598782 - in /hadoop/common/trunk/hadoop-hdfs-project: hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/ hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/ hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/...

Author: brandonli
Date: Fri May 30 23:53:00 2014
New Revision: 1598782

URL: http://svn.apache.org/r1598782
Log:
HDFS-6056. Clean up NFS config settings. Contributed by Brandon Li

Added:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfiguration.java
Modified:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/Mountd.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/PrivilegedNfsGatewayStarter.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestMountd.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestReaddir.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestDFSClientCache.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestExportsTable.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestOpenFileCtxCache.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/resources/core-site.xml
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java?rev=1598782&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java Fri May 30 23:53:00 2014
@@ -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.conf;
+
+public class NfsConfigKeys {
+
+  // The IP port number for NFS and mountd.
+  public final static String DFS_NFS_SERVER_PORT_KEY = "nfs.server.port";
+  public final static int DFS_NFS_SERVER_PORT_DEFAULT = 2049;
+  public final static String DFS_NFS_MOUNTD_PORT_KEY = "nfs.mountd.port";
+  public final static int DFS_NFS_MOUNTD_PORT_DEFAULT = 4242;
+  
+  public static final String DFS_NFS_FILE_DUMP_KEY = "nfs.file.dump";
+  public static final boolean DFS_NFS_FILE_DUMP_DEFAULT = true;
+  public static final String DFS_NFS_FILE_DUMP_DIR_KEY = "nfs.file.dump.dir";
+  public static final String DFS_NFS_FILE_DUMP_DIR_DEFAULT = "/tmp/.hdfs-nfs";
+  
+  public static final String DFS_NFS_MAX_READ_TRANSFER_SIZE_KEY = "nfs.rtmax";
+  public static final int DFS_NFS_MAX_READ_TRANSFER_SIZE_DEFAULT = 1024 * 1024;
+  public static final String DFS_NFS_MAX_WRITE_TRANSFER_SIZE_KEY = "nfs.wtmax";
+  public static final int DFS_NFS_MAX_WRITE_TRANSFER_SIZE_DEFAULT = 1024 * 1024;
+  public static final String DFS_NFS_MAX_READDIR_TRANSFER_SIZE_KEY = "nfs.dtmax";
+  public static final int DFS_NFS_MAX_READDIR_TRANSFER_SIZE_DEFAULT = 64 * 1024;
+
+  public static final String DFS_NFS_MAX_OPEN_FILES_KEY = "nfs.max.open.files";
+  public static final int DFS_NFS_MAX_OPEN_FILES_DEFAULT = 256;
+
+  public static final String DFS_NFS_STREAM_TIMEOUT_KEY = "nfs.stream.timeout";
+  public static final long DFS_NFS_STREAM_TIMEOUT_DEFAULT = 10 * 60 * 1000; // 10 minutes
+  public static final long DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT = 10 * 1000; // 10 seconds
+
+  public final static String DFS_NFS_EXPORT_POINT_KEY = "nfs.export.point";
+  public final static String DFS_NFS_EXPORT_POINT_DEFAULT = "/";
+  
+  public static final String DFS_NFS_KEYTAB_FILE_KEY = "nfs.keytab.file";
+  public static final String DFS_NFS_KERBEROS_PRINCIPAL_KEY = "nfs.kerberos.principal";
+  public static final String DFS_NFS_REGISTRATION_PORT_KEY = "nfs.registration.port";
+  public static final int    DFS_NFS_REGISTRATION_PORT_DEFAULT = 40; // Currently unassigned.
+  public static final String  DFS_NFS_ALLOW_INSECURE_PORTS_KEY = "nfs.allow.insecure.ports";
+  public static final boolean DFS_NFS_ALLOW_INSECURE_PORTS_DEFAULT = true;
+}
\ No newline at end of file

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfiguration.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfiguration.java?rev=1598782&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfiguration.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfiguration.java Fri May 30 23:53:00 2014
@@ -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.conf;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+
+/**
+ * Adds deprecated keys into the configuration.
+ */
+public class NfsConfiguration extends HdfsConfiguration {
+  static {
+    addDeprecatedKeys();
+  }
+
+  private static void addDeprecatedKeys() {
+    Configuration.addDeprecations(new DeprecationDelta[] {
+        new DeprecationDelta("nfs3.server.port",
+            NfsConfigKeys.DFS_NFS_SERVER_PORT_KEY),
+        new DeprecationDelta("nfs3.mountd.port",
+            NfsConfigKeys.DFS_NFS_MOUNTD_PORT_KEY),
+        new DeprecationDelta("dfs.nfs.exports.cache.expirytime.millis",
+            Nfs3Constant.NFS_EXPORTS_CACHE_EXPIRYTIME_MILLIS_KEY),
+        new DeprecationDelta("hadoop.nfs.userupdate.milly",
+            Nfs3Constant.NFS_USERGROUP_UPDATE_MILLIS_KEY),
+        new DeprecationDelta("dfs.nfs3.enableDump",
+            NfsConfigKeys.DFS_NFS_FILE_DUMP_KEY),
+        new DeprecationDelta("dfs.nfs3.dump.dir",
+            NfsConfigKeys.DFS_NFS_FILE_DUMP_DIR_KEY),
+        new DeprecationDelta("dfs.nfs3.max.open.files",
+            NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_KEY),
+        new DeprecationDelta("dfs.nfs3.stream.timeout",
+            NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_KEY),
+        new DeprecationDelta("dfs.nfs3.export.point",
+            NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY) });
+  }
+}
\ No newline at end of file

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/Mountd.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/Mountd.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/Mountd.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/Mountd.java Fri May 30 23:53:00 2014
@@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.nfs.mount
 import java.io.IOException;
 import java.net.DatagramSocket;
 
-import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.mount.MountdBase;
 
 /**
@@ -32,13 +32,13 @@ import org.apache.hadoop.mount.MountdBas
  */
 public class Mountd extends MountdBase {
 
-  public Mountd(Configuration config, DatagramSocket registrationSocket,
+  public Mountd(NfsConfiguration config, DatagramSocket registrationSocket,
       boolean allowInsecurePorts) throws IOException {
     super(new RpcProgramMountd(config, registrationSocket, allowInsecurePorts));
   }
   
   public static void main(String[] args) throws IOException {
-    Configuration config = new Configuration();
+    NfsConfiguration config = new NfsConfiguration();
     Mountd mountd = new Mountd(config, null, true);
     mountd.start(true);
   }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java Fri May 30 23:53:00 2014
@@ -16,9 +16,6 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hdfs.nfs.mount;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NFS_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY;
-
 import java.io.IOException;
 import java.net.DatagramSocket;
 import java.net.InetAddress;
@@ -29,8 +26,9 @@ 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.nfs.conf.NfsConfigKeys;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.mount.MountEntry;
@@ -39,7 +37,6 @@ import org.apache.hadoop.mount.MountResp
 import org.apache.hadoop.nfs.AccessPrivilege;
 import org.apache.hadoop.nfs.NfsExports;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
-import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
 import org.apache.hadoop.nfs.nfs3.Nfs3Status;
 import org.apache.hadoop.oncrpc.RpcAcceptedReply;
 import org.apache.hadoop.oncrpc.RpcCall;
@@ -66,9 +63,7 @@ public class RpcProgramMountd extends Rp
   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 */
@@ -79,19 +74,22 @@ public class RpcProgramMountd extends Rp
   
   private final NfsExports hostsMatcher;
 
-  public RpcProgramMountd(Configuration config, DatagramSocket registrationSocket,
-      boolean allowInsecurePorts) throws IOException {
+  public RpcProgramMountd(NfsConfiguration config,
+      DatagramSocket registrationSocket, boolean allowInsecurePorts)
+      throws IOException {
     // Note that RPC cache is not enabled
-    super("mountd", "localhost", config.getInt("nfs3.mountd.port", PORT),
-        PROGRAM, VERSION_1, VERSION_3, registrationSocket, allowInsecurePorts);
+    super("mountd", "localhost", config.getInt(
+        NfsConfigKeys.DFS_NFS_MOUNTD_PORT_KEY,
+        NfsConfigKeys.DFS_NFS_MOUNTD_PORT_DEFAULT), PROGRAM, VERSION_1,
+        VERSION_3, registrationSocket, allowInsecurePorts);
     exports = new ArrayList<String>();
-    exports.add(config.get(Nfs3Constant.EXPORT_POINT,
-        Nfs3Constant.EXPORT_POINT_DEFAULT));
+    exports.add(config.get(NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY,
+        NfsConfigKeys.DFS_NFS_EXPORT_POINT_DEFAULT));
     this.hostsMatcher = NfsExports.getInstance(config);
     this.mounts = Collections.synchronizedList(new ArrayList<MountEntry>());
     UserGroupInformation.setConfiguration(config);
-    SecurityUtil.login(config, DFS_NFS_KEYTAB_FILE_KEY,
-            DFS_NFS_KERBEROS_PRINCIPAL_KEY);
+    SecurityUtil.login(config, NfsConfigKeys.DFS_NFS_KEYTAB_FILE_KEY,
+        NfsConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY);
     this.dfsClient = new DFSClient(NameNode.getAddress(config), config);
   }
   

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java Fri May 30 23:53:00 2014
@@ -30,10 +30,10 @@ import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSInputStream;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -72,7 +72,7 @@ class DFSClientCache {
   final static int DEFAULT_DFS_INPUTSTREAM_CACHE_SIZE = 1024;
   final static int DEFAULT_DFS_INPUTSTREAM_CACHE_TTL = 10 * 60;
 
-  private final Configuration config;
+  private final NfsConfiguration config;
 
   private static class DFSInputStreamCaheKey {
     final String userId;
@@ -99,11 +99,11 @@ class DFSClientCache {
     }
   }
 
-  DFSClientCache(Configuration config) {
+  DFSClientCache(NfsConfiguration config) {
     this(config, DEFAULT_DFS_CLIENT_CACHE_SIZE);
   }
   
-  DFSClientCache(Configuration config, int clientCache) {
+  DFSClientCache(NfsConfiguration config, int clientCache) {
     this.config = config;
     this.clientCache = CacheBuilder.newBuilder()
         .maximumSize(clientCache)

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java Fri May 30 23:53:00 2014
@@ -20,8 +20,8 @@ package org.apache.hadoop.hdfs.nfs.nfs3;
 import java.io.IOException;
 import java.net.DatagramSocket;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.nfs.mount.Mountd;
 import org.apache.hadoop.nfs.nfs3.Nfs3Base;
 import org.apache.hadoop.util.StringUtils;
@@ -36,16 +36,11 @@ import com.google.common.annotations.Vis
 public class Nfs3 extends Nfs3Base {
   private Mountd mountd;
   
-  static {
-    Configuration.addDefaultResource("hdfs-default.xml");
-    Configuration.addDefaultResource("hdfs-site.xml");
-  }
-  
-  public Nfs3(Configuration conf) throws IOException {
+  public Nfs3(NfsConfiguration conf) throws IOException {
     this(conf, null, true);
   }
   
-  public Nfs3(Configuration conf, DatagramSocket registrationSocket,
+  public Nfs3(NfsConfiguration conf, DatagramSocket registrationSocket,
       boolean allowInsecurePorts) throws IOException {
     super(new RpcProgramNfs3(conf, registrationSocket, allowInsecurePorts), conf);
     mountd = new Mountd(conf, registrationSocket, allowInsecurePorts);
@@ -64,11 +59,11 @@ public class Nfs3 extends Nfs3Base {
   static void startService(String[] args,
       DatagramSocket registrationSocket) throws IOException {
     StringUtils.startupShutdownMessage(Nfs3.class, args, LOG);
-    Configuration conf = new Configuration();
+    NfsConfiguration conf = new NfsConfiguration();
     boolean allowInsecurePorts = conf.getBoolean(
-        DFSConfigKeys.DFS_NFS_ALLOW_INSECURE_PORTS_KEY,
-        DFSConfigKeys.DFS_NFS_ALLOW_INSECURE_PORTS_DEFAULT);
-    final Nfs3 nfsServer = new Nfs3(new Configuration(), registrationSocket,
+        NfsConfigKeys.DFS_NFS_ALLOW_INSECURE_PORTS_KEY,
+        NfsConfigKeys.DFS_NFS_ALLOW_INSECURE_PORTS_DEFAULT);
+    final Nfs3 nfsServer = new Nfs3(conf, registrationSocket,
         allowInsecurePorts);
     nfsServer.startServiceInternal(true);
   }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java Fri May 30 23:53:00 2014
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FSDataInputS
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
 import org.apache.hadoop.hdfs.nfs.nfs3.WriteCtx.DataState;
 import org.apache.hadoop.io.BytesWritable.Comparator;
 import org.apache.hadoop.io.IOUtils;
@@ -823,7 +824,7 @@ class OpenFileCtx {
    */
   public synchronized boolean streamCleanup(long fileId, long streamTimeout) {
     Preconditions
-        .checkState(streamTimeout >= Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT);
+        .checkState(streamTimeout >= NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT);
     if (!activeState) {
       return true;
     }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java Fri May 30 23:53:00 2014
@@ -24,9 +24,9 @@ import java.util.concurrent.ConcurrentMa
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
-import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Time;
 
@@ -48,9 +48,9 @@ class OpenFileCtxCache {
   private final long streamTimeout;
   private final StreamMonitor streamMonitor;
 
-  OpenFileCtxCache(Configuration config, long streamTimeout) {
-    maxStreams = config.getInt(Nfs3Constant.MAX_OPEN_FILES,
-        Nfs3Constant.MAX_OPEN_FILES_DEFAULT);
+  OpenFileCtxCache(NfsConfiguration config, long streamTimeout) {
+    maxStreams = config.getInt(NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_KEY,
+        NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_DEFAULT);
     LOG.info("Maximum open streams is " + maxStreams);
     this.streamTimeout = streamTimeout;
     streamMonitor = new StreamMonitor();
@@ -102,7 +102,7 @@ class OpenFileCtxCache {
     } else {
       long idleTime = Time.monotonicNow()
           - idlest.getValue().getLastAccessTime();
-      if (idleTime < Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT) {
+      if (idleTime < NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("idlest stream's idle time:" + idleTime);
         }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/PrivilegedNfsGatewayStarter.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/PrivilegedNfsGatewayStarter.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/PrivilegedNfsGatewayStarter.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/PrivilegedNfsGatewayStarter.java Fri May 30 23:53:00 2014
@@ -21,9 +21,8 @@ import java.net.InetSocketAddress;
 
 import org.apache.commons.daemon.Daemon;
 import org.apache.commons.daemon.DaemonContext;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 
 /**
  * This class is used to allow the initial registration of the NFS gateway with
@@ -42,12 +41,12 @@ public class PrivilegedNfsGatewayStarter
   @Override
   public void init(DaemonContext context) throws Exception {
     System.err.println("Initializing privileged NFS client socket...");
-    Configuration conf = new HdfsConfiguration();
-    int clientPort = conf.getInt(DFSConfigKeys.DFS_NFS_REGISTRATION_PORT_KEY,
-        DFSConfigKeys.DFS_NFS_REGISTRATION_PORT_DEFAULT);
+    NfsConfiguration conf = new NfsConfiguration();
+    int clientPort = conf.getInt(NfsConfigKeys.DFS_NFS_REGISTRATION_PORT_KEY,
+        NfsConfigKeys.DFS_NFS_REGISTRATION_PORT_DEFAULT);
     if (clientPort < 1 || clientPort > 1023) {
       throw new RuntimeException("Must start privileged NFS server with '" +
-          DFSConfigKeys.DFS_NFS_REGISTRATION_PORT_KEY + "' configured to a " +
+          NfsConfigKeys.DFS_NFS_REGISTRATION_PORT_KEY + "' configured to a " +
           "privileged port.");
     }
     registrationSocket = new DatagramSocket(

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java Fri May 30 23:53:00 2014
@@ -28,7 +28,6 @@ import java.util.EnumSet;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
@@ -41,6 +40,8 @@ import org.apache.hadoop.fs.permission.F
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -131,9 +132,6 @@ import org.jboss.netty.channel.ChannelHa
 
 import com.google.common.annotations.VisibleForTesting;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NFS_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY;
-
 /**
  * RPC program corresponding to nfs daemon. See {@link Nfs3}.
  */
@@ -144,7 +142,7 @@ public class RpcProgramNfs3 extends RpcP
   
   static final Log LOG = LogFactory.getLog(RpcProgramNfs3.class);
 
-  private final Configuration config = new Configuration();
+  private final NfsConfiguration config;
   private final WriteManager writeManager;
   private final IdUserGroup iug;
   private final DFSClientCache clientCache;
@@ -159,15 +157,17 @@ public class RpcProgramNfs3 extends RpcP
   
   private final RpcCallCache rpcCallCache;
 
-  public RpcProgramNfs3(Configuration config, DatagramSocket registrationSocket,
+  public RpcProgramNfs3(NfsConfiguration config, DatagramSocket registrationSocket,
       boolean allowInsecurePorts) throws IOException {
-    super("NFS3", "localhost", config.getInt(Nfs3Constant.NFS3_SERVER_PORT,
-        Nfs3Constant.NFS3_SERVER_PORT_DEFAULT), Nfs3Constant.PROGRAM,
+    super("NFS3", "localhost", config.getInt(
+        NfsConfigKeys.DFS_NFS_SERVER_PORT_KEY,
+        NfsConfigKeys.DFS_NFS_SERVER_PORT_DEFAULT), Nfs3Constant.PROGRAM,
         Nfs3Constant.VERSION, Nfs3Constant.VERSION, registrationSocket,
         allowInsecurePorts);
    
+    this.config = config;
     config.set(FsPermission.UMASK_LABEL, "000");
-    iug = new IdUserGroup();
+    iug = new IdUserGroup(config);
     
     exports = NfsExports.getInstance(config);
     writeManager = new WriteManager(iug, config);
@@ -180,13 +180,13 @@ public class RpcProgramNfs3 extends RpcP
         CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
         CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
     
-    writeDumpDir = config.get(Nfs3Constant.FILE_DUMP_DIR_KEY,
-        Nfs3Constant.FILE_DUMP_DIR_DEFAULT);
-    boolean enableDump = config.getBoolean(Nfs3Constant.ENABLE_FILE_DUMP_KEY,
-        Nfs3Constant.ENABLE_FILE_DUMP_DEFAULT);
+    writeDumpDir = config.get(NfsConfigKeys.DFS_NFS_FILE_DUMP_DIR_KEY,
+        NfsConfigKeys.DFS_NFS_FILE_DUMP_DIR_DEFAULT);
+    boolean enableDump = config.getBoolean(NfsConfigKeys.DFS_NFS_FILE_DUMP_KEY,
+        NfsConfigKeys.DFS_NFS_FILE_DUMP_DEFAULT);
     UserGroupInformation.setConfiguration(config);
-    SecurityUtil.login(config, DFS_NFS_KEYTAB_FILE_KEY,
-            DFS_NFS_KERBEROS_PRINCIPAL_KEY);
+    SecurityUtil.login(config, NfsConfigKeys.DFS_NFS_KEYTAB_FILE_KEY,
+        NfsConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY);
 
     if (!enableDump) {
       writeDumpDir = null;
@@ -567,8 +567,8 @@ public class RpcProgramNfs3 extends RpcP
             + handle.getFileId());
         return new READLINK3Response(Nfs3Status.NFS3ERR_SERVERFAULT);
       }
-      int rtmax = config.getInt(Nfs3Constant.MAX_READ_TRANSFER_SIZE_KEY,
-              Nfs3Constant.MAX_READ_TRANSFER_SIZE_DEFAULT);
+      int rtmax = config.getInt(NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_KEY,
+          NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_DEFAULT);
       if (rtmax < target.getBytes().length) {
         LOG.error("Link size: " + target.getBytes().length
             + " is larger than max transfer size: " + rtmax);
@@ -665,8 +665,8 @@ public class RpcProgramNfs3 extends RpcP
     }
 
     try {
-      int rtmax = config.getInt(Nfs3Constant.MAX_READ_TRANSFER_SIZE_KEY,
-              Nfs3Constant.MAX_READ_TRANSFER_SIZE_DEFAULT);
+      int rtmax = config.getInt(NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_KEY,
+          NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_DEFAULT);
       int buffSize = Math.min(rtmax, count);
       byte[] readbuffer = new byte[buffSize];
 
@@ -1740,12 +1740,15 @@ public class RpcProgramNfs3 extends RpcP
     }
 
     try {
-      int rtmax = config.getInt(Nfs3Constant.MAX_READ_TRANSFER_SIZE_KEY,
-              Nfs3Constant.MAX_READ_TRANSFER_SIZE_DEFAULT);
-      int wtmax = config.getInt(Nfs3Constant.MAX_WRITE_TRANSFER_SIZE_KEY,
-              Nfs3Constant.MAX_WRITE_TRANSFER_SIZE_DEFAULT);
-      int dtperf = config.getInt(Nfs3Constant.MAX_READDIR_TRANSFER_SIZE_KEY,
-              Nfs3Constant.MAX_READDIR_TRANSFER_SIZE_DEFAULT);
+      int rtmax = config.getInt(
+          NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_KEY,
+          NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_DEFAULT);
+      int wtmax = config.getInt(
+          NfsConfigKeys.DFS_NFS_MAX_WRITE_TRANSFER_SIZE_KEY,
+          NfsConfigKeys.DFS_NFS_MAX_WRITE_TRANSFER_SIZE_DEFAULT);
+      int dtperf = config.getInt(
+          NfsConfigKeys.DFS_NFS_MAX_READDIR_TRANSFER_SIZE_KEY,
+          NfsConfigKeys.DFS_NFS_MAX_READDIR_TRANSFER_SIZE_DEFAULT);
 
       Nfs3FileAttributes attrs = Nfs3Utils.getFileAttr(dfsClient,
           Nfs3Utils.getFileIdPath(handle), iug);

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java Fri May 30 23:53:00 2014
@@ -21,10 +21,11 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.nfs.nfs3.OpenFileCtx.COMMIT_STATUS;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.ipc.RemoteException;
@@ -50,7 +51,7 @@ import com.google.common.annotations.Vis
 public class WriteManager {
   public static final Log LOG = LogFactory.getLog(WriteManager.class);
 
-  private final Configuration config;
+  private final NfsConfiguration config;
   private final IdUserGroup iug;
  
   private AsyncDataService asyncDataService;
@@ -78,19 +79,19 @@ public class WriteManager {
     return fileContextCache.put(h, ctx);
   }
   
-  WriteManager(IdUserGroup iug, final Configuration config) {
+  WriteManager(IdUserGroup iug, final NfsConfiguration config) {
     this.iug = iug;
     this.config = config;
-    streamTimeout = config.getLong(Nfs3Constant.OUTPUT_STREAM_TIMEOUT,
-        Nfs3Constant.OUTPUT_STREAM_TIMEOUT_DEFAULT);
+    streamTimeout = config.getLong(NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_KEY,
+        NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_DEFAULT);
     LOG.info("Stream timeout is " + streamTimeout + "ms.");
-    if (streamTimeout < Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT) {
+    if (streamTimeout < NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT) {
       LOG.info("Reset stream timeout to minimum value "
-          + Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT + "ms.");
-      streamTimeout = Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT;
+          + NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT + "ms.");
+      streamTimeout = NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT;
     }
-    maxStreams = config.getInt(Nfs3Constant.MAX_OPEN_FILES,
-        Nfs3Constant.MAX_OPEN_FILES_DEFAULT);
+    maxStreams = config.getInt(NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_KEY,
+        NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_DEFAULT);
     LOG.info("Maximum open streams is "+ maxStreams);
     this.fileContextCache = new OpenFileCtxCache(config, streamTimeout);
   }
@@ -171,8 +172,8 @@ public class WriteManager {
       }
 
       // Add open stream
-      String writeDumpDir = config.get(Nfs3Constant.FILE_DUMP_DIR_KEY,
-          Nfs3Constant.FILE_DUMP_DIR_DEFAULT);
+      String writeDumpDir = config.get(NfsConfigKeys.DFS_NFS_FILE_DUMP_DIR_KEY,
+          NfsConfigKeys.DFS_NFS_FILE_DUMP_DIR_DEFAULT);
       openFileCtx = new OpenFileCtx(fos, latestAttr, writeDumpDir + "/"
           + fileHandle.getFileId(), dfsClient, iug);
 

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestMountd.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestMountd.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestMountd.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestMountd.java Fri May 30 23:53:00 2014
@@ -23,8 +23,8 @@ import java.net.InetAddress;
 
 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.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.nfs.mount.RpcProgramMountd;
 import org.apache.hadoop.hdfs.nfs.nfs3.Nfs3;
 import org.apache.hadoop.hdfs.nfs.nfs3.RpcProgramNfs3;
@@ -38,7 +38,7 @@ public class TestMountd {
   @Test
   public void testStart() throws IOException {
     // Start minicluster
-    Configuration config = new Configuration();
+    NfsConfiguration config = new NfsConfiguration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(config).numDataNodes(1)
         .build();
     cluster.waitActive();

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java Fri May 30 23:53:00 2014
@@ -23,7 +23,8 @@ import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.nfs.nfs3.Nfs3Utils;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
 import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
@@ -155,10 +156,10 @@ public class TestOutOfOrderWrite {
     Arrays.fill(data3, (byte) 9);
 
     // NFS3 Create request
-    Configuration conf = new Configuration();
+    NfsConfiguration conf = new NfsConfiguration();
     WriteClient client = new WriteClient("localhost", conf.getInt(
-        Nfs3Constant.NFS3_SERVER_PORT, Nfs3Constant.NFS3_SERVER_PORT_DEFAULT),
-        create(), false);
+        NfsConfigKeys.DFS_NFS_SERVER_PORT_KEY,
+        NfsConfigKeys.DFS_NFS_SERVER_PORT_DEFAULT), create(), false);
     client.run();
 
     while (handle == null) {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestReaddir.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestReaddir.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestReaddir.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestReaddir.java Fri May 30 23:53:00 2014
@@ -24,11 +24,11 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.util.List;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.nfs.nfs3.Nfs3;
 import org.apache.hadoop.hdfs.nfs.nfs3.RpcProgramNfs3;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -53,7 +53,7 @@ import org.mockito.Mockito;
  */
 public class TestReaddir {
 
-  static Configuration config = new Configuration();
+  static NfsConfiguration config = new NfsConfiguration();
   static MiniDFSCluster cluster = null;
   static DistributedFileSystem hdfs;
   static NameNode nn;

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestDFSClientCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestDFSClientCache.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestDFSClientCache.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestDFSClientCache.java Fri May 30 23:53:00 2014
@@ -18,24 +18,24 @@
 package org.apache.hadoop.hdfs.nfs.nfs3;
 
 import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.KERBEROS;
+import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertThat;
-import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.Test;
 
 public class TestDFSClientCache {
   @Test
   public void testEviction() throws IOException {
-    Configuration conf = new Configuration();
+    NfsConfiguration conf = new NfsConfiguration();
     conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "hdfs://localhost");
 
     // Only one entry will be in the cache
@@ -59,7 +59,7 @@ public class TestDFSClientCache {
     String currentUser = "test-user";
 
 
-    Configuration conf = new Configuration();
+    NfsConfiguration conf = new NfsConfiguration();
     UserGroupInformation currentUserUgi
             = UserGroupInformation.createRemoteUser(currentUser);
     currentUserUgi.setAuthenticationMethod(KERBEROS);
@@ -83,7 +83,7 @@ public class TestDFSClientCache {
 
     UserGroupInformation currentUserUgi = UserGroupInformation
             .createUserForTesting(currentUser, new String[0]);
-    Configuration conf = new Configuration();
+    NfsConfiguration conf = new NfsConfiguration();
     conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "hdfs://localhost");
     DFSClientCache cache = new DFSClientCache(conf);
     UserGroupInformation ugiResult

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestExportsTable.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestExportsTable.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestExportsTable.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestExportsTable.java Fri May 30 23:53:00 2014
@@ -21,22 +21,22 @@ import static org.junit.Assert.assertTru
 
 import java.io.IOException;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.nfs.mount.Mountd;
 import org.apache.hadoop.hdfs.nfs.mount.RpcProgramMountd;
-import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
 import org.junit.Test;
 
 public class TestExportsTable {
  
   @Test
   public void testExportPoint() throws IOException {
-    Configuration config = new Configuration();
+    NfsConfiguration config = new NfsConfiguration();
     MiniDFSCluster cluster = null;
 
     String exportPoint = "/myexport1";
-    config.setStrings(Nfs3Constant.EXPORT_POINT, exportPoint);
+    config.setStrings(NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY, exportPoint);
     // Use emphral port in case tests are running in parallel
     config.setInt("nfs3.mountd.port", 0);
     config.setInt("nfs3.server.port", 0);

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestOpenFileCtxCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestOpenFileCtxCache.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestOpenFileCtxCache.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestOpenFileCtxCache.java Fri May 30 23:53:00 2014
@@ -22,13 +22,13 @@ import static org.junit.Assert.assertTru
 
 import java.io.IOException;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.nfs.nfs3.OpenFileCtx.CommitCtx;
 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.junit.Test;
 import org.mockito.Mockito;
@@ -38,10 +38,10 @@ public class TestOpenFileCtxCache {
 
   @Test
   public void testEviction() throws IOException, InterruptedException {
-    Configuration conf = new Configuration();
+    NfsConfiguration conf = new NfsConfiguration();
 
     // Only two entries will be in the cache
-    conf.setInt(Nfs3Constant.MAX_OPEN_FILES, 2);
+    conf.setInt(NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_KEY, 2);
 
     DFSClient dfsClient = Mockito.mock(DFSClient.class);
     Nfs3FileAttributes attr = new Nfs3FileAttributes();
@@ -49,15 +49,15 @@ public class TestOpenFileCtxCache {
     Mockito.when(fos.getPos()).thenReturn((long) 0);
 
     OpenFileCtx context1 = new OpenFileCtx(fos, attr, "/dumpFilePath",
-        dfsClient, new IdUserGroup());
+        dfsClient, new IdUserGroup(new NfsConfiguration()));
     OpenFileCtx context2 = new OpenFileCtx(fos, attr, "/dumpFilePath",
-        dfsClient, new IdUserGroup());
+        dfsClient, new IdUserGroup(new NfsConfiguration()));
     OpenFileCtx context3 = new OpenFileCtx(fos, attr, "/dumpFilePath",
-        dfsClient, new IdUserGroup());
+        dfsClient, new IdUserGroup(new NfsConfiguration()));
     OpenFileCtx context4 = new OpenFileCtx(fos, attr, "/dumpFilePath",
-        dfsClient, new IdUserGroup());
+        dfsClient, new IdUserGroup(new NfsConfiguration()));
     OpenFileCtx context5 = new OpenFileCtx(fos, attr, "/dumpFilePath",
-        dfsClient, new IdUserGroup());
+        dfsClient, new IdUserGroup(new NfsConfiguration()));
 
     OpenFileCtxCache cache = new OpenFileCtxCache(conf, 10 * 60 * 100);
 
@@ -71,7 +71,7 @@ public class TestOpenFileCtxCache {
     assertTrue(cache.size() == 2);
 
     // Wait for the oldest stream to be evict-able, insert again
-    Thread.sleep(Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT);
+    Thread.sleep(NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT);
     assertTrue(cache.size() == 2);
 
     ret = cache.put(new FileHandle(3), context3);
@@ -90,17 +90,17 @@ public class TestOpenFileCtxCache {
         new WriteCtx(null, 0, 0, 0, null, null, null, 0, false, null));
     context4.getPendingCommitsForTest().put(new Long(100),
         new CommitCtx(0, null, 0, attr));
-    Thread.sleep(Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT);
+    Thread.sleep(NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT);
     ret = cache.put(new FileHandle(5), context5);
     assertFalse(ret);
   }
 
   @Test
   public void testScan() throws IOException, InterruptedException {
-    Configuration conf = new Configuration();
+    NfsConfiguration conf = new NfsConfiguration();
 
     // Only two entries will be in the cache
-    conf.setInt(Nfs3Constant.MAX_OPEN_FILES, 2);
+    conf.setInt(NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_KEY, 2);
 
     DFSClient dfsClient = Mockito.mock(DFSClient.class);
     Nfs3FileAttributes attr = new Nfs3FileAttributes();
@@ -108,13 +108,13 @@ public class TestOpenFileCtxCache {
     Mockito.when(fos.getPos()).thenReturn((long) 0);
 
     OpenFileCtx context1 = new OpenFileCtx(fos, attr, "/dumpFilePath",
-        dfsClient, new IdUserGroup());
+        dfsClient, new IdUserGroup(new NfsConfiguration()));
     OpenFileCtx context2 = new OpenFileCtx(fos, attr, "/dumpFilePath",
-        dfsClient, new IdUserGroup());
+        dfsClient, new IdUserGroup(new NfsConfiguration()));
     OpenFileCtx context3 = new OpenFileCtx(fos, attr, "/dumpFilePath",
-        dfsClient, new IdUserGroup());
+        dfsClient, new IdUserGroup(new NfsConfiguration()));
     OpenFileCtx context4 = new OpenFileCtx(fos, attr, "/dumpFilePath",
-        dfsClient, new IdUserGroup());
+        dfsClient, new IdUserGroup(new NfsConfiguration()));
 
     OpenFileCtxCache cache = new OpenFileCtxCache(conf, 10 * 60 * 100);
 
@@ -123,8 +123,8 @@ public class TestOpenFileCtxCache {
     assertTrue(ret);
     ret = cache.put(new FileHandle(2), context2);
     assertTrue(ret);
-    Thread.sleep(Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT + 1);
-    cache.scan(Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT);
+    Thread.sleep(NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT + 1);
+    cache.scan(NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT);
     assertTrue(cache.size() == 0);
 
     // Test cleaning inactive entry
@@ -133,7 +133,7 @@ public class TestOpenFileCtxCache {
     ret = cache.put(new FileHandle(4), context4);
     assertTrue(ret);
     context3.setActiveStatusForTest(false);
-    cache.scan(Nfs3Constant.OUTPUT_STREAM_TIMEOUT_DEFAULT);
+    cache.scan(NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_DEFAULT);
     assertTrue(cache.size() == 1);
     assertTrue(cache.get(new FileHandle(3)) == null);
     assertTrue(cache.get(new FileHandle(4)) != null);

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java Fri May 30 23:53:00 2014
@@ -17,6 +17,11 @@
  */
 package org.apache.hadoop.hdfs.nfs.nfs3;
 
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
 import org.junit.Assert;
 import org.junit.Test;
@@ -63,4 +68,41 @@ public class TestRpcProgramNfs3 {
       }
     }
   }
+
+  @Test
+  public void testDeprecatedKeys() {
+    NfsConfiguration conf = new NfsConfiguration();
+    conf.setInt("nfs3.server.port", 998);
+    assertTrue(conf.getInt(NfsConfigKeys.DFS_NFS_SERVER_PORT_KEY, 0) == 998);
+
+    conf.setInt("nfs3.mountd.port", 999);
+    assertTrue(conf.getInt(NfsConfigKeys.DFS_NFS_MOUNTD_PORT_KEY, 0) == 999);
+
+    conf.set("dfs.nfs.exports.allowed.hosts", "host1");
+    assertTrue(conf.get(CommonConfigurationKeys.NFS_EXPORTS_ALLOWED_HOSTS_KEY)
+        .equals("host1"));
+
+    conf.setInt("dfs.nfs.exports.cache.expirytime.millis", 1000);
+    assertTrue(conf.getInt(
+        Nfs3Constant.NFS_EXPORTS_CACHE_EXPIRYTIME_MILLIS_KEY, 0) == 1000);
+
+    conf.setInt("hadoop.nfs.userupdate.milly", 10);
+    assertTrue(conf.getInt(Nfs3Constant.NFS_USERGROUP_UPDATE_MILLIS_KEY, 0) == 10);
+
+    conf.set("dfs.nfs3.dump.dir", "/nfs/tmp");
+    assertTrue(conf.get(NfsConfigKeys.DFS_NFS_FILE_DUMP_DIR_KEY).equals(
+        "/nfs/tmp"));
+
+    conf.setBoolean("dfs.nfs3.enableDump", false);
+    assertTrue(conf.getBoolean(NfsConfigKeys.DFS_NFS_FILE_DUMP_KEY, true) == false);
+
+    conf.setInt("dfs.nfs3.max.open.files", 500);
+    assertTrue(conf.getInt(NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_KEY, 0) == 500);
+
+    conf.setInt("dfs.nfs3.stream.timeout", 6000);
+    assertTrue(conf.getInt(NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_KEY, 0) == 6000);
+
+    conf.set("dfs.nfs3.export.point", "/dir1");
+    assertTrue(conf.get(NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY).equals("/dir1"));
+  }
 }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java Fri May 30 23:53:00 2014
@@ -27,11 +27,10 @@ import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.concurrent.ConcurrentNavigableMap;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSClient;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.nfs.nfs3.OpenFileCtx.COMMIT_STATUS;
 import org.apache.hadoop.hdfs.nfs.nfs3.OpenFileCtx.CommitCtx;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -139,7 +138,7 @@ public class TestWrites {
     Mockito.when(fos.getPos()).thenReturn((long) 0);
 
     OpenFileCtx ctx = new OpenFileCtx(fos, attr, "/dumpFilePath", dfsClient,
-        new IdUserGroup());
+        new IdUserGroup(new NfsConfiguration()));
 
     COMMIT_STATUS ret;
 
@@ -201,13 +200,14 @@ public class TestWrites {
     Nfs3FileAttributes attr = new Nfs3FileAttributes();
     HdfsDataOutputStream fos = Mockito.mock(HdfsDataOutputStream.class);
     Mockito.when(fos.getPos()).thenReturn((long) 0);
+    NfsConfiguration config = new NfsConfiguration();
 
     OpenFileCtx ctx = new OpenFileCtx(fos, attr, "/dumpFilePath", dfsClient,
-        new IdUserGroup());
+        new IdUserGroup(config));
 
     FileHandle h = new FileHandle(1); // fake handle for "/dumpFilePath"
     COMMIT_STATUS ret;
-    WriteManager wm = new WriteManager(new IdUserGroup(), new Configuration());
+    WriteManager wm = new WriteManager(new IdUserGroup(config), config);
     assertTrue(wm.addOpenFileStream(h, ctx));
     
     // Test inactive open file context
@@ -280,7 +280,7 @@ public class TestWrites {
 
   @Test
   public void testWriteStableHow() throws IOException, InterruptedException {
-    HdfsConfiguration config = new HdfsConfiguration();
+    NfsConfiguration config = new NfsConfiguration();
     DFSClient client = null;
     MiniDFSCluster cluster = null;
     RpcProgramNfs3 nfsd;

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/resources/core-site.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/resources/core-site.xml?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/resources/core-site.xml (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/resources/core-site.xml Fri May 30 23:53:00 2014
@@ -18,12 +18,12 @@
 
 <configuration>
 <property>
-  <name>nfs3.server.port</name>
+  <name>nfs.server.port</name>
   <value>2079</value>
 </property>
 
 <property>
-  <name>nfs3.mountd.port</name>
+  <name>nfs.mountd.port</name>
   <value>4272</value>
 </property>
 </configuration>

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Fri May 30 23:53:00 2014
@@ -459,6 +459,8 @@ Release 2.5.0 - UNRELEASED
 
     HDFS-6472. fix typo in webapps/hdfs/explorer.js. (Juan Yu via wang)
 
+    HDFS-6056. Clean up NFS config settings (brandonli)
+
   OPTIMIZATIONS
 
     HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java Fri May 30 23:53:00 2014
@@ -639,12 +639,6 @@ public class DFSConfigKeys extends Commo
   public static final String DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE =
       "dfs.client.hedged.read.threadpool.size";
   public static final int     DEFAULT_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE = 0;
-  public static final String  DFS_NFS_KEYTAB_FILE_KEY = "dfs.nfs.keytab.file";
-  public static final String  DFS_NFS_KERBEROS_PRINCIPAL_KEY = "dfs.nfs.kerberos.principal";
-  public static final String  DFS_NFS_REGISTRATION_PORT_KEY = "dfs.nfs.registration.port";
-  public static final int     DFS_NFS_REGISTRATION_PORT_DEFAULT = 40; // Currently unassigned.
-  public static final String  DFS_NFS_ALLOW_INSECURE_PORTS_KEY = "dfs.nfs.allow.insecure.ports";
-  public static final boolean DFS_NFS_ALLOW_INSECURE_PORTS_DEFAULT = true;
 
    // Slow io warning log threshold settings for dfsclient and datanode.
    public static final String DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY =

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml Fri May 30 23:53:00 2014
@@ -1296,8 +1296,56 @@
     non-zero integer.
   </description>
 </property>
+
+<property>
+  <name>nfs.server.port</name>
+  <value>2049</value>
+  <description>
+      Specify the port number used by Hadoop NFS.
+  </description>
+</property>
+
+<property>
+  <name>nfs.mountd.port</name>
+  <value>4242</value>
+  <description>
+      Specify the port number used by Hadoop mount daemon.
+  </description>
+</property>
+
+<property>    
+  <name>nfs.dump.dir</name>
+  <value>/tmp/.hdfs-nfs</value>
+  <description>
+    This directory is used to temporarily save out-of-order writes before
+    writing to HDFS. For each file, the out-of-order writes are dumped after
+    they are accumulated to exceed certain threshold (e.g., 1MB) in memory. 
+    One needs to make sure the directory has enough space.
+  </description>
+</property>
+
+<property>
+  <name>nfs.rtmax</name>
+  <value>1048576</value>
+  <description>This is the maximum size in bytes of a READ request
+    supported by the NFS gateway. If you change this, make sure you
+    also update the nfs mount's rsize(add rsize= # of bytes to the 
+    mount directive).
+  </description>
+</property>
+
+<property>
+  <name>nfs.wtmax</name>
+  <value>1048576</value>
+  <description>This is the maximum size in bytes of a WRITE request
+    supported by the NFS gateway. If you change this, make sure you
+    also update the nfs mount's wsize(add wsize= # of bytes to the 
+    mount directive).
+  </description>
+</property>
+
 <property>
-  <name>dfs.nfs.keytab.file</name>
+  <name>nfs.keytab.file</name>
   <value></value>
   <description>
     *Note*: Advanced property. Change with caution.
@@ -1307,7 +1355,7 @@
 </property>
 
 <property>
-  <name>dfs.nfs.kerberos.principal</name>
+  <name>nfs.kerberos.principal</name>
   <value></value>
   <description>
     *Note*: Advanced property. Change with caution.
@@ -1318,7 +1366,7 @@
 </property>
 
 <property>
-  <name>dfs.nfs.allow.insecure.ports</name>
+  <name>nfs.allow.insecure.ports</name>
   <value>true</value>
   <description>
     When set to false, client connections originating from unprivileged ports

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm?rev=1598782&r1=1598781&r2=1598782&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm Fri May 30 23:53:00 2014
@@ -76,14 +76,14 @@ HDFS NFS Gateway
 
 ----
   <property>
-    <name>dfs.nfs.keytab.file</name>
+    <name>nfs.keytab.file</name>
     <value>/etc/hadoop/conf/nfsserver.keytab</value> <!-- path to the nfs gateway keytab -->
   </property>
 ----
 
 ----
   <property>
-    <name>dfs.nfs.kerberos.principal</name>
+    <name>nfs.kerberos.principal</name>
     <value>nfsserver/_HOST@YOUR-REALM.COM</value>
   </property>
 ----
@@ -121,7 +121,7 @@ HDFS NFS Gateway
 
 ----
   <property>    
-    <name>dfs.nfs3.dump.dir</name>
+    <name>nfs.dump.dir</name>
     <value>/tmp/.hdfs-nfs</value>
   </property>
 ---- 
@@ -134,7 +134,7 @@ HDFS NFS Gateway
 
 ----
 <property>
-  <name>dfs.nfs.rtmax</name>
+  <name>nfs.rtmax</name>
   <value>1048576</value>
   <description>This is the maximum size in bytes of a READ request
     supported by the NFS gateway. If you change this, make sure you
@@ -146,7 +146,7 @@ HDFS NFS Gateway
 
 ----
 <property>
-  <name>dfs.nfs.wtmax</name>
+  <name>nfs.wtmax</name>
   <value>65536</value>
   <description>This is the maximum size in bytes of a WRITE request
     supported by the NFS gateway. If you change this, make sure you
@@ -167,7 +167,7 @@ HDFS NFS Gateway
 
 ----
 <property>
-  <name>dfs.nfs.exports.allowed.hosts</name>
+  <name>nfs.exports.allowed.hosts</name>
   <value>* rw</value>
 </property>
 ----
@@ -345,7 +345,7 @@ HDFS NFS Gateway
   file in the event one wishes to access the HDFS NFS Gateway from a system with
   a completely disparate set of UIDs/GIDs. By default this file is located at
   "/etc/nfs.map", but a custom location can be configured by setting the
-  "dfs.nfs.static.mapping.file" property to the path of the static mapping file.
+  "nfs.static.mapping.file" property to the path of the static mapping file.
   The format of the static mapping file is similar to what is described in the
   exports(5) manual page, but roughly it is: