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 sz...@apache.org on 2014/07/29 02:49:19 UTC

svn commit: r1614234 [2/2] - in /hadoop/common/branches/HDFS-6584/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/ hadoop-hdfs/src/main/java...

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShm.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShm.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShm.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShm.java Tue Jul 29 00:49:14 2014
@@ -32,11 +32,16 @@ import com.google.common.base.Preconditi
  * DfsClientShm is a subclass of ShortCircuitShm which is used by the
  * DfsClient.
  * When the UNIX domain socket associated with this shared memory segment
- * closes unexpectedly, we mark the slots inside this segment as stale.
- * ShortCircuitReplica objects that contain stale slots are themselves stale,
+ * closes unexpectedly, we mark the slots inside this segment as disconnected.
+ * ShortCircuitReplica objects that contain disconnected slots are stale,
  * and will not be used to service new reads or mmap operations.
  * However, in-progress read or mmap operations will continue to proceed.
  * Once the last slot is deallocated, the segment can be safely munmapped.
+ *
+ * Slots may also become stale because the associated replica has been deleted
+ * on the DataNode.  In this case, the DataNode will clear the 'valid' bit.
+ * The client will then see these slots as stale (see
+ * #{ShortCircuitReplica#isStale}).
  */
 public class DfsClientShm extends ShortCircuitShm
     implements DomainSocketWatcher.Handler {
@@ -58,7 +63,7 @@ public class DfsClientShm extends ShortC
    *
    * {@link DfsClientShm#handle} sets this to true.
    */
-  private boolean stale = false;
+  private boolean disconnected = false;
 
   DfsClientShm(ShmId shmId, FileInputStream stream, EndpointShmManager manager,
       DomainPeer peer) throws IOException {
@@ -76,14 +81,14 @@ public class DfsClientShm extends ShortC
   }
 
   /**
-   * Determine if the shared memory segment is stale.
+   * Determine if the shared memory segment is disconnected from the DataNode.
    *
    * This must be called with the DfsClientShmManager lock held.
    *
    * @return   True if the shared memory segment is stale.
    */
-  public synchronized boolean isStale() {
-    return stale;
+  public synchronized boolean isDisconnected() {
+    return disconnected;
   }
 
   /**
@@ -97,8 +102,8 @@ public class DfsClientShm extends ShortC
   public boolean handle(DomainSocket sock) {
     manager.unregisterShm(getShmId());
     synchronized (this) {
-      Preconditions.checkState(!stale);
-      stale = true;
+      Preconditions.checkState(!disconnected);
+      disconnected = true;
       boolean hadSlots = false;
       for (Iterator<Slot> iter = slotIterator(); iter.hasNext(); ) {
         Slot slot = iter.next();

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java Tue Jul 29 00:49:14 2014
@@ -271,12 +271,12 @@ public class DfsClientShmManager impleme
             loading = false;
             finishedLoading.signalAll();
           }
-          if (shm.isStale()) {
+          if (shm.isDisconnected()) {
             // If the peer closed immediately after the shared memory segment
             // was created, the DomainSocketWatcher callback might already have
-            // fired and marked the shm as stale.  In this case, we obviously
-            // don't want to add the SharedMemorySegment to our list of valid
-            // not-full segments.
+            // fired and marked the shm as disconnected.  In this case, we
+            // obviously don't want to add the SharedMemorySegment to our list
+            // of valid not-full segments.
             if (LOG.isDebugEnabled()) {
               LOG.debug(this + ": the UNIX domain socket associated with " +
                   "this short-circuit memory closed before we could make " +
@@ -299,7 +299,7 @@ public class DfsClientShmManager impleme
     void freeSlot(Slot slot) {
       DfsClientShm shm = (DfsClientShm)slot.getShm();
       shm.unregisterSlot(slot.getSlotIdx());
-      if (shm.isStale()) {
+      if (shm.isDisconnected()) {
         // Stale shared memory segments should not be tracked here.
         Preconditions.checkState(!full.containsKey(shm.getShmId()));
         Preconditions.checkState(!notFull.containsKey(shm.getShmId()));

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java Tue Jul 29 00:49:14 2014
@@ -306,6 +306,13 @@ public class ShortCircuitShm {
           (slotAddress - baseAddress) / BYTES_PER_SLOT);
     }
 
+    /**
+     * Clear the slot.
+     */
+    void clear() {
+      unsafe.putLongVolatile(null, this.slotAddress, 0);
+    }
+
     private boolean isSet(long flag) {
       long prev = unsafe.getLongVolatile(null, this.slotAddress);
       return (prev & flag) != 0;
@@ -535,6 +542,7 @@ public class ShortCircuitShm {
     }
     allocatedSlots.set(idx, true);
     Slot slot = new Slot(calculateSlotAddress(idx), blockId);
+    slot.clear();
     slot.makeValid();
     slots[idx] = slot;
     if (LOG.isTraceEnabled()) {
@@ -583,7 +591,7 @@ public class ShortCircuitShm {
     Slot slot = new Slot(calculateSlotAddress(slotIdx), blockId);
     if (!slot.isValid()) {
       throw new InvalidRequestException(this + ": slot " + slotIdx +
-          " has not been allocated.");
+          " is not marked as valid.");
     }
     slots[slotIdx] = slot;
     allocatedSlots.set(slotIdx, true);

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrNameParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrNameParam.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrNameParam.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrNameParam.java Tue Jul 29 00:49:14 2014
@@ -25,8 +25,8 @@ public class XAttrNameParam extends Stri
   /** Default parameter value. **/
   public static final String DEFAULT = "";
   
-  private static Domain DOMAIN = new Domain(NAME, 
-      Pattern.compile("^(user\\.|trusted\\.|system\\.|security\\.).+"));
+  private static Domain DOMAIN = new Domain(NAME,
+      Pattern.compile(".*"));
   
   public XAttrNameParam(final String str) {
     super(DOMAIN, str == null || str.equals(DEFAULT) ? null : str);

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto Tue Jul 29 00:49:14 2014
@@ -281,6 +281,19 @@ message GetDatanodeReportResponseProto {
   repeated DatanodeInfoProto di = 1;
 }
 
+message GetDatanodeStorageReportRequestProto {
+  required DatanodeReportTypeProto type = 1;
+}
+
+message DatanodeStorageReportProto {
+  required DatanodeInfoProto datanodeInfo = 1;
+  repeated StorageReportProto storageReports = 2;
+}
+
+message GetDatanodeStorageReportResponseProto {
+  repeated DatanodeStorageReportProto datanodeStorageReports = 1;
+}
+
 message GetPreferredBlockSizeRequestProto {
   required string filename = 1;
 }
@@ -672,6 +685,8 @@ service ClientNamenodeProtocol {
   rpc getFsStats(GetFsStatusRequestProto) returns(GetFsStatsResponseProto);
   rpc getDatanodeReport(GetDatanodeReportRequestProto)
       returns(GetDatanodeReportResponseProto);
+  rpc getDatanodeStorageReport(GetDatanodeStorageReportRequestProto)
+      returns(GetDatanodeStorageReportResponseProto);
   rpc getPreferredBlockSize(GetPreferredBlockSizeRequestProto)
       returns(GetPreferredBlockSizeResponseProto);
   rpc setSafeMode(SetSafeModeRequestProto)

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto Tue Jul 29 00:49:14 2014
@@ -45,20 +45,6 @@ message DatanodeRegistrationProto {
 }
 
 /**
- * Represents a storage available on the datanode
- */
-message DatanodeStorageProto {
-  enum StorageState {
-    NORMAL = 0;
-    READ_ONLY_SHARED = 1;
-  }
-
-  required string storageUuid = 1;
-  optional StorageState state = 2 [default = NORMAL];
-  optional StorageTypeProto storageType = 3 [default = DISK];
-}
-
-/**
  * Commands sent from namenode to the datanodes
  */
 message DatanodeCommandProto {
@@ -196,16 +182,6 @@ message HeartbeatRequestProto {
   optional uint64 cacheUsed = 7 [default = 0 ];
 }
 
-message StorageReportProto {
-  required string storageUuid = 1 [ deprecated = true ];
-  optional bool failed = 2 [ default = false ];
-  optional uint64 capacity = 3 [ default = 0 ];
-  optional uint64 dfsUsed = 4 [ default = 0 ];
-  optional uint64 remaining = 5 [ default = 0 ];
-  optional uint64 blockPoolUsed = 6 [ default = 0 ];
-  optional DatanodeStorageProto storage = 7; // supersedes StorageUuid
-}
-
 /**
  * state - State the NN is in when returning response to the DN
  * txid - Highest transaction ID this NN has seen

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto Tue Jul 29 00:49:14 2014
@@ -100,6 +100,30 @@ message DatanodeInfoProto {
 }
 
 /**
+ * Represents a storage available on the datanode
+ */
+message DatanodeStorageProto {
+  enum StorageState {
+    NORMAL = 0;
+    READ_ONLY_SHARED = 1;
+  }
+
+  required string storageUuid = 1;
+  optional StorageState state = 2 [default = NORMAL];
+  optional StorageTypeProto storageType = 3 [default = DISK];
+}
+
+message StorageReportProto {
+  required string storageUuid = 1 [ deprecated = true ];
+  optional bool failed = 2 [ default = false ];
+  optional uint64 capacity = 3 [ default = 0 ];
+  optional uint64 dfsUsed = 4 [ default = 0 ];
+  optional uint64 remaining = 5 [ default = 0 ];
+  optional uint64 blockPoolUsed = 6 [ default = 0 ];
+  optional DatanodeStorageProto storage = 7; // supersedes StorageUuid
+}
+
+/**
  * Summary of a file or directory
  */
 message ContentSummaryProto {

Propchange: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs:r1611735-1614231

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html Tue Jul 29 00:49:14 2014
@@ -66,7 +66,6 @@
 <div class="row">
   <hr />
   <div class="col-xs-2"><p>Hadoop, 2014.</p></div>
-  <div class="col-xs-1 pull-right"><a style="color: #ddd" href="dfshealth.jsp">Legacy UI</a></div>
 </div>
 </div>
 
@@ -283,7 +282,7 @@
   <tr class="danger">
     <td>{name} ({xferaddr})</td>
     <td>{lastContact}</td>
-    <td>Dead{?decomissioned}, Decomissioned{/decomissioned}</td>
+    <td>Dead{?decommissioned}, Decommissioned{/decommissioned}</td>
     <td>-</td>
     <td>-</td>
     <td>-</td>

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/index.html
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/index.html?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/index.html (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/index.html Tue Jul 29 00:49:14 2014
@@ -18,18 +18,7 @@
     "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
 <html xmlns="http://www.w3.org/1999/xhtml">
 <head>
-<meta http-equiv="REFRESH" content="1;url=dfshealth.jsp" />
+<meta http-equiv="REFRESH" content="0;url=dfshealth.html" />
 <title>Hadoop Administration</title>
 </head>
-<body>
-<script type="text/javascript">
-//<![CDATA[
-window.location.href='dfshealth.html';
-//]]>
-</script>
-<h1>Hadoop Administration</h1>
-<ul>
-<li><a href="dfshealth.jsp">DFS Health/Status</a></li>
-</ul>
-</body>
 </html>

Propchange: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary:r1609845-1614231

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/index.html
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/index.html?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/index.html (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/index.html Tue Jul 29 00:49:14 2014
@@ -21,15 +21,4 @@
   <meta http-equiv="REFRESH" content="0;url=status.html" />
   <title>Hadoop Administration</title>
 </head>
-<body>
-<script type="text/javascript">
-//<![CDATA[
-window.location.href='status.html';
-//]]>
-</script>
-<h1>Hadoop Administration</h1>
-<ul>
-  <li><a href="status.jsp">Status</a></li>
-</ul>
-</body>
 </html>
\ No newline at end of file

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm Tue Jul 29 00:49:14 2014
@@ -44,10 +44,13 @@ HDFS NFS Gateway
 
 * {Configuration}
 
-   The user running the NFS-gateway must be able to proxy all the users using the NFS mounts. 
-   For instance, if user 'nfsserver' is running the gateway, and users belonging to the groups 'nfs-users1'
-   and 'nfs-users2' use the NFS mounts, then in core-site.xml of the namenode, the following must be set
-   (NOTE: replace 'nfsserver' with the user name starting the gateway in your cluster):
+   The NFS-gateway uses proxy user to proxy all the users accessing the NFS mounts. 
+   In non-secure mode, the user running the gateway is the proxy user, while in secure mode the
+   user in Kerberos keytab is the proxy user. Suppose the proxy user is 'nfsserver'
+   and users belonging to the groups 'nfs-users1'
+   and 'nfs-users2' use the NFS mounts, then in core-site.xml of the NameNode, the following
+   two properities must be set and only NameNode needs restart after the configuration change
+   (NOTE: replace the string 'nfsserver' with the proxy user name in your cluster):
 
 ----
 <property>
@@ -72,7 +75,9 @@ HDFS NFS Gateway
 ----
 
    The above are the only required configuration for the NFS gateway in non-secure mode. For Kerberized
-   hadoop clusters, the following configurations need to be added to hdfs-site.xml:
+   hadoop clusters, the following configurations need to be added to hdfs-site.xml for the gateway (NOTE: replace 
+   string "nfsserver" with the proxy user name and ensure the user contained in the keytab is
+   also the same proxy user):
 
 ----
   <property>
@@ -87,6 +92,8 @@ HDFS NFS Gateway
     <value>nfsserver/_HOST@YOUR-REALM.COM</value>
   </property>
 ----
+  
+   The rest of the NFS gateway configurations are optional for both secure and non-secure mode.
 
    The AIX NFS client has a {{{https://issues.apache.org/jira/browse/HDFS-6549}few known issues}}
    that prevent it from working correctly by default with the HDFS NFS
@@ -108,7 +115,7 @@ HDFS NFS Gateway
    have been committed.
 
    It's strongly recommended for the users to update a few configuration properties based on their use
-   cases. All the related configuration properties can be added or updated in hdfs-site.xml.
+   cases. All the following configuration properties can be added or updated in hdfs-site.xml.
   
    * If the client mounts the export with access time update allowed, make sure the following 
     property is not disabled in the configuration file. Only NameNode needs to restart after 
@@ -145,36 +152,6 @@ HDFS NFS Gateway
   </property>
 ---- 
 
-   * For optimal performance, it is recommended that rtmax be updated to
-     1MB. However, note that this 1MB is a per client allocation, and not
-     from a shared memory pool, and therefore a larger value may adversely 
-     affect small reads, consuming a lot of memory. The maximum value of 
-     this property is 1MB.
-
-----
-<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>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
-    also update the nfs mount's wsize(add wsize= # of bytes to the 
-    mount directive).
-  </description>
-</property>
-----
-
   * By default, the export can be mounted by any client. To better control the access,
     users can update the following property. The value string contains machine name and
     access privilege, separated by whitespace
@@ -238,8 +215,10 @@ HDFS NFS Gateway
 
    [[3]] Start mountd and nfsd.
    
-     No root privileges are required for this command. However, ensure that the user starting
-     the Hadoop cluster and the user starting the NFS gateway are same.
+     No root privileges are required for this command. In non-secure mode, the NFS gateway
+     should be started by the proxy user mentioned at the beginning of this user guide. 
+     While in secure mode, any user can start NFS gateway 
+     as long as the user has read access to the Kerberos keytab defined in "nfs.keytab.file".
 
 -------------------------
      hadoop nfs3
@@ -339,7 +318,10 @@ HDFS NFS Gateway
 -------------------------------------------------------------------
 
   Then the users can access HDFS as part of the local file system except that, 
-  hard link and random write are not supported yet.
+  hard link and random write are not supported yet. To optimize the performance
+  of large file I/O, one can increase the NFS transfer size(rsize and wsize) during mount.
+  By default, NFS gateway supports 1MB as the maximum transfer size. For larger data
+  transfer size, one needs to update "nfs.rtmax" and "nfs.rtmax" in hdfs-site.xml.
 
 * {Allow mounts from unprivileged clients}
 

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java Tue Jul 29 00:49:14 2014
@@ -2653,6 +2653,75 @@ public class TestDFSShell {
     }
   }
 
+  /*
+   * 1. Test that CLI throws an exception and returns non-0 when user does
+   * not have permission to read an xattr.
+   * 2. Test that CLI throws an exception and returns non-0 when a non-existent
+   * xattr is requested.
+   */
+  @Test (timeout = 120000)
+  public void testGetFAttrErrors() throws Exception {
+    final UserGroupInformation user = UserGroupInformation.
+        createUserForTesting("user", new String[] {"mygroup"});
+    MiniDFSCluster cluster = null;
+    PrintStream bakErr = null;
+    try {
+      final Configuration conf = new HdfsConfiguration();
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      cluster.waitActive();
+
+      final FileSystem fs = cluster.getFileSystem();
+      final Path p = new Path("/foo");
+      fs.mkdirs(p);
+      bakErr = System.err;
+
+      final FsShell fshell = new FsShell(conf);
+      final ByteArrayOutputStream out = new ByteArrayOutputStream();
+      System.setErr(new PrintStream(out));
+
+      // No permission for "other".
+      fs.setPermission(p, new FsPermission((short) 0700));
+
+      {
+        final int ret = ToolRunner.run(fshell, new String[] {
+            "-setfattr", "-n", "user.a1", "-v", "1234", "/foo"});
+        assertEquals("Returned should be 0", 0, ret);
+        out.reset();
+      }
+
+      user.doAs(new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            int ret = ToolRunner.run(fshell, new String[] {
+                "-getfattr", "-n", "user.a1", "/foo"});
+            String str = out.toString();
+            assertTrue("xattr value was incorrectly returned",
+                str.indexOf("1234") == -1);
+            out.reset();
+            return null;
+          }
+        });
+
+      {
+        final int ret = ToolRunner.run(fshell, new String[]{
+            "-getfattr", "-n", "user.nonexistent", "/foo"});
+        String str = out.toString();
+        assertTrue("xattr value was incorrectly returned",
+          str.indexOf(
+              "getfattr: At least one of the attributes provided was not found")
+               >= 0);
+        out.reset();
+      }
+    } finally {
+      if (bakErr != null) {
+        System.setErr(bakErr);
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
   /**
    * Test that the server trash configuration is respected when
    * the client configuration is not set.

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java Tue Jul 29 00:49:14 2014
@@ -70,6 +70,9 @@ public class TestDFSUpgradeFromImage {
   private static final String HADOOP_DFS_DIR_TXT = "hadoop-dfs-dir.txt";
   private static final String HADOOP22_IMAGE = "hadoop-22-dfs-dir.tgz";
   private static final String HADOOP1_BBW_IMAGE = "hadoop1-bbw.tgz";
+  private static final String HADOOP1_RESERVED_IMAGE = "hadoop-1-reserved.tgz";
+  private static final String HADOOP023_RESERVED_IMAGE =
+      "hadoop-0.23-reserved.tgz";
   private static final String HADOOP2_RESERVED_IMAGE = "hadoop-2-reserved.tgz";
 
   private static class ReferenceFileInfo {
@@ -326,6 +329,140 @@ public class TestDFSUpgradeFromImage {
   }
 
   /**
+   * Test upgrade from a branch-1.2 image with reserved paths
+   */
+  @Test
+  public void testUpgradeFromRel1ReservedImage() throws Exception {
+    unpackStorage(HADOOP1_RESERVED_IMAGE);
+    MiniDFSCluster cluster = null;
+    // Try it once without setting the upgrade flag to ensure it fails
+    final Configuration conf = new Configuration();
+    // Try it again with a custom rename string
+    try {
+      FSImageFormat.setRenameReservedPairs(
+          ".snapshot=.user-snapshot," +
+              ".reserved=.my-reserved");
+      cluster =
+          new MiniDFSCluster.Builder(conf)
+              .format(false)
+              .startupOption(StartupOption.UPGRADE)
+              .numDataNodes(0).build();
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      // Make sure the paths were renamed as expected
+      // Also check that paths are present after a restart, checks that the
+      // upgraded fsimage has the same state.
+      final String[] expected = new String[] {
+          "/.my-reserved",
+          "/.user-snapshot",
+          "/.user-snapshot/.user-snapshot",
+          "/.user-snapshot/open",
+          "/dir1",
+          "/dir1/.user-snapshot",
+          "/dir2",
+          "/dir2/.user-snapshot",
+          "/user",
+          "/user/andrew",
+          "/user/andrew/.user-snapshot",
+      };
+      for (int i=0; i<2; i++) {
+        // Restart the second time through this loop
+        if (i==1) {
+          cluster.finalizeCluster(conf);
+          cluster.restartNameNode(true);
+        }
+        ArrayList<Path> toList = new ArrayList<Path>();
+        toList.add(new Path("/"));
+        ArrayList<String> found = new ArrayList<String>();
+        while (!toList.isEmpty()) {
+          Path p = toList.remove(0);
+          FileStatus[] statuses = dfs.listStatus(p);
+          for (FileStatus status: statuses) {
+            final String path = status.getPath().toUri().getPath();
+            System.out.println("Found path " + path);
+            found.add(path);
+            if (status.isDirectory()) {
+              toList.add(status.getPath());
+            }
+          }
+        }
+        for (String s: expected) {
+          assertTrue("Did not find expected path " + s, found.contains(s));
+        }
+        assertEquals("Found an unexpected path while listing filesystem",
+            found.size(), expected.length);
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Test upgrade from a 0.23.11 image with reserved paths
+   */
+  @Test
+  public void testUpgradeFromRel023ReservedImage() throws Exception {
+    unpackStorage(HADOOP023_RESERVED_IMAGE);
+    MiniDFSCluster cluster = null;
+    // Try it once without setting the upgrade flag to ensure it fails
+    final Configuration conf = new Configuration();
+    // Try it again with a custom rename string
+    try {
+      FSImageFormat.setRenameReservedPairs(
+          ".snapshot=.user-snapshot," +
+              ".reserved=.my-reserved");
+      cluster =
+          new MiniDFSCluster.Builder(conf)
+              .format(false)
+              .startupOption(StartupOption.UPGRADE)
+              .numDataNodes(0).build();
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      // Make sure the paths were renamed as expected
+      // Also check that paths are present after a restart, checks that the
+      // upgraded fsimage has the same state.
+      final String[] expected = new String[] {
+          "/.user-snapshot",
+          "/dir1",
+          "/dir1/.user-snapshot",
+          "/dir2",
+          "/dir2/.user-snapshot"
+      };
+      for (int i=0; i<2; i++) {
+        // Restart the second time through this loop
+        if (i==1) {
+          cluster.finalizeCluster(conf);
+          cluster.restartNameNode(true);
+        }
+        ArrayList<Path> toList = new ArrayList<Path>();
+        toList.add(new Path("/"));
+        ArrayList<String> found = new ArrayList<String>();
+        while (!toList.isEmpty()) {
+          Path p = toList.remove(0);
+          FileStatus[] statuses = dfs.listStatus(p);
+          for (FileStatus status: statuses) {
+            final String path = status.getPath().toUri().getPath();
+            System.out.println("Found path " + path);
+            found.add(path);
+            if (status.isDirectory()) {
+              toList.add(status.getPath());
+            }
+          }
+        }
+        for (String s: expected) {
+          assertTrue("Did not find expected path " + s, found.contains(s));
+        }
+        assertEquals("Found an unexpected path while listing filesystem",
+            found.size(), expected.length);
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  /**
    * Test upgrade from 2.0 image with a variety of .snapshot and .reserved
    * paths to test renaming on upgrade
    */

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java Tue Jul 29 00:49:14 2014
@@ -53,6 +53,8 @@ public class TestDatanodeConfig {
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_DATANODE_HTTPS_PORT_KEY, 0);
     conf.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, "localhost:0");
+    conf.set(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY, "localhost:0");
+    conf.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, "localhost:0");
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
     cluster.waitActive();
   }

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeReport.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeReport.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeReport.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeReport.java Tue Jul 29 00:49:14 2014
@@ -21,19 +21,26 @@ import static org.apache.hadoop.test.Met
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
 import static org.junit.Assert.assertEquals;
 
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+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.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.junit.Test;
 
 /**
  * This test ensures the all types of data node report work correctly.
  */
 public class TestDatanodeReport {
+  static final Log LOG = LogFactory.getLog(TestDatanodeReport.class);
   final static private Configuration conf = new HdfsConfiguration();
   final static private int NUM_OF_DATANODES = 4;
     
@@ -50,20 +57,18 @@ public class TestDatanodeReport {
     try {
       //wait until the cluster is up
       cluster.waitActive();
-
-      InetSocketAddress addr = new InetSocketAddress("localhost",
-          cluster.getNameNodePort());
-      DFSClient client = new DFSClient(addr, conf);
-
-      assertEquals(client.datanodeReport(DatanodeReportType.ALL).length,
-                   NUM_OF_DATANODES);
-      assertEquals(client.datanodeReport(DatanodeReportType.LIVE).length,
-                   NUM_OF_DATANODES);
-      assertEquals(client.datanodeReport(DatanodeReportType.DEAD).length, 0);
+      final String bpid = cluster.getNamesystem().getBlockPoolId();
+      final List<DataNode> datanodes = cluster.getDataNodes();
+      final DFSClient client = cluster.getFileSystem().dfs;
+
+      assertReports(NUM_OF_DATANODES, DatanodeReportType.ALL, client, datanodes, bpid);
+      assertReports(NUM_OF_DATANODES, DatanodeReportType.LIVE, client, datanodes, bpid);
+      assertReports(0, DatanodeReportType.DEAD, client, datanodes, bpid);
 
       // bring down one datanode
-      ArrayList<DataNode> datanodes = cluster.getDataNodes();
-      datanodes.remove(datanodes.size()-1).shutdown();
+      final DataNode last = datanodes.get(datanodes.size() - 1);
+      LOG.info("XXX shutdown datanode " + last.getDatanodeUuid());
+      last.shutdown();
 
       DatanodeInfo[] nodeInfo = client.datanodeReport(DatanodeReportType.DEAD);
       while (nodeInfo.length != 1) {
@@ -74,22 +79,59 @@ public class TestDatanodeReport {
         nodeInfo = client.datanodeReport(DatanodeReportType.DEAD);
       }
 
-      assertEquals(client.datanodeReport(DatanodeReportType.LIVE).length,
-                   NUM_OF_DATANODES-1);
-      assertEquals(client.datanodeReport(DatanodeReportType.ALL).length,
-                   NUM_OF_DATANODES);
+      assertReports(NUM_OF_DATANODES, DatanodeReportType.ALL, client, datanodes, null);
+      assertReports(NUM_OF_DATANODES - 1, DatanodeReportType.LIVE, client, datanodes, null);
+      assertReports(1, DatanodeReportType.DEAD, client, datanodes, null);
 
       Thread.sleep(5000);
       assertGauge("ExpiredHeartbeats", 1, getMetrics("FSNamesystem"));
-    }finally {
+    } finally {
       cluster.shutdown();
     }
   }
- 
-  public static void main(String[] args) throws Exception {
-    new TestDatanodeReport().testDatanodeReport();
-  }
   
-}
-
+  final static Comparator<StorageReport> CMP = new Comparator<StorageReport>() {
+    @Override
+    public int compare(StorageReport left, StorageReport right) {
+      return left.getStorage().getStorageID().compareTo(
+            right.getStorage().getStorageID());
+    }
+  };
 
+  static void assertReports(int numDatanodes, DatanodeReportType type,
+      DFSClient client, List<DataNode> datanodes, String bpid) throws IOException {
+    final DatanodeInfo[] infos = client.datanodeReport(type);
+    assertEquals(numDatanodes, infos.length);
+    final DatanodeStorageReport[] reports = client.getDatanodeStorageReport(type);
+    assertEquals(numDatanodes, reports.length);
+    
+    for(int i = 0; i < infos.length; i++) {
+      assertEquals(infos[i], reports[i].getDatanodeInfo());
+      
+      final DataNode d = findDatanode(infos[i].getDatanodeUuid(), datanodes);
+      if (bpid != null) {
+        //check storage
+        final StorageReport[] computed = reports[i].getStorageReports();
+        Arrays.sort(computed, CMP);
+        final StorageReport[] expected = d.getFSDataset().getStorageReports(bpid);
+        Arrays.sort(expected, CMP);
+  
+        assertEquals(expected.length, computed.length);
+        for(int j = 0; j < expected.length; j++) {
+          assertEquals(expected[j].getStorage().getStorageID(),
+                       computed[j].getStorage().getStorageID());
+        }
+      }
+    }
+  }
+  
+  static DataNode findDatanode(String id, List<DataNode> datanodes) {
+    for(DataNode d : datanodes) {
+      if (d.getDatanodeUuid().equals(id)) {
+        return d;
+      }
+    }
+    throw new IllegalStateException("Datnode " + id + " not in datanode list: "
+        + datanodes);
+  }
+}
\ No newline at end of file

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java Tue Jul 29 00:49:14 2014
@@ -31,25 +31,25 @@ import org.apache.hadoop.fs.permission.A
 import org.apache.hadoop.fs.permission.AclEntryType;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeStorageProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
@@ -67,9 +67,18 @@ import org.apache.hadoop.hdfs.server.com
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
-import org.apache.hadoop.hdfs.server.protocol.*;
+import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSXAttrBaseTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSXAttrBaseTest.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSXAttrBaseTest.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSXAttrBaseTest.java Tue Jul 29 00:49:14 2014
@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.na
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.FileNotFoundException;
 import java.security.PrivilegedExceptionAction;
 import java.util.EnumSet;
 import java.util.List;
@@ -46,6 +45,7 @@ import static org.apache.hadoop.fs.permi
 import static org.apache.hadoop.fs.permission.FsAction.ALL;
 import static org.apache.hadoop.fs.permission.FsAction.READ;
 import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import org.junit.After;
@@ -261,11 +261,12 @@ public class FSXAttrBaseTest {
       fs.setXAttr(path, "user.", value1, EnumSet.of(XAttrSetFlag.CREATE, 
           XAttrSetFlag.REPLACE));
       Assert.fail("Setting xattr with empty name should fail.");
+    } catch (RemoteException e) {
+      assertEquals("Unexpected RemoteException: " + e, e.getClassName(),
+          HadoopIllegalArgumentException.class.getCanonicalName());
+      GenericTestUtils.assertExceptionContains("XAttr name cannot be empty", e);
     } catch (HadoopIllegalArgumentException e) {
       GenericTestUtils.assertExceptionContains("XAttr name cannot be empty", e);
-    } catch (IllegalArgumentException e) {
-      GenericTestUtils.assertExceptionContains("Invalid value: \"user.\" does " + 
-          "not belong to the domain ^(user\\.|trusted\\.|system\\.|security\\.).+", e);
     }
     
     // Set xattr with invalid name: "a1"
@@ -274,11 +275,12 @@ public class FSXAttrBaseTest {
           XAttrSetFlag.REPLACE));
       Assert.fail("Setting xattr with invalid name prefix or without " +
           "name prefix should fail.");
+    } catch (RemoteException e) {
+      assertEquals("Unexpected RemoteException: " + e, e.getClassName(),
+          HadoopIllegalArgumentException.class.getCanonicalName());
+      GenericTestUtils.assertExceptionContains("XAttr name must be prefixed", e);
     } catch (HadoopIllegalArgumentException e) {
       GenericTestUtils.assertExceptionContains("XAttr name must be prefixed", e);
-    } catch (IllegalArgumentException e) {
-      GenericTestUtils.assertExceptionContains("Invalid value: \"a1\" does " + 
-          "not belong to the domain ^(user\\.|trusted\\.|system\\.|security\\.).+", e);
     }
     
     // Set xattr without XAttrSetFlag
@@ -341,9 +343,18 @@ public class FSXAttrBaseTest {
   }
   
   /**
-   * Tests for getting xattr
-   * 1. To get xattr which does not exist.
-   * 2. To get multiple xattrs.
+   * getxattr tests. Test that getxattr throws an exception if any of
+   * the following are true:
+   * an xattr that was requested doesn't exist
+   * the caller specifies an unknown namespace
+   * the caller doesn't have access to the namespace
+   * the caller doesn't have permission to get the value of the xattr
+   * the caller does not have search access to the parent directory
+   * the caller has only read access to the owning directory
+   * the caller has only search access to the owning directory and
+   * execute/search access to the actual entity
+   * the caller does not have search access to the owning directory and read
+   * access to the actual entity
    */
   @Test(timeout = 120000)
   public void testGetXAttrs() throws Exception {
@@ -351,21 +362,159 @@ public class FSXAttrBaseTest {
     fs.setXAttr(path, name1, value1, EnumSet.of(XAttrSetFlag.CREATE));
     fs.setXAttr(path, name2, value2, EnumSet.of(XAttrSetFlag.CREATE));
     
-    // XAttr does not exist.
-    byte[] value = fs.getXAttr(path, name3);
-    Assert.assertEquals(value, null);
-    
-    List<String> names = Lists.newArrayList();
-    names.add(name1);
-    names.add(name2);
-    names.add(name3);
-    Map<String, byte[]> xattrs = fs.getXAttrs(path, names);
-    Assert.assertEquals(xattrs.size(), 2);
-    Assert.assertArrayEquals(value1, xattrs.get(name1));
-    Assert.assertArrayEquals(value2, xattrs.get(name2));
+    /* An XAttr that was requested does not exist. */
+    try {
+      final byte[] value = fs.getXAttr(path, name3);
+      Assert.fail("expected IOException");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains(
+          "At least one of the attributes provided was not found.", e);
+    }
+    
+    /* Throw an exception if an xattr that was requested does not exist. */
+    {
+      final List<String> names = Lists.newArrayList();
+      names.add(name1);
+      names.add(name2);
+      names.add(name3);
+      try {
+        final Map<String, byte[]> xattrs = fs.getXAttrs(path, names);
+        Assert.fail("expected IOException");
+      } catch (IOException e) {
+        GenericTestUtils.assertExceptionContains(
+            "At least one of the attributes provided was not found.", e);
+      }
+    }
     
     fs.removeXAttr(path, name1);
     fs.removeXAttr(path, name2);
+
+    /* Unknown namespace should throw an exception. */
+    try {
+      final byte[] xattr = fs.getXAttr(path, "wackynamespace.foo");
+      Assert.fail("expected IOException");
+    } catch (Exception e) {
+      GenericTestUtils.assertExceptionContains
+          ("An XAttr name must be prefixed with user/trusted/security/system, " +
+           "followed by a '.'",
+          e);
+    }
+
+    /*
+     * The 'trusted' namespace should not be accessible and should throw an
+     * exception.
+     */
+    final UserGroupInformation user = UserGroupInformation.
+        createUserForTesting("user", new String[] {"mygroup"});
+    fs.setXAttr(path, "trusted.foo", "1234".getBytes());
+    try {
+      user.doAs(new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            final FileSystem userFs = dfsCluster.getFileSystem();
+            final byte[] xattr = userFs.getXAttr(path, "trusted.foo");
+            return null;
+          }
+        });
+      Assert.fail("expected IOException");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("User doesn't have permission", e);
+    }
+
+    fs.setXAttr(path, name1, "1234".getBytes());
+
+    /*
+     * Test that an exception is thrown if the caller doesn't have permission to
+     * get the value of the xattr.
+     */
+
+    /* Set access so that only the owner has access. */
+    fs.setPermission(path, new FsPermission((short) 0700));
+    try {
+      user.doAs(new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            final FileSystem userFs = dfsCluster.getFileSystem();
+            final byte[] xattr = userFs.getXAttr(path, name1);
+            return null;
+          }
+        });
+      Assert.fail("expected IOException");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("Permission denied", e);
+    }
+
+    /*
+     * The caller must have search access to the parent directory.
+     */
+    final Path childDir = new Path(path, "child" + pathCount);
+    /* Set access to parent so that only the owner has access. */
+    FileSystem.mkdirs(fs, childDir, FsPermission.createImmutable((short)0700));
+    fs.setXAttr(childDir, name1, "1234".getBytes());
+    try {
+      user.doAs(new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            final FileSystem userFs = dfsCluster.getFileSystem();
+            final byte[] xattr = userFs.getXAttr(childDir, name1);
+            return null;
+          }
+        });
+      Assert.fail("expected IOException");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("Permission denied", e);
+    }
+
+    /* Check that read access to the owning directory is not good enough. */
+    fs.setPermission(path, new FsPermission((short) 0704));
+    try {
+      user.doAs(new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            final FileSystem userFs = dfsCluster.getFileSystem();
+            final byte[] xattr = userFs.getXAttr(childDir, name1);
+            return null;
+          }
+        });
+      Assert.fail("expected IOException");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("Permission denied", e);
+    }
+
+    /*
+     * Check that search access to the owning directory and search/execute
+     * access to the actual entity with extended attributes is not good enough.
+     */
+    fs.setPermission(path, new FsPermission((short) 0701));
+    fs.setPermission(childDir, new FsPermission((short) 0701));
+    try {
+      user.doAs(new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            final FileSystem userFs = dfsCluster.getFileSystem();
+            final byte[] xattr = userFs.getXAttr(childDir, name1);
+            return null;
+          }
+        });
+      Assert.fail("expected IOException");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("Permission denied", e);
+    }
+
+    /*
+     * Check that search access to the owning directory and read access to
+     * the actual entity with the extended attribute is good enough.
+     */
+    fs.setPermission(path, new FsPermission((short) 0701));
+    fs.setPermission(childDir, new FsPermission((short) 0704));
+    user.doAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public Object run() throws Exception {
+          final FileSystem userFs = dfsCluster.getFileSystem();
+          final byte[] xattr = userFs.getXAttr(childDir, name1);
+          return null;
+        }
+      });
   }
   
   /**
@@ -402,6 +551,166 @@ public class FSXAttrBaseTest {
     fs.removeXAttr(path, name3);
   }
 
+  /**
+   * removexattr tests. Test that removexattr throws an exception if any of
+   * the following are true:
+   * an xattr that was requested doesn't exist
+   * the caller specifies an unknown namespace
+   * the caller doesn't have access to the namespace
+   * the caller doesn't have permission to get the value of the xattr
+   * the caller does not have "execute" (scan) access to the parent directory
+   * the caller has only read access to the owning directory
+   * the caller has only execute access to the owning directory and execute
+   * access to the actual entity
+   * the caller does not have execute access to the owning directory and write
+   * access to the actual entity
+   */
+  @Test(timeout = 120000)
+  public void testRemoveXAttrPermissions() throws Exception {
+    FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));
+    fs.setXAttr(path, name1, value1, EnumSet.of(XAttrSetFlag.CREATE));
+    fs.setXAttr(path, name2, value2, EnumSet.of(XAttrSetFlag.CREATE));
+    fs.setXAttr(path, name3, null, EnumSet.of(XAttrSetFlag.CREATE));
+
+    try {
+      fs.removeXAttr(path, name2);
+      fs.removeXAttr(path, name2);
+      Assert.fail("expected IOException");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("No matching attributes found", e);
+    }
+
+    /* Unknown namespace should throw an exception. */
+    final String expectedExceptionString = "An XAttr name must be prefixed " +
+        "with user/trusted/security/system, followed by a '.'";
+    try {
+      fs.removeXAttr(path, "wackynamespace.foo");
+      Assert.fail("expected IOException");
+    } catch (RemoteException e) {
+      assertEquals("Unexpected RemoteException: " + e, e.getClassName(),
+          HadoopIllegalArgumentException.class.getCanonicalName());
+      GenericTestUtils.assertExceptionContains(expectedExceptionString, e);
+    } catch (HadoopIllegalArgumentException e) {
+      GenericTestUtils.assertExceptionContains(expectedExceptionString, e);
+    }
+
+    /*
+     * The 'trusted' namespace should not be accessible and should throw an
+     * exception.
+     */
+    final UserGroupInformation user = UserGroupInformation.
+        createUserForTesting("user", new String[] {"mygroup"});
+    fs.setXAttr(path, "trusted.foo", "1234".getBytes());
+    try {
+      user.doAs(new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            final FileSystem userFs = dfsCluster.getFileSystem();
+            userFs.removeXAttr(path, "trusted.foo");
+            return null;
+          }
+        });
+      Assert.fail("expected IOException");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("User doesn't have permission", e);
+    } finally {
+      fs.removeXAttr(path, "trusted.foo");
+    }
+
+    /*
+     * Test that an exception is thrown if the caller doesn't have permission to
+     * get the value of the xattr.
+     */
+
+    /* Set access so that only the owner has access. */
+    fs.setPermission(path, new FsPermission((short) 0700));
+    try {
+      user.doAs(new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            final FileSystem userFs = dfsCluster.getFileSystem();
+            userFs.removeXAttr(path, name1);
+            return null;
+          }
+        });
+      Assert.fail("expected IOException");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("Permission denied", e);
+    }
+
+    /*
+     * The caller must have "execute" (scan) access to the parent directory.
+     */
+    final Path childDir = new Path(path, "child" + pathCount);
+    /* Set access to parent so that only the owner has access. */
+    FileSystem.mkdirs(fs, childDir, FsPermission.createImmutable((short)0700));
+    fs.setXAttr(childDir, name1, "1234".getBytes());
+    try {
+      user.doAs(new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            final FileSystem userFs = dfsCluster.getFileSystem();
+            userFs.removeXAttr(childDir, name1);
+            return null;
+          }
+        });
+      Assert.fail("expected IOException");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("Permission denied", e);
+    }
+
+    /* Check that read access to the owning directory is not good enough. */
+    fs.setPermission(path, new FsPermission((short) 0704));
+    try {
+      user.doAs(new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            final FileSystem userFs = dfsCluster.getFileSystem();
+            userFs.removeXAttr(childDir, name1);
+            return null;
+          }
+        });
+      Assert.fail("expected IOException");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("Permission denied", e);
+    }
+
+    /*
+     * Check that execute access to the owning directory and scan access to
+     * the actual entity with extended attributes is not good enough.
+     */
+    fs.setPermission(path, new FsPermission((short) 0701));
+    fs.setPermission(childDir, new FsPermission((short) 0701));
+    try {
+      user.doAs(new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            final FileSystem userFs = dfsCluster.getFileSystem();
+            userFs.removeXAttr(childDir, name1);
+            return null;
+          }
+        });
+      Assert.fail("expected IOException");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("Permission denied", e);
+    }
+
+    /*
+     * Check that execute access to the owning directory and write access to
+     * the actual entity with extended attributes is good enough.
+     */
+    fs.setPermission(path, new FsPermission((short) 0701));
+    fs.setPermission(childDir, new FsPermission((short) 0706));
+    user.doAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public Object run() throws Exception {
+          final FileSystem userFs = dfsCluster.getFileSystem();
+          userFs.removeXAttr(childDir, name1);
+          return null;
+        }
+      });
+  }
+
   @Test(timeout = 120000)
   public void testRenameFileWithXAttr() throws Exception {
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java Tue Jul 29 00:49:14 2014
@@ -546,6 +546,7 @@ public class TestINodeFile {
     Configuration conf = new Configuration();
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY,
         DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT);
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
     MiniDFSCluster cluster = null;
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
@@ -593,6 +594,19 @@ public class TestINodeFile {
       // ClientProtocol#getPreferredBlockSize
       assertEquals(testFileBlockSize,
           nnRpc.getPreferredBlockSize(testFileInodePath.toString()));
+
+      /*
+       * HDFS-6749 added missing calls to FSDirectory.resolvePath in the
+       * following four methods. The calls below ensure that
+       * /.reserved/.inodes paths work properly. No need to check return
+       * values as these methods are tested elsewhere.
+       */
+      {
+        fs.isFileClosed(testFileInodePath);
+        fs.getAclStatus(testFileInodePath);
+        fs.getXAttrs(testFileInodePath);
+        fs.listXAttrs(testFileInodePath);
+      }
       
       // symbolic link related tests
       

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java Tue Jul 29 00:49:14 2014
@@ -212,18 +212,25 @@ public class TestNNStorageRetentionManag
     tc.addImage("/foo1/current/" + getImageFileName(300), false);
     tc.addImage("/foo1/current/" + getImageFileName(400), false);
 
+    // Segments containing txns upto txId 250 are extra and should be purged.
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(1, 100), true);
-    // Without lowering the max segments to retain, we'd retain all segments
-    // going back to txid 150 (300 - 150).
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(101, 175), true);
+    tc.addLog("/foo2/current/" + getInProgressEditsFileName(176) + ".empty",
+        true);
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(176, 200), true);
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(201, 225), true);
+    tc.addLog("/foo2/current/" + getInProgressEditsFileName(226) + ".corrupt",
+        true);
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(226, 240), true);
     // Only retain 2 extra segments. The 301-350 and 351-400 segments are
     // considered required, not extra.
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(241, 275), false);
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(276, 300), false);
+    tc.addLog("/foo2/current/" + getInProgressEditsFileName(301) + ".empty",
+        false);
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(301, 350), false);
+    tc.addLog("/foo2/current/" + getInProgressEditsFileName(351) + ".corrupt",
+        false);
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(351, 400), false);
     tc.addLog("/foo2/current/" + getInProgressEditsFileName(401), false);
     runTest(tc);

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java Tue Jul 29 00:49:14 2014
@@ -415,7 +415,7 @@ public class TestNamenodeRetryCache {
     
     LightWeightCache<CacheEntry, CacheEntry> cacheSet = 
         (LightWeightCache<CacheEntry, CacheEntry>) namesystem.getRetryCache().getCacheSet();
-    assertEquals(22, cacheSet.size());
+    assertEquals(23, cacheSet.size());
     
     Map<CacheEntry, CacheEntry> oldEntries = 
         new HashMap<CacheEntry, CacheEntry>();
@@ -434,7 +434,7 @@ public class TestNamenodeRetryCache {
     assertTrue(namesystem.hasRetryCache());
     cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) namesystem
         .getRetryCache().getCacheSet();
-    assertEquals(22, cacheSet.size());
+    assertEquals(23, cacheSet.size());
     iter = cacheSet.iterator();
     while (iter.hasNext()) {
       CacheEntry entry = iter.next();

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java Tue Jul 29 00:49:14 2014
@@ -160,7 +160,7 @@ public class TestRetryCacheWithHA {
     FSNamesystem fsn0 = cluster.getNamesystem(0);
     LightWeightCache<CacheEntry, CacheEntry> cacheSet = 
         (LightWeightCache<CacheEntry, CacheEntry>) fsn0.getRetryCache().getCacheSet();
-    assertEquals(22, cacheSet.size());
+    assertEquals(23, cacheSet.size());
     
     Map<CacheEntry, CacheEntry> oldEntries = 
         new HashMap<CacheEntry, CacheEntry>();
@@ -181,7 +181,7 @@ public class TestRetryCacheWithHA {
     FSNamesystem fsn1 = cluster.getNamesystem(1);
     cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) fsn1
         .getRetryCache().getCacheSet();
-    assertEquals(22, cacheSet.size());
+    assertEquals(23, cacheSet.size());
     iter = cacheSet.iterator();
     while (iter.hasNext()) {
       CacheEntry entry = iter.next();
@@ -1047,6 +1047,49 @@ public class TestRetryCacheWithHA {
     }
   }
 
+  /** removeXAttr */
+  class RemoveXAttrOp extends AtMostOnceOp {
+    private final String src;
+
+    RemoveXAttrOp(DFSClient client, String src) {
+      super("removeXAttr", client);
+      this.src = src;
+    }
+
+    @Override
+    void prepare() throws Exception {
+      Path p = new Path(src);
+      if (!dfs.exists(p)) {
+        DFSTestUtil.createFile(dfs, p, BlockSize, DataNodes, 0);
+        client.setXAttr(src, "user.key", "value".getBytes(),
+          EnumSet.of(XAttrSetFlag.CREATE));
+      }
+    }
+
+    @Override
+    void invoke() throws Exception {
+      client.removeXAttr(src, "user.key");
+    }
+
+    @Override
+    boolean checkNamenodeBeforeReturn() throws Exception {
+      for (int i = 0; i < CHECKTIMES; i++) {
+        Map<String, byte[]> iter = dfs.getXAttrs(new Path(src));
+        Set<String> keySet = iter.keySet();
+        if (!keySet.contains("user.key")) {
+          return true;
+        }
+        Thread.sleep(1000);
+      }
+      return false;
+    }
+
+    @Override
+    Object getResult() {
+      return null;
+    }
+  }
+
   @Test (timeout=60000)
   public void testCreateSnapshot() throws Exception {
     final DFSClient client = genClientWithDummyHandler();
@@ -1183,6 +1226,13 @@ public class TestRetryCacheWithHA {
     testClientRetryWithFailover(op);
   }
 
+  @Test (timeout=60000)
+  public void testRemoveXAttr() throws Exception {
+    DFSClient client = genClientWithDummyHandler();
+    AtMostOnceOp op = new RemoveXAttrOp(client, "/removexattr");
+    testClientRetryWithFailover(op);
+  }
+
   /**
    * When NN failover happens, if the client did not receive the response and
    * send a retry request to the other NN, the same response should be recieved

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java Tue Jul 29 00:49:14 2014
@@ -23,6 +23,7 @@ import static org.apache.hadoop.hdfs.DFS
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY;
 import static org.hamcrest.CoreMatchers.equalTo;
 
 import java.io.DataOutputStream;
@@ -30,7 +31,9 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.Map;
 
 import org.apache.commons.lang.mutable.MutableBoolean;
@@ -462,6 +465,7 @@ public class TestShortCircuitCache {
       }
     }, 10, 60000);
     cluster.shutdown();
+    sockDir.close();
   }
 
   @Test(timeout=60000)
@@ -516,4 +520,98 @@ public class TestShortCircuitCache {
     });
     cluster.shutdown();
   }
+
+  /**
+   * Test unlinking a file whose blocks we are caching in the DFSClient.
+   * The DataNode will notify the DFSClient that the replica is stale via the
+   * ShortCircuitShm.
+   */
+  @Test(timeout=60000)
+  public void testUnlinkingReplicasInFileDescriptorCache() throws Exception {
+    BlockReaderTestUtil.enableShortCircuitShmTracing();
+    TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
+    Configuration conf = createShortCircuitConf(
+        "testUnlinkingReplicasInFileDescriptorCache", sockDir);
+    // We don't want the CacheCleaner to time out short-circuit shared memory
+    // segments during the test, so set the timeout really high.
+    conf.setLong(DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
+        1000000000L);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+    DistributedFileSystem fs = cluster.getFileSystem();
+    final ShortCircuitCache cache =
+        fs.getClient().getClientContext().getShortCircuitCache();
+    cache.getDfsClientShmManager().visit(new Visitor() {
+      @Override
+      public void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
+          throws IOException {
+        // The ClientShmManager starts off empty.
+        Assert.assertEquals(0,  info.size());
+      }
+    });
+    final Path TEST_PATH = new Path("/test_file");
+    final int TEST_FILE_LEN = 8193;
+    final int SEED = 0xFADE0;
+    DFSTestUtil.createFile(fs, TEST_PATH, TEST_FILE_LEN,
+        (short)1, SEED);
+    byte contents[] = DFSTestUtil.readFileBuffer(fs, TEST_PATH);
+    byte expected[] = DFSTestUtil.
+        calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
+    Assert.assertTrue(Arrays.equals(contents, expected));
+    // Loading this file brought the ShortCircuitReplica into our local
+    // replica cache.
+    final DatanodeInfo datanode =
+        new DatanodeInfo(cluster.getDataNodes().get(0).getDatanodeId());
+    cache.getDfsClientShmManager().visit(new Visitor() {
+      @Override
+      public void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
+          throws IOException {
+        Assert.assertTrue(info.get(datanode).full.isEmpty());
+        Assert.assertFalse(info.get(datanode).disabled);
+        Assert.assertEquals(1, info.get(datanode).notFull.values().size());
+        DfsClientShm shm =
+            info.get(datanode).notFull.values().iterator().next();
+        Assert.assertFalse(shm.isDisconnected());
+      }
+    });
+    // Remove the file whose blocks we just read.
+    fs.delete(TEST_PATH, false);
+
+    // Wait for the replica to be purged from the DFSClient's cache.
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      MutableBoolean done = new MutableBoolean(true);
+      @Override
+      public Boolean get() {
+        try {
+          done.setValue(true);
+          cache.getDfsClientShmManager().visit(new Visitor() {
+            @Override
+            public void visit(HashMap<DatanodeInfo,
+                  PerDatanodeVisitorInfo> info) throws IOException {
+              Assert.assertTrue(info.get(datanode).full.isEmpty());
+              Assert.assertFalse(info.get(datanode).disabled);
+              Assert.assertEquals(1,
+                  info.get(datanode).notFull.values().size());
+              DfsClientShm shm = info.get(datanode).notFull.values().
+                  iterator().next();
+              // Check that all slots have been invalidated.
+              for (Iterator<Slot> iter = shm.slotIterator();
+                   iter.hasNext(); ) {
+                Slot slot = iter.next();
+                if (slot.isValid()) {
+                  done.setValue(false);
+                }
+              }
+            }
+          });
+        } catch (IOException e) {
+          LOG.error("error running visitor", e);
+        }
+        return done.booleanValue();
+      }
+    }, 10, 60000);
+    cluster.shutdown();
+    sockDir.close();
+  }
 }

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java Tue Jul 29 00:49:14 2014
@@ -39,14 +39,18 @@ import org.apache.hadoop.hdfs.DFSConfigK
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.TestDFSClientRetries;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
-import org.apache.hadoop.hdfs.TestDFSClientRetries;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.ipc.RetriableException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
 
 /** Test WebHDFS */
 public class TestWebHDFS {
@@ -445,4 +449,37 @@ public class TestWebHDFS {
       }
     }
   }
+
+  /**
+   * Make sure a RetriableException is thrown when rpcServer is null in
+   * NamenodeWebHdfsMethods.
+   */
+  @Test
+  public void testRaceWhileNNStartup() throws Exception {
+    MiniDFSCluster cluster = null;
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+      cluster.waitActive();
+      final NameNode namenode = cluster.getNameNode();
+      final NamenodeProtocols rpcServer = namenode.getRpcServer();
+      Whitebox.setInternalState(namenode, "rpcServer", null);
+
+      final Path foo = new Path("/foo");
+      final FileSystem webHdfs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+          WebHdfsFileSystem.SCHEME);
+      try {
+        webHdfs.mkdirs(foo);
+        fail("Expected RetriableException");
+      } catch (RetriableException e) {
+        GenericTestUtils.assertExceptionContains("Namenode is in startup mode",
+            e);
+      }
+      Whitebox.setInternalState(namenode, "rpcServer", rpcServer);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSForHA.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSForHA.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSForHA.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSForHA.java Tue Jul 29 00:49:14 2014
@@ -18,6 +18,15 @@
 
 package org.apache.hadoop.hdfs.web;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -29,18 +38,14 @@ import org.apache.hadoop.hdfs.DFSTestUti
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.token.Token;
 import org.junit.Assert;
 import org.junit.Test;
-
-import java.io.IOException;
-import java.net.URI;
-
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
+import org.mockito.internal.util.reflection.Whitebox;
 
 public class TestWebHDFSForHA {
   private static final String LOGICAL_NAME = "minidfs";
@@ -182,4 +187,61 @@ public class TestWebHDFSForHA {
       }
     }
   }
+
+  /**
+   * Make sure the WebHdfsFileSystem will retry based on RetriableException when
+   * rpcServer is null in NamenodeWebHdfsMethods while NameNode starts up.
+   */
+  @Test (timeout=120000)
+  public void testRetryWhileNNStartup() throws Exception {
+    final Configuration conf = DFSTestUtil.newHAConfiguration(LOGICAL_NAME);
+    MiniDFSCluster cluster = null;
+    final Map<String, Boolean> resultMap = new HashMap<String, Boolean>();
+
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).nnTopology(topo)
+          .numDataNodes(0).build();
+      HATestUtil.setFailoverConfigurations(cluster, conf, LOGICAL_NAME);
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+
+      final NameNode namenode = cluster.getNameNode(0);
+      final NamenodeProtocols rpcServer = namenode.getRpcServer();
+      Whitebox.setInternalState(namenode, "rpcServer", null);
+
+      new Thread() {
+        @Override
+        public void run() {
+          boolean result = false;
+          FileSystem fs = null;
+          try {
+            fs = FileSystem.get(WEBHDFS_URI, conf);
+            final Path dir = new Path("/test");
+            result = fs.mkdirs(dir);
+          } catch (IOException e) {
+            result = false;
+          } finally {
+            IOUtils.cleanup(null, fs);
+          }
+          synchronized (TestWebHDFSForHA.this) {
+            resultMap.put("mkdirs", result);
+            TestWebHDFSForHA.this.notifyAll();
+          }
+        }
+      }.start();
+
+      Thread.sleep(1000);
+      Whitebox.setInternalState(namenode, "rpcServer", rpcServer);
+      synchronized (this) {
+        while (!resultMap.containsKey("mkdirs")) {
+          this.wait();
+        }
+        Assert.assertTrue(resultMap.get("mkdirs"));
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java Tue Jul 29 00:49:14 2014
@@ -355,12 +355,6 @@ public class TestParam {
   public void testXAttrNameParam() {
     final XAttrNameParam p = new XAttrNameParam("user.a1");
     Assert.assertEquals(p.getXAttrName(), "user.a1");
-    try {
-      new XAttrNameParam("a1");
-      Assert.fail();
-    } catch (IllegalArgumentException e) {
-      LOG.info("EXPECTED: " + e);
-    }
   }
   
   @Test

Modified: hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml?rev=1614234&r1=1614233&r2=1614234&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml Tue Jul 29 00:49:14 2014
@@ -986,6 +986,8 @@
         <NAMESPACE>USER</NAMESPACE>
         <NAME>a2</NAME>
       </XATTR>
+      <RPC_CLIENTID>e03f4a52-3d85-4e05-8942-286185e639bd</RPC_CLIENTID>
+      <RPC_CALLID>82</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>