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 2013/11/10 05:25:58 UTC

svn commit: r1540444 - in /hadoop/common/branches/branch-2/hadoop-hdfs-project: hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/ hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/ hadoop-hdfs-nfs/src/test/java/org/apache/hado...

Author: brandonli
Date: Sun Nov 10 04:25:57 2013
New Revision: 1540444

URL: http://svn.apache.org/r1540444
Log:
HDFS-5469. Merging change r1540443 from trunk

Added:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestExportsTable.java
      - copied unchanged from r1540443, hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestExportsTable.java
Modified:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/Mountd.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/AsyncDataService.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestMountd.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestReaddir.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

Modified: hadoop/common/branches/branch-2/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/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/Mountd.java?rev=1540444&r1=1540443&r2=1540444&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/Mountd.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/Mountd.java Sun Nov 10 04:25:57 2013
@@ -18,8 +18,6 @@
 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;
@@ -32,23 +30,14 @@ import org.apache.hadoop.mount.MountdBas
  * 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 Mountd(Configuration config) throws IOException {
+    super(new RpcProgramMountd(config));
   }
   
   public static void main(String[] args) throws IOException {
-    List<String> exports = new ArrayList<String>();
-    exports.add("/");
-    Mountd mountd = new Mountd(exports);
+    Configuration config = new Configuration();
+    Mountd mountd = new Mountd(config);
     mountd.start(true);
   }
 }

Modified: hadoop/common/branches/branch-2/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/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java?rev=1540444&r1=1540443&r2=1540444&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java Sun Nov 10 04:25:57 2013
@@ -36,6 +36,7 @@ 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;
@@ -49,6 +50,8 @@ import org.jboss.netty.buffer.ChannelBuf
 import org.jboss.netty.buffer.ChannelBuffers;
 import org.jboss.netty.channel.ChannelHandlerContext;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * RPC program corresponding to mountd daemon. See {@link Mountd}.
  */
@@ -71,23 +74,15 @@ public class RpcProgramMountd extends Rp
   
   private final NfsExports hostsMatcher;
 
-  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 {
+  public RpcProgramMountd(Configuration config) throws IOException {
     // Note that RPC cache is not enabled
     super("mountd", "localhost", config.getInt("nfs3.mountd.port", PORT),
         PROGRAM, VERSION_1, VERSION_3);
-    
+    exports = new ArrayList<String>();
+    exports.add(config.get(Nfs3Constant.EXPORT_POINT,
+        Nfs3Constant.EXPORT_POINT_DEFAULT));
     this.hostsMatcher = NfsExports.getInstance(config);
     this.mounts = Collections.synchronizedList(new ArrayList<MountEntry>());
-    this.exports = Collections.unmodifiableList(exports);
     this.dfsClient = new DFSClient(NameNode.getAddress(config), config);
   }
   
@@ -200,7 +195,7 @@ public class RpcProgramMountd extends Rp
     } else if (mntproc == MNTPROC.UMNTALL) {
       umntall(out, xid, client);
     } else if (mntproc == MNTPROC.EXPORT) {
-      // Currently only support one NFS export "/"
+      // Currently only support one NFS export 
       List<NfsExports> hostsMatchers = new ArrayList<NfsExports>();
       hostsMatchers.add(hostsMatcher);
       out = MountResponse.writeExportList(out, xid, exports, hostsMatchers);
@@ -220,4 +215,9 @@ public class RpcProgramMountd extends Rp
     // Not required, because cache is turned off
     return false;
   }
+
+  @VisibleForTesting
+  public List<String> getExports() {
+    return this.exports;
+  }
 }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/AsyncDataService.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/AsyncDataService.java?rev=1540444&r1=1540443&r2=1540444&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/AsyncDataService.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/AsyncDataService.java Sun Nov 10 04:25:57 2013
@@ -46,6 +46,7 @@ public class AsyncDataService {
 
   public AsyncDataService() {
     threadFactory = new ThreadFactory() {
+      @Override
       public Thread newThread(Runnable r) {
         return new Thread(threadGroup, r);
       }
@@ -129,6 +130,7 @@ public class AsyncDataService {
           + openFileCtx.getNextOffset();
     }
 
+    @Override
     public void run() {
       try {
         openFileCtx.executeWriteBack();

Modified: hadoop/common/branches/branch-2/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/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java?rev=1540444&r1=1540443&r2=1540444&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java Sun Nov 10 04:25:57 2013
@@ -118,6 +118,7 @@ class DFSClientCache {
 
         // Guava requires CacheLoader never returns null.
         return ugi.doAs(new PrivilegedExceptionAction<DFSClient>() {
+          @Override
           public DFSClient run() throws IOException {
             return new DFSClient(NameNode.getAddress(config), config);
           }

Modified: hadoop/common/branches/branch-2/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/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java?rev=1540444&r1=1540443&r2=1540444&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java Sun Nov 10 04:25:57 2013
@@ -18,12 +18,9 @@
 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.mount.MountdBase;
 import org.apache.hadoop.nfs.nfs3.Nfs3Base;
 import org.apache.hadoop.util.StringUtils;
 
@@ -42,28 +39,24 @@ public class Nfs3 extends Nfs3Base {
     Configuration.addDefaultResource("hdfs-site.xml");
   }
   
-  public Nfs3(List<String> exports) throws IOException {
-    super(new RpcProgramNfs3());
-    mountd = new Mountd(exports);
-  }
-
-  @VisibleForTesting
-  public Nfs3(List<String> exports, Configuration config) throws IOException {
-    super(new RpcProgramNfs3(config), config);
-    mountd = new Mountd(exports, config);
+  public Nfs3(Configuration conf) throws IOException {
+    super(new RpcProgramNfs3(conf), conf);
+    mountd = new Mountd(conf);
   }
 
   public Mountd getMountd() {
     return mountd;
   }
   
+  @VisibleForTesting
+  public void startServiceInternal(boolean register) throws IOException {
+    mountd.start(register); // Start mountd
+    start(register);
+  }
+  
   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.mountd.start(true); // Start mountd
-    nfsServer.start(true);
+    StringUtils.startupShutdownMessage(Nfs3.class, args, LOG);    
+    final Nfs3 nfsServer = new Nfs3(new Configuration());
+    nfsServer.startServiceInternal(true);
   }
 }

Modified: hadoop/common/branches/branch-2/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/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java?rev=1540444&r1=1540443&r2=1540444&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java Sun Nov 10 04:25:57 2013
@@ -163,12 +163,9 @@ public class RpcProgramNfs3 extends RpcP
   
   private final RpcCallCache rpcCallCache;
 
-  public RpcProgramNfs3() throws IOException {
-    this(new Configuration());
-  }
-
   public RpcProgramNfs3(Configuration config) throws IOException {
-    super("NFS3", "localhost", Nfs3Constant.PORT, Nfs3Constant.PROGRAM,
+    super("NFS3", "localhost", config.getInt(Nfs3Constant.NFS3_SERVER_PORT,
+        Nfs3Constant.NFS3_SERVER_PORT_DEFAULT), Nfs3Constant.PROGRAM,
         Nfs3Constant.VERSION, Nfs3Constant.VERSION);
    
     config.set(FsPermission.UMASK_LABEL, "000");

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestMountd.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestMountd.java?rev=1540444&r1=1540443&r2=1540444&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestMountd.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestMountd.java Sun Nov 10 04:25:57 2013
@@ -20,8 +20,6 @@ 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;
@@ -45,11 +43,13 @@ public class TestMountd {
         .build();
     cluster.waitActive();
     
+    // Use emphral port in case tests are running in parallel
+    config.setInt("nfs3.mountd.port", 0);
+    config.setInt("nfs3.server.port", 0);
+    
     // Start nfs
-    List<String> exports = new ArrayList<String>();
-    exports.add("/");
-    Nfs3 nfs3 = new Nfs3(exports, config);
-    nfs3.start(false);
+    Nfs3 nfs3 = new Nfs3(config);
+    nfs3.startServiceInternal(false);
 
     RpcProgramMountd mountd = (RpcProgramMountd) nfs3.getMountd()
         .getRpcProgram();

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java?rev=1540444&r1=1540443&r2=1540444&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java Sun Nov 10 04:25:57 2013
@@ -23,6 +23,7 @@ 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.nfs3.Nfs3Utils;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
 import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
@@ -154,7 +155,9 @@ public class TestOutOfOrderWrite {
     Arrays.fill(data3, (byte) 9);
 
     // NFS3 Create request
-    WriteClient client = new WriteClient("localhost", Nfs3Constant.PORT,
+    Configuration conf = new Configuration();
+    WriteClient client = new WriteClient("localhost", conf.getInt(
+        Nfs3Constant.NFS3_SERVER_PORT, Nfs3Constant.NFS3_SERVER_PORT_DEFAULT),
         create(), false);
     client.run();
 

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestReaddir.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestReaddir.java?rev=1540444&r1=1540443&r2=1540444&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestReaddir.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestReaddir.java Sun Nov 10 04:25:57 2013
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertTru
 
 import java.io.IOException;
 import java.net.InetAddress;
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
@@ -67,11 +66,13 @@ public class TestReaddir {
     hdfs = cluster.getFileSystem();
     nn = cluster.getNameNode();
 
+    // Use emphral port in case tests are running in parallel
+    config.setInt("nfs3.mountd.port", 0);
+    config.setInt("nfs3.server.port", 0);
+    
     // Start nfs
-    List<String> exports = new ArrayList<String>();
-    exports.add("/");
-    Nfs3 nfs3 = new Nfs3(exports, config);
-    nfs3.start(false);
+    Nfs3 nfs3 = new Nfs3(config);
+    nfs3.startServiceInternal(false);
 
     nfsd = (RpcProgramNfs3) nfs3.getRpcProgram();
 

Modified: hadoop/common/branches/branch-2/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/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java?rev=1540444&r1=1540443&r2=1540444&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java Sun Nov 10 04:25:57 2013
@@ -23,10 +23,7 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentNavigableMap;
 
 import junit.framework.Assert;
@@ -215,11 +212,13 @@ public class TestWrites {
       cluster.waitActive();
       client = new DFSClient(NameNode.getAddress(config), config);
 
+      // Use emphral port in case tests are running in parallel
+      config.setInt("nfs3.mountd.port", 0);
+      config.setInt("nfs3.server.port", 0);
+      
       // Start nfs
-      List<String> exports = new ArrayList<String>();
-      exports.add("/");
-      Nfs3 nfs3 = new Nfs3(exports, config);
-      nfs3.start(false);
+      Nfs3 nfs3 = new Nfs3(config);
+      nfs3.startServiceInternal(false);
       nfsd = (RpcProgramNfs3) nfs3.getRpcProgram();
 
       HdfsFileStatus status = client.getFileInfo("/");

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1540444&r1=1540443&r2=1540444&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Sun Nov 10 04:25:57 2013
@@ -245,6 +245,9 @@ Release 2.2.1 - UNRELEASED
 
     HDFS-5364. Add OpenFileCtx cache. (brandonli)
 
+    HDFS-5469. Add configuration property for the sub-directroy export path
+    (brandonli)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES