Ver Fonte

Merge r1477868 through r1479732 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1479733 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze há 12 anos atrás
pai
commit
393188c445
100 ficheiros alterados com 1883 adições e 606 exclusões
  1. 23 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 1 1
      hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.cmd
  3. 1 1
      hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
  4. 88 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  5. 7 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java
  6. 95 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SecureIOUtils.java
  7. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/GzipCodec.java
  8. 9 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
  9. 6 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
  10. 16 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
  11. 16 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RemoteException.java
  12. 47 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcNoSuchMethodException.java
  13. 46 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcNoSuchProtocolException.java
  14. 17 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcServerException.java
  15. 75 29
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  16. 49 49
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
  17. 7 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
  18. 2 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/SSLFactory.java
  19. 11 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PlatformName.java
  20. 1 1
      hadoop-common-project/hadoop-common/src/main/proto/ProtobufRpcEngine.proto
  21. 34 5
      hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto
  22. 3 0
      hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h
  23. 8 0
      hadoop-common-project/hadoop-common/src/main/winutils/main.c
  24. 224 0
      hadoop-common-project/hadoop-common/src/main/winutils/readlink.c
  25. 11 0
      hadoop-common-project/hadoop-common/src/main/winutils/symlink.c
  26. 2 1
      hadoop-common-project/hadoop-common/src/main/winutils/winutils.vcxproj
  27. 34 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  28. 0 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextPermissionBase.java
  29. 40 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFSFileContextSymlink.java
  30. 87 27
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSecureIOUtils.java
  31. 5 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java
  32. 8 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
  33. 118 25
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestWinUtils.java
  34. 7 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  35. 8 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  36. 0 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  37. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
  38. 0 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsPermission.java
  39. 2 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java
  40. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
  41. 31 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  42. 24 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java
  43. 147 147
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
  44. 14 0
      hadoop-mapreduce-project/CHANGES.txt
  45. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  46. 128 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
  47. 5 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java
  48. 15 15
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/impl/pb/client/MRClientProtocolPBClientImpl.java
  49. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/security/MRDelegationTokenRenewer.java
  50. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  51. 4 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  52. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
  53. 66 21
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
  54. 43 19
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
  55. 10 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java
  56. 5 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java
  57. 2 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java
  58. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java
  59. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java
  60. 29 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/resources/hdfs-site.xml
  61. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
  62. 12 0
      hadoop-yarn-project/CHANGES.txt
  63. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/YarnRemoteException.java
  64. 1 26
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/impl/pb/YarnRemoteExceptionPBImpl.java
  65. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
  66. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java
  67. 9 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/RMAdmin.java
  68. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/AMRMProtocolPBClientImpl.java
  69. 13 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ClientRMProtocolPBClientImpl.java
  70. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java
  71. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java
  72. 50 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/RPCUtil.java
  73. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/RMDelegationTokenIdentifier.java
  74. 6 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerConstants.java
  75. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceTrackerPBClientImpl.java
  76. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/impl/pb/client/LocalizationProtocolPBClientImpl.java
  77. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  78. 7 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
  79. 10 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
  80. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java
  81. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
  82. 4 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java
  83. 7 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
  84. 17 25
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
  85. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
  86. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
  87. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java
  88. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  89. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  90. 5 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
  91. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMDelegationTokenSecretManager.java
  92. 7 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
  93. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
  94. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
  95. 15 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
  96. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
  97. 5 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java
  98. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java
  99. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
  100. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java

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

@@ -19,6 +19,8 @@ Trunk (Unreleased)
 
     HADOOP-9194. RPC Support for QoS. (Junping Du via llu)
 
+    HADOOP-9425 Add error codes to rpc-response (sanjay Radia)
+
   NEW FEATURES
     
     HADOOP-8561. Introduce HADOOP_PROXY_USER for secure impersonation in child
@@ -166,6 +168,8 @@ Trunk (Unreleased)
     HADOOP-9486. Promoted Windows and Shell related utils from YARN to Hadoop
     Common. (Chris Nauroth via vinodkv)
 
+    HADOOP-8608. Add Configuration API for parsing time durations. (cdouglas)
+
   BUG FIXES
 
     HADOOP-9451. Fault single-layer config if node group topology is enabled.
@@ -534,6 +538,15 @@ Trunk (Unreleased)
 
     HADOOP-9413. Add common utils for File#setReadable/Writable/Executable &
     File#canRead/Write/Execute that work cross-platform. (Ivan Mitic via suresh)
+
+    HADOOP-9532. HADOOP_CLIENT_OPTS is appended twice by Windows cmd scripts.
+    (Chris Nauroth via suresh)
+
+    HADOOP-9043. Disallow in winutils creating symlinks with forwards slashes.
+    (Chris Nauroth and Arpit Agarwal via suresh)
+
+    HADOOP-9483. winutils support for readlink command.
+    (Arpit Agarwal via suresh)
     
 Release 2.0.5-beta - UNRELEASED
 
@@ -583,6 +596,13 @@ Release 2.0.5-beta - UNRELEASED
     HADOOP-9322. LdapGroupsMapping doesn't seem to set a timeout for
     its directory search. (harsh)
 
+    HADOOP-9523. Provide a generic IBM java vendor flag in PlatformName.java
+    to support non-Sun JREs. (Tian Hong Wang via suresh)
+
+    HADOOP-9511. Adding support for additional input streams (FSDataInputStream
+    and RandomAccessFile) in SecureIOUtils so as to help YARN-578. (Omkar Vinit
+    Joshi via vinodkv)
+
   OPTIMIZATIONS
 
     HADOOP-9150. Avoid unnecessary DNS resolution attempts for logical URIs
@@ -668,6 +688,9 @@ Release 2.0.5-beta - UNRELEASED
     HADOOP-9504. MetricsDynamicMBeanBase has concurrency issues in
     createMBeanInfo (Liang Xie via jlowe)
 
+    HADOOP-9455. HADOOP_CLIENT_OPTS appended twice causes JVM failures.
+    (Chris Nauroth via suresh)
+
 Release 2.0.4-alpha - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.cmd

@@ -43,7 +43,7 @@ if exist %HADOOP_HOME%\contrib\capacity-scheduler (
 @rem set HADOOP_NAMENODE_INIT_HEAPSIZE=""
 
 @rem Extra Java runtime options.  Empty by default.
-@rem set HADOOP_OPTS=-Djava.net.preferIPv4Stack=true %HADOOP_CLIENT_OPTS%
+@rem set HADOOP_OPTS=-Djava.net.preferIPv4Stack=true
 
 @rem Command specific options appended to HADOOP_OPTS when specified
 if not defined HADOOP_SECURITY_LOGGER (

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh

@@ -45,7 +45,7 @@ done
 #export HADOOP_NAMENODE_INIT_HEAPSIZE=""
 
 # Extra Java runtime options.  Empty by default.
-export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true $HADOOP_CLIENT_OPTS"
+export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true"
 
 MAC_OSX=false
 case "`uname`" in

+ 88 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -54,6 +54,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.regex.PatternSyntaxException;
+import java.util.concurrent.TimeUnit;
 
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
@@ -1145,6 +1146,93 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       : Enum.valueOf(defaultValue.getDeclaringClass(), val);
   }
 
+  enum ParsedTimeDuration {
+    NS {
+      TimeUnit unit() { return TimeUnit.NANOSECONDS; }
+      String suffix() { return "ns"; }
+    },
+    US {
+      TimeUnit unit() { return TimeUnit.MICROSECONDS; }
+      String suffix() { return "us"; }
+    },
+    MS {
+      TimeUnit unit() { return TimeUnit.MILLISECONDS; }
+      String suffix() { return "ms"; }
+    },
+    S {
+      TimeUnit unit() { return TimeUnit.SECONDS; }
+      String suffix() { return "s"; }
+    },
+    M {
+      TimeUnit unit() { return TimeUnit.MINUTES; }
+      String suffix() { return "m"; }
+    },
+    H {
+      TimeUnit unit() { return TimeUnit.HOURS; }
+      String suffix() { return "h"; }
+    },
+    D {
+      TimeUnit unit() { return TimeUnit.DAYS; }
+      String suffix() { return "d"; }
+    };
+    abstract TimeUnit unit();
+    abstract String suffix();
+    static ParsedTimeDuration unitFor(String s) {
+      for (ParsedTimeDuration ptd : values()) {
+        // iteration order is in decl order, so SECONDS matched last
+        if (s.endsWith(ptd.suffix())) {
+          return ptd;
+        }
+      }
+      return null;
+    }
+    static ParsedTimeDuration unitFor(TimeUnit unit) {
+      for (ParsedTimeDuration ptd : values()) {
+        if (ptd.unit() == unit) {
+          return ptd;
+        }
+      }
+      return null;
+    }
+  }
+
+  /**
+   * Set the value of <code>name</code> to the given time duration. This
+   * is equivalent to <code>set(&lt;name&gt;, value + &lt;time suffix&gt;)</code>.
+   * @param name Property name
+   * @param value Time duration
+   * @param unit Unit of time
+   */
+  public void setTimeDuration(String name, long value, TimeUnit unit) {
+    set(name, value + ParsedTimeDuration.unitFor(unit).suffix());
+  }
+
+  /**
+   * Return time duration in the given time unit. Valid units are encoded in
+   * properties as suffixes: nanoseconds (ns), microseconds (us), milliseconds
+   * (ms), seconds (s), minutes (m), hours (h), and days (d).
+   * @param name Property name
+   * @param defaultValue Value returned if no mapping exists.
+   * @param unit Unit to convert the stored property, if it exists.
+   * @throws NumberFormatException If the property stripped of its unit is not
+   *         a number
+   */
+  public long getTimeDuration(String name, long defaultValue, TimeUnit unit) {
+    String vStr = get(name);
+    if (null == vStr) {
+      return defaultValue;
+    }
+    vStr = vStr.trim();
+    ParsedTimeDuration vUnit = ParsedTimeDuration.unitFor(vStr);
+    if (null == vUnit) {
+      LOG.warn("No unit for " + name + "(" + vStr + ") assuming " + unit);
+      vUnit = ParsedTimeDuration.unitFor(unit);
+    } else {
+      vStr = vStr.substring(0, vStr.lastIndexOf(vUnit.suffix()));
+    }
+    return unit.convert(Long.parseLong(vStr), vUnit.unit());
+  }
+
   /**
    * Get the value of the <code>name</code> property as a <code>Pattern</code>.
    * If no such property is specified, or if the specified value is not a valid

+ 7 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.fs.local;
 
 import java.io.IOException;
+import java.io.File;
 import java.io.FileNotFoundException;
 import java.net.URI;
 import java.net.URISyntaxException;
@@ -90,8 +91,8 @@ public class RawLocalFs extends DelegateToFileSystem {
     // NB: Use createSymbolicLink in java.nio.file.Path once available
     try {
       Shell.execCommand(Shell.getSymlinkCommand(
-        getPathWithoutSchemeAndAuthority(target),
-        getPathWithoutSchemeAndAuthority(link)));
+        getPathWithoutSchemeAndAuthority(target).getPath(),
+        getPathWithoutSchemeAndAuthority(link).getPath()));
     } catch (IOException x) {
       throw new IOException("Unable to create symlink: "+x.getMessage());
     }
@@ -175,12 +176,12 @@ public class RawLocalFs extends DelegateToFileSystem {
     throw new AssertionError();
   }
 
-  private static String getPathWithoutSchemeAndAuthority(Path path) {
-    // This code depends on Path.toString() to remove the leading slash before
-    // the drive specification on Windows.
+  private static File getPathWithoutSchemeAndAuthority(Path path) {
     Path newPath = path.isUriPathAbsolute() ?
       new Path(null, null, path.toUri().getPath()) :
       path;
-    return newPath.toString();
+
+    // Path.toString() removes leading slash before drive spec on Windows.
+    return new File(newPath.toString());
   }
 }

+ 95 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SecureIOUtils.java

@@ -18,22 +18,23 @@
 package org.apache.hadoop.io;
 
 import java.io.File;
-import java.io.FileDescriptor;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.RandomAccessFile;
 import java.util.Arrays;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.io.nativeio.Errno;
 import org.apache.hadoop.io.nativeio.NativeIO;
-import org.apache.hadoop.io.nativeio.NativeIOException;
 import org.apache.hadoop.io.nativeio.NativeIO.POSIX.Stat;
 import org.apache.hadoop.security.UserGroupInformation;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * This class provides secure APIs for opening and creating files on the local
  * disk. The main issue this class tries to handle is that of symlink traversal.
@@ -89,6 +90,95 @@ public class SecureIOUtils {
   private final static boolean skipSecurity;
   private final static FileSystem rawFilesystem;
 
+  /**
+   * Open the given File for random read access, verifying the expected user/
+   * group constraints if security is enabled.
+   * 
+   * Note that this function provides no additional security checks if hadoop
+   * security is disabled, since doing the checks would be too expensive when
+   * native libraries are not available.
+   * 
+   * @param f file that we are trying to open
+   * @param mode mode in which we want to open the random access file
+   * @param expectedOwner the expected user owner for the file
+   * @param expectedGroup the expected group owner for the file
+   * @throws IOException if an IO error occurred or if the user/group does
+   * not match when security is enabled.
+   */
+  public static RandomAccessFile openForRandomRead(File f,
+      String mode, String expectedOwner, String expectedGroup)
+      throws IOException {
+    if (!UserGroupInformation.isSecurityEnabled()) {
+      return new RandomAccessFile(f, mode);
+    }
+    return forceSecureOpenForRandomRead(f, mode, expectedOwner, expectedGroup);
+  }
+
+  /**
+   * Same as openForRandomRead except that it will run even if security is off.
+   * This is used by unit tests.
+   */
+  @VisibleForTesting
+  protected static RandomAccessFile forceSecureOpenForRandomRead(File f,
+      String mode, String expectedOwner, String expectedGroup)
+      throws IOException {
+    RandomAccessFile raf = new RandomAccessFile(f, mode);
+    boolean success = false;
+    try {
+      Stat stat = NativeIO.POSIX.getFstat(raf.getFD());
+      checkStat(f, stat.getOwner(), stat.getGroup(), expectedOwner,
+          expectedGroup);
+      success = true;
+      return raf;
+    } finally {
+      if (!success) {
+        raf.close();
+      }
+    }
+  }
+
+  /**
+   * Opens the {@link FSDataInputStream} on the requested file on local file
+   * system, verifying the expected user/group constraints if security is
+   * enabled.
+   * @param file absolute path of the file
+   * @param expectedOwner the expected user owner for the file
+   * @param expectedGroup the expected group owner for the file
+   * @throws IOException if an IO Error occurred or the user/group does not
+   * match if security is enabled
+   */
+  public static FSDataInputStream openFSDataInputStream(File file,
+      String expectedOwner, String expectedGroup) throws IOException {
+    if (!UserGroupInformation.isSecurityEnabled()) {
+      return rawFilesystem.open(new Path(file.getAbsolutePath()));
+    }
+    return forceSecureOpenFSDataInputStream(file, expectedOwner, expectedGroup);
+  }
+
+  /**
+   * Same as openFSDataInputStream except that it will run even if security is
+   * off. This is used by unit tests.
+   */
+  @VisibleForTesting
+  protected static FSDataInputStream forceSecureOpenFSDataInputStream(
+      File file,
+      String expectedOwner, String expectedGroup) throws IOException {
+    final FSDataInputStream in =
+        rawFilesystem.open(new Path(file.getAbsolutePath()));
+    boolean success = false;
+    try {
+      Stat stat = NativeIO.POSIX.getFstat(in.getFileDescriptor());
+      checkStat(file, stat.getOwner(), stat.getGroup(), expectedOwner,
+          expectedGroup);
+      success = true;
+      return in;
+    } finally {
+      if (!success) {
+        in.close();
+      }
+    }
+  }
+
   /**
    * Open the given File for read access, verifying the expected user/group
    * constraints if security is enabled.
@@ -115,7 +205,8 @@ public class SecureIOUtils {
    * Same as openForRead() except that it will run even if security is off.
    * This is used by unit tests.
    */
-  static FileInputStream forceSecureOpenForRead(File f, String expectedOwner,
+  @VisibleForTesting
+  protected static FileInputStream forceSecureOpenForRead(File f, String expectedOwner,
       String expectedGroup) throws IOException {
 
     FileInputStream fis = new FileInputStream(f);

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/GzipCodec.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.compress.DefaultCodec;
 import org.apache.hadoop.io.compress.zlib.*;
+import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
 
 /**
  * This class creates gzip compressors/decompressors. 
@@ -41,10 +42,9 @@ public class GzipCodec extends DefaultCodec {
 
     private static class ResetableGZIPOutputStream extends GZIPOutputStream {
       private static final int TRAILER_SIZE = 8;
-      public static final String JVMVendor= System.getProperty("java.vendor");
       public static final String JVMVersion= System.getProperty("java.version");
       private static final boolean HAS_BROKEN_FINISH =
-          (JVMVendor.contains("IBM") && JVMVersion.contains("1.6.0"));
+          (IBM_JAVA && JVMVersion.contains("1.6.0"));
 
       public ResetableGZIPOutputStream(OutputStream out) throws IOException {
         super(out);

+ 9 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java

@@ -65,6 +65,7 @@ import org.apache.hadoop.io.retry.RetryPolicy.RetryAction;
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto;
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto.OperationProto;
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto;
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto;
 import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.net.NetUtils;
@@ -987,8 +988,15 @@ public class Client {
                   "ServerDidNotSetExceptionClassName";
           final String errorMsg = header.hasErrorMsg() ? 
                 header.getErrorMsg() : "ServerDidNotSetErrorMsg" ;
+          final RpcErrorCodeProto erCode = 
+                    (header.hasErrorDetail() ? header.getErrorDetail() : null);
+          if (erCode == null) {
+             LOG.warn("Detailed error code not set by server on rpc error");
+          }
           RemoteException re = 
-              new RemoteException(exceptionClassName, errorMsg);
+              ( (erCode == null) ? 
+                  new RemoteException(exceptionClassName, errorMsg) :
+              new RemoteException(exceptionClassName, errorMsg, erCode));
           if (status == RpcStatusProto.ERROR) {
             call.setException(re);
             calls.remove(callId);

+ 6 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java

@@ -437,8 +437,8 @@ public class ProtobufRpcEngine implements RpcEngine {
      */
     static class ProtoBufRpcInvoker implements RpcInvoker {
       private static ProtoClassProtoImpl getProtocolImpl(RPC.Server server,
-          String protoName, long version) throws IOException {
-        ProtoNameVer pv = new ProtoNameVer(protoName, version);
+          String protoName, long clientVersion) throws RpcServerException {
+        ProtoNameVer pv = new ProtoNameVer(protoName, clientVersion);
         ProtoClassProtoImpl impl = 
             server.getProtocolImplMap(RPC.RpcKind.RPC_PROTOCOL_BUFFER).get(pv);
         if (impl == null) { // no match for Protocol AND Version
@@ -446,10 +446,11 @@ public class ProtobufRpcEngine implements RpcEngine {
               server.getHighestSupportedProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, 
                   protoName);
           if (highest == null) {
-            throw new IOException("Unknown protocol: " + protoName);
+            throw new RpcNoSuchProtocolException(
+                "Unknown protocol: " + protoName);
           }
           // protocol supported but not the version that client wants
-          throw new RPC.VersionMismatch(protoName, version,
+          throw new RPC.VersionMismatch(protoName, clientVersion,
               highest.version);
         }
         return impl;
@@ -513,7 +514,7 @@ public class ProtobufRpcEngine implements RpcEngine {
           String msg = "Unknown method " + methodName + " called on " 
                                 + connectionProtocolName + " protocol.";
           LOG.warn(msg);
-          throw new RpcServerException(msg);
+          throw new RpcNoSuchMethodException(msg);
         }
         Message prototype = service.getRequestPrototype(methodDescriptor);
         Message param = prototype.newBuilderForType()

+ 16 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java

@@ -43,6 +43,8 @@ import org.apache.hadoop.io.*;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.ipc.Client.ConnectionId;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolInfoService;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SaslRpcServer;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -213,7 +215,7 @@ public class RPC {
   /**
    * A version mismatch for the RPC protocol.
    */
-  public static class VersionMismatch extends IOException {
+  public static class VersionMismatch extends RpcServerException {
     private static final long serialVersionUID = 0;
 
     private String interfaceName;
@@ -257,6 +259,19 @@ public class RPC {
     public long getServerVersion() {
       return serverVersion;
     }
+    /**
+     * get the rpc status corresponding to this exception
+     */
+    public RpcStatusProto getRpcStatusProto() {
+      return RpcStatusProto.ERROR;
+    }
+
+    /**
+     * get the detailed rpc status corresponding to this exception
+     */
+    public RpcErrorCodeProto getRpcErrorCodeProto() {
+      return RpcErrorCodeProto.ERROR_RPC_VERSION_MISMATCH;
+    }
   }
 
   /**

+ 16 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RemoteException.java

@@ -21,22 +21,38 @@ package org.apache.hadoop.ipc;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto;
 import org.xml.sax.Attributes;
 
 public class RemoteException extends IOException {
   /** For java.io.Serializable */
   private static final long serialVersionUID = 1L;
+  private final int errorCode;
 
   private String className;
   
   public RemoteException(String className, String msg) {
     super(msg);
     this.className = className;
+    errorCode = -1;
+  }
+  
+  public RemoteException(String className, String msg, RpcErrorCodeProto erCode) {
+    super(msg);
+    this.className = className;
+    if (erCode != null)
+      errorCode = erCode.getNumber();
+    else 
+      errorCode = -1;
   }
   
   public String getClassName() {
     return className;
   }
+  
+  public RpcErrorCodeProto getErrorCode() {
+    return RpcErrorCodeProto.valueOf(errorCode);
+  }
 
   /**
    * If this remote exception wraps up one of the lookupTypes

+ 47 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcNoSuchMethodException.java

@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ipc;
+
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto;
+
+
+/**
+ * No such Method for an Rpc Call
+ *
+ */
+public class RpcNoSuchMethodException extends RpcServerException {
+  private static final long serialVersionUID = 1L;
+  public RpcNoSuchMethodException(final String message) {
+    super(message);
+  }
+  
+  /**
+   * get the rpc status corresponding to this exception
+   */
+  public RpcStatusProto getRpcStatusProto() {
+    return RpcStatusProto.ERROR;
+  }
+
+  /**
+   * get the detailed rpc status corresponding to this exception
+   */
+  public RpcErrorCodeProto getRpcErrorCodeProto() {
+    return RpcErrorCodeProto.ERROR_NO_SUCH_METHOD;
+  }
+}

+ 46 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcNoSuchProtocolException.java

@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ipc;
+
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto;
+
+/**
+ * No such protocol (i.e. interface) for and Rpc Call
+ *
+ */
+public class RpcNoSuchProtocolException extends RpcServerException {
+  private static final long serialVersionUID = 1L;
+  public RpcNoSuchProtocolException(final String message) {
+    super(message);
+  }
+  
+  /**
+   * get the rpc status corresponding to this exception
+   */
+  public RpcStatusProto getRpcStatusProto() {
+    return RpcStatusProto.ERROR;
+  }
+
+  /**
+   * get the detailed rpc status corresponding to this exception
+   */
+  public RpcErrorCodeProto getRpcErrorCodeProto() {
+    return RpcErrorCodeProto.ERROR_NO_SUCH_PROTOCOL;
+  }
+}

+ 17 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcServerException.java

@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.ipc;
 
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto;
+
 /**
  * Indicates an exception on the RPC server 
  */
@@ -42,4 +45,18 @@ public class RpcServerException extends RpcException {
   public RpcServerException(final String message, final Throwable cause) {
     super(message, cause);
   }
+  
+  /**
+   * get the rpc status corresponding to this exception
+   */
+  public RpcStatusProto getRpcStatusProto() {
+    return RpcStatusProto.ERROR;
+  }
+
+  /**
+   * get the detailed rpc status corresponding to this exception
+   */
+  public RpcErrorCodeProto getRpcErrorCodeProto() {
+    return RpcErrorCodeProto.ERROR_RPC_SERVER;
+  }
 }

+ 75 - 29
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -83,6 +83,7 @@ import org.apache.hadoop.ipc.metrics.RpcDetailedMetrics;
 import org.apache.hadoop.ipc.metrics.RpcMetrics;
 import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto;
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.*;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
@@ -1438,7 +1439,8 @@ public abstract class Server {
         final String ioeMessage  = ioe.getLocalizedMessage();
         if (authMethod == AuthMethod.SIMPLE) {
           setupResponse(authFailedResponse, authFailedCall,
-              RpcStatusProto.FATAL, null, ioeClass, ioeMessage);
+              RpcStatusProto.FATAL, RpcErrorCodeProto.FATAL_UNAUTHORIZED, 
+              null, ioeClass, ioeMessage);
           responder.doRespond(authFailedCall);
         } else {
           doSaslReply(SaslStatus.ERROR, null, ioeClass, ioeMessage);
@@ -1528,7 +1530,8 @@ public abstract class Server {
       if (clientVersion >= 9) {
         // Versions >>9  understand the normal response
         Call fakeCall =  new Call(-1, null, this);
-        setupResponse(buffer, fakeCall, RpcStatusProto.FATAL,
+        setupResponse(buffer, fakeCall, 
+            RpcStatusProto.FATAL, RpcErrorCodeProto.FATAL_VERSION_MISMATCH,
             null, VersionMismatch.class.getName(), errMsg);
         responder.doRespond(fakeCall);
       } else if (clientVersion >= 3) {
@@ -1557,8 +1560,9 @@ public abstract class Server {
       ByteArrayOutputStream buffer = new ByteArrayOutputStream();
 
       Call fakeCall = new Call(-1, null, this);
-      setupResponse(buffer, fakeCall, RpcStatusProto.FATAL, null,
-          IpcException.class.getName(), errMsg);
+      setupResponse(buffer, fakeCall, 
+       RpcStatusProto.FATAL,  RpcErrorCodeProto.FATAL_UNSUPPORTED_SERIALIZATION,
+       null, IpcException.class.getName(), errMsg);
       responder.doRespond(fakeCall);
     }
     
@@ -1647,7 +1651,7 @@ public abstract class Server {
     private void processOneRpc(byte[] buf) throws IOException,
         InterruptedException {
       if (connectionContextRead) {
-        processData(buf);
+        processRpcRequest(buf);
       } else {
         processConnectionContext(buf);
         connectionContextRead = true;
@@ -1659,7 +1663,17 @@ public abstract class Server {
       }
     }
     
-    private void processData(byte[] buf) throws  IOException, InterruptedException {
+    /**
+     * Process an RPC Request - the connection headers and context have been
+     * read
+     * @param buf - contains the RPC request header and the rpc request
+     * @throws RpcServerException due to fatal rpc layer issues such as
+     *   invalid header. In this case a RPC fatal status response is sent back
+     *   to client.
+     */
+    
+    private void processRpcRequest(byte[] buf) 
+        throws  RpcServerException, IOException, InterruptedException {
       DataInputStream dis =
         new DataInputStream(new ByteArrayInputStream(buf));
       RpcRequestHeaderProto header = RpcRequestHeaderProto.parseDelimitedFrom(dis);
@@ -1667,51 +1681,58 @@ public abstract class Server {
       if (LOG.isDebugEnabled())
         LOG.debug(" got #" + header.getCallId());
       if (!header.hasRpcOp()) {
-        throw new IOException(" IPC Server: No rpc op in rpcRequestHeader");
+        String err = " IPC Server: No rpc op in rpcRequestHeader";
+        respondBadRpcHeader(new Call(header.getCallId(), null, this),
+            RpcServerException.class.getName(), err);
+        throw new RpcServerException(err);
       }
       if (header.getRpcOp() != 
           RpcRequestHeaderProto.OperationProto.RPC_FINAL_PACKET) {
-        throw new IOException("IPC Server does not implement operation" + 
-              header.getRpcOp());
+        String err = "IPC Server does not implement rpc header operation" + 
+                header.getRpcOp();
+        respondBadRpcHeader(new Call(header.getCallId(), null, this),
+            RpcServerException.class.getName(), err);
+        throw new RpcServerException(err);
       }
       // If we know the rpc kind, get its class so that we can deserialize
       // (Note it would make more sense to have the handler deserialize but 
       // we continue with this original design.
       if (!header.hasRpcKind()) {
-        throw new IOException(" IPC Server: No rpc kind in rpcRequestHeader");
+        String err = " IPC Server: No rpc kind in rpcRequestHeader";
+        respondBadRpcHeader(new Call(header.getCallId(), null, this),
+            RpcServerException.class.getName(), err);
+        throw new RpcServerException(err);
       }
       Class<? extends Writable> rpcRequestClass = 
           getRpcRequestWrapper(header.getRpcKind());
       if (rpcRequestClass == null) {
         LOG.warn("Unknown rpc kind "  + header.getRpcKind() + 
             " from client " + getHostAddress());
-        final Call readParamsFailedCall = 
-            new Call(header.getCallId(), null, this);
-        ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
-
-        setupResponse(responseBuffer, readParamsFailedCall, RpcStatusProto.FATAL, null,
-            IOException.class.getName(),
-            "Unknown rpc kind "  + header.getRpcKind());
-        responder.doRespond(readParamsFailedCall);
-        return;   
+        final String err = "Unknown rpc kind in rpc header"  + 
+            header.getRpcKind();
+        respondBadRpcHeader(new Call(header.getCallId(), null, this),
+            RpcServerException.class.getName(), err);
+        throw new RpcServerException(err);   
       }
       Writable rpcRequest;
       try { //Read the rpc request
         rpcRequest = ReflectionUtils.newInstance(rpcRequestClass, conf);
         rpcRequest.readFields(dis);
-      } catch (Throwable t) {
+      } catch (Throwable t) { // includes runtime exception from newInstance
         LOG.warn("Unable to read call parameters for client " +
                  getHostAddress() + "on connection protocol " +
             this.protocolName + " for rpcKind " + header.getRpcKind(),  t);
         final Call readParamsFailedCall = 
             new Call(header.getCallId(), null, this);
         ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
+        String err = "IPC server unable to read call parameters: "+ t.getMessage();
 
-        setupResponse(responseBuffer, readParamsFailedCall, RpcStatusProto.FATAL, null,
-            t.getClass().getName(),
-            "IPC server unable to read call parameters: " + t.getMessage());
+        setupResponse(responseBuffer, readParamsFailedCall, 
+            RpcStatusProto.FATAL, RpcErrorCodeProto.FATAL_DESERIALIZING_REQUEST,
+            null, t.getClass().getName(),
+            err);
         responder.doRespond(readParamsFailedCall);
-        return;
+        throw new RpcServerException(err, t);
       }
         
       Call call = new Call(header.getCallId(), rpcRequest, this, 
@@ -1737,7 +1758,8 @@ public abstract class Server {
         rpcMetrics.incrAuthorizationSuccesses();
       } catch (AuthorizationException ae) {
         rpcMetrics.incrAuthorizationFailures();
-        setupResponse(authFailedResponse, authFailedCall, RpcStatusProto.FATAL, null,
+        setupResponse(authFailedResponse, authFailedCall, 
+            RpcStatusProto.FATAL,  RpcErrorCodeProto.FATAL_UNAUTHORIZED, null,
             ae.getClass().getName(), ae.getMessage());
         responder.doRespond(authFailedCall);
         return false;
@@ -1799,6 +1821,8 @@ public abstract class Server {
           }
           String errorClass = null;
           String error = null;
+          RpcStatusProto returnStatus = RpcStatusProto.SUCCESS;
+          RpcErrorCodeProto detailedErr = null;
           Writable value = null;
 
           CurCall.set(call);
@@ -1839,7 +1863,14 @@ public abstract class Server {
             } else {
               LOG.info(logMsg, e);
             }
-
+            if (e instanceof RpcServerException) {
+              RpcServerException rse = ((RpcServerException)e); 
+              returnStatus = rse.getRpcStatusProto();
+              detailedErr = rse.getRpcErrorCodeProto();
+            } else {
+              returnStatus = RpcStatusProto.ERROR;
+              detailedErr = RpcErrorCodeProto.ERROR_APPLICATION;
+            }
             errorClass = e.getClass().getName();
             error = StringUtils.stringifyException(e);
             // Remove redundant error class name from the beginning of the stack trace
@@ -1854,8 +1885,8 @@ public abstract class Server {
             // responder.doResponse() since setupResponse may use
             // SASL to encrypt response data and SASL enforces
             // its own message ordering.
-            setupResponse(buf, call, (error == null) ? RpcStatusProto.SUCCESS
-                : RpcStatusProto.ERROR, value, errorClass, error);
+            setupResponse(buf, call, returnStatus, detailedErr, 
+                value, errorClass, error);
             
             // Discard the large buf and reset it back to smaller size 
             // to free up heap
@@ -2026,7 +2057,7 @@ public abstract class Server {
    * @throws IOException
    */
   private void setupResponse(ByteArrayOutputStream responseBuf,
-                             Call call, RpcStatusProto status, 
+                             Call call, RpcStatusProto status, RpcErrorCodeProto erCode,
                              Writable rv, String errorClass, String error) 
   throws IOException {
     responseBuf.reset();
@@ -2065,6 +2096,7 @@ public abstract class Server {
         // buffer is reset at the top, and since status is changed
         // to ERROR it won't infinite loop.
         setupResponse(responseBuf, call, RpcStatusProto.ERROR,
+            RpcErrorCodeProto.ERROR_SERIALIZING_RESPONSE,
             null, t.getClass().getName(),
             StringUtils.stringifyException(t));
         return;
@@ -2072,6 +2104,7 @@ public abstract class Server {
     } else { // Rpc Failure
       headerBuilder.setExceptionClassName(errorClass);
       headerBuilder.setErrorMsg(error);
+      headerBuilder.setErrorDetail(erCode);
       RpcResponseHeaderProto header = headerBuilder.build();
       int headerLen = header.getSerializedSize();
       final int fullLength  = 
@@ -2116,6 +2149,19 @@ public abstract class Server {
     call.setResponse(ByteBuffer.wrap(response.toByteArray()));
   }
   
+  
+  private void respondBadRpcHeader(Call call, String errorClass, String error)
+      throws IOException
+  {
+    ByteArrayOutputStream responseBuf = new ByteArrayOutputStream();
+    setupResponse(responseBuf, call, 
+        RpcStatusProto.FATAL,  RpcErrorCodeProto.FATAL_INVALID_RPC_HEADER,
+        null, errorClass, error);
+    responder.doRespond(call);
+    return; 
+    
+  }
+  
   private void wrapWithSasl(ByteArrayOutputStream response, Call call)
       throws IOException {
     if (call.connection.saslServer != null) {

+ 49 - 49
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java

@@ -416,62 +416,62 @@ public class WritableRpcEngine implements RpcEngine {
      @Override
       public Writable call(org.apache.hadoop.ipc.RPC.Server server,
           String protocolName, Writable rpcRequest, long receivedTime)
-          throws IOException {
-        try {
-          Invocation call = (Invocation)rpcRequest;
-          if (server.verbose) log("Call: " + call);
-
-          // Verify rpc version
-          if (call.getRpcVersion() != writableRpcVersion) {
-            // Client is using a different version of WritableRpc
-            throw new IOException(
-                "WritableRpc version mismatch, client side version="
-                    + call.getRpcVersion() + ", server side version="
-                    + writableRpcVersion);
-          }
+          throws IOException, RPC.VersionMismatch {
+
+        Invocation call = (Invocation)rpcRequest;
+        if (server.verbose) log("Call: " + call);
+
+        // Verify writable rpc version
+        if (call.getRpcVersion() != writableRpcVersion) {
+          // Client is using a different version of WritableRpc
+          throw new RpcServerException(
+              "WritableRpc version mismatch, client side version="
+                  + call.getRpcVersion() + ", server side version="
+                  + writableRpcVersion);
+        }
 
-          long clientVersion = call.getProtocolVersion();
-          final String protoName;
-          ProtoClassProtoImpl protocolImpl;
-          if (call.declaringClassProtocolName.equals(VersionedProtocol.class.getName())) {
-            // VersionProtocol methods are often used by client to figure out
-            // which version of protocol to use.
-            //
-            // Versioned protocol methods should go the protocolName protocol
-            // rather than the declaring class of the method since the
-            // the declaring class is VersionedProtocol which is not 
-            // registered directly.
-            // Send the call to the highest  protocol version
-            VerProtocolImpl highest = server.getHighestSupportedProtocol(
-                RPC.RpcKind.RPC_WRITABLE, protocolName);
+        long clientVersion = call.getProtocolVersion();
+        final String protoName;
+        ProtoClassProtoImpl protocolImpl;
+        if (call.declaringClassProtocolName.equals(VersionedProtocol.class.getName())) {
+          // VersionProtocol methods are often used by client to figure out
+          // which version of protocol to use.
+          //
+          // Versioned protocol methods should go the protocolName protocol
+          // rather than the declaring class of the method since the
+          // the declaring class is VersionedProtocol which is not 
+          // registered directly.
+          // Send the call to the highest  protocol version
+          VerProtocolImpl highest = server.getHighestSupportedProtocol(
+              RPC.RpcKind.RPC_WRITABLE, protocolName);
+          if (highest == null) {
+            throw new RpcServerException("Unknown protocol: " + protocolName);
+          }
+          protocolImpl = highest.protocolTarget;
+        } else {
+          protoName = call.declaringClassProtocolName;
+
+          // Find the right impl for the protocol based on client version.
+          ProtoNameVer pv = 
+              new ProtoNameVer(call.declaringClassProtocolName, clientVersion);
+          protocolImpl = 
+              server.getProtocolImplMap(RPC.RpcKind.RPC_WRITABLE).get(pv);
+          if (protocolImpl == null) { // no match for Protocol AND Version
+             VerProtocolImpl highest = 
+                 server.getHighestSupportedProtocol(RPC.RpcKind.RPC_WRITABLE, 
+                     protoName);
             if (highest == null) {
-              throw new IOException("Unknown protocol: " + protocolName);
-            }
-            protocolImpl = highest.protocolTarget;
-          } else {
-            protoName = call.declaringClassProtocolName;
-
-            // Find the right impl for the protocol based on client version.
-            ProtoNameVer pv = 
-                new ProtoNameVer(call.declaringClassProtocolName, clientVersion);
-            protocolImpl = 
-                server.getProtocolImplMap(RPC.RpcKind.RPC_WRITABLE).get(pv);
-            if (protocolImpl == null) { // no match for Protocol AND Version
-               VerProtocolImpl highest = 
-                   server.getHighestSupportedProtocol(RPC.RpcKind.RPC_WRITABLE, 
-                       protoName);
-              if (highest == null) {
-                throw new IOException("Unknown protocol: " + protoName);
-              } else { // protocol supported but not the version that client wants
-                throw new RPC.VersionMismatch(protoName, clientVersion,
-                  highest.version);
-              }
+              throw new RpcServerException("Unknown protocol: " + protoName);
+            } else { // protocol supported but not the version that client wants
+              throw new RPC.VersionMismatch(protoName, clientVersion,
+                highest.version);
             }
           }
+        }
           
 
           // Invoke the protocol method
-
+       try {
           long startTime = Time.now();
           Method method = 
               protocolImpl.protocolClass.getMethod(call.getMethodName(),

+ 7 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java

@@ -64,6 +64,7 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Time;
+import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -306,12 +307,11 @@ public class UserGroupInformation {
       System.getProperty("os.name").startsWith("Windows");
   private static final boolean is64Bit =
       System.getProperty("os.arch").contains("64");
-  private static final boolean ibmJava = System.getProperty("java.vendor").contains("IBM");
   private static final boolean aix = System.getProperty("os.name").equals("AIX");
 
   /* Return the OS login module class name */
   private static String getOSLoginModuleName() {
-    if (ibmJava) {
+    if (IBM_JAVA) {
       if (windows) {
         return is64Bit ? "com.ibm.security.auth.module.Win64LoginModule"
             : "com.ibm.security.auth.module.NTLoginModule";
@@ -333,7 +333,7 @@ public class UserGroupInformation {
     ClassLoader cl = ClassLoader.getSystemClassLoader();
     try {
       String principalClass = null;
-      if (ibmJava) {
+      if (IBM_JAVA) {
         if (is64Bit) {
           principalClass = "com.ibm.security.auth.UsernamePrincipal";
         } else {
@@ -430,7 +430,7 @@ public class UserGroupInformation {
     private static final Map<String,String> USER_KERBEROS_OPTIONS = 
       new HashMap<String,String>();
     static {
-      if (ibmJava) {
+      if (IBM_JAVA) {
         USER_KERBEROS_OPTIONS.put("useDefaultCcache", "true");
       } else {
         USER_KERBEROS_OPTIONS.put("doNotPrompt", "true");
@@ -439,7 +439,7 @@ public class UserGroupInformation {
       }
       String ticketCache = System.getenv("KRB5CCNAME");
       if (ticketCache != null) {
-        if (ibmJava) {
+        if (IBM_JAVA) {
           // The first value searched when "useDefaultCcache" is used.
           System.setProperty("KRB5CCNAME", ticketCache);
         } else {
@@ -455,7 +455,7 @@ public class UserGroupInformation {
     private static final Map<String,String> KEYTAB_KERBEROS_OPTIONS = 
       new HashMap<String,String>();
     static {
-      if (ibmJava) {
+      if (IBM_JAVA) {
         KEYTAB_KERBEROS_OPTIONS.put("credsType", "both");
       } else {
         KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true");
@@ -487,7 +487,7 @@ public class UserGroupInformation {
       } else if (USER_KERBEROS_CONFIG_NAME.equals(appName)) {
         return USER_KERBEROS_CONF;
       } else if (KEYTAB_KERBEROS_CONFIG_NAME.equals(appName)) {
-        if (ibmJava) {
+        if (IBM_JAVA) {
           KEYTAB_KERBEROS_OPTIONS.put("useKeytab",
               prependFileAuthority(keytabFile));
         } else {

+ 2 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/SSLFactory.java

@@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.util.ReflectionUtils;
+import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
 
 import javax.net.ssl.HostnameVerifier;
 import javax.net.ssl.HttpsURLConnection;
@@ -58,9 +59,7 @@ public class SSLFactory implements ConnectionConfigurator {
     "hadoop.ssl.client.conf";
   public static final String SSL_SERVER_CONF_KEY =
     "hadoop.ssl.server.conf";
-  private static final boolean IBMJAVA = 
-      System.getProperty("java.vendor").contains("IBM");
-  public static final String SSLCERTIFICATE = IBMJAVA?"ibmX509":"SunX509"; 
+  public static final String SSLCERTIFICATE = IBM_JAVA?"ibmX509":"SunX509"; 
 
   public static final boolean DEFAULT_SSL_REQUIRE_CLIENT_CERT = false;
 

+ 11 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PlatformName.java

@@ -32,20 +32,24 @@ public class PlatformName {
    * The complete platform 'name' to identify the platform as 
    * per the java-vm.
    */
-  private static final String platformName =
+  public static final String PLATFORM_NAME =
       (Shell.WINDOWS ? System.getenv("os") : System.getProperty("os.name"))
       + "-" + System.getProperty("os.arch")
       + "-" + System.getProperty("sun.arch.data.model");
   
   /**
-   * Get the complete platform as per the java-vm.
-   * @return returns the complete platform as per the java-vm.
+   * The java vendor name used in this platform. 
    */
-  public static String getPlatformName() {
-    return platformName;
-  }
+  public static final String JAVA_VENDOR_NAME = System.getProperty("java.vendor");
+
+  /**
+   * A public static variable to indicate the current java vendor is 
+   * IBM java or not. 
+   */
+  public static final boolean IBM_JAVA = JAVA_VENDOR_NAME.contains("IBM");
   
   public static void main(String[] args) {
-    System.out.println(platformName);
+    System.out.println("platform name: " + PLATFORM_NAME);
+    System.out.println("java vendor name: " + JAVA_VENDOR_NAME);
   }
 }

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/proto/ProtobufRpcEngine.proto

@@ -1,4 +1,4 @@
-/**DER
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information

+ 34 - 5
hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto

@@ -62,27 +62,55 @@ message RpcRequestHeaderProto { // the header for the RpcRequest
 
 /**
  * Rpc Response Header
- * ** If request is successfull response is returned as below ********
  * +------------------------------------------------------------------+
- * | Rpc reponse length in bytes (4 bytes int)                        |
+ * | Rpc total response length in bytes (4 bytes int)                 |
  * |  (sum of next two parts)                                         |
  * +------------------------------------------------------------------+
  * | RpcResponseHeaderProto - serialized delimited ie has len         |
  * +------------------------------------------------------------------+
  * | if request is successful:                                        |
  * |   - RpcResponse -  The actual rpc response  bytes follow         |
- *       the response header                                          |
+ * |     the response header                                          |
  * |     This response is serialized based on RpcKindProto            |
  * | if request fails :                                               |
  * |   The rpc response header contains the necessary info            |
  * +------------------------------------------------------------------+
  *
+ * Note that rpc response header is also used when connection setup fails. 
+ * Ie the response looks like a rpc response with a fake callId.
  */
 message RpcResponseHeaderProto {
+  /**
+    * 
+    * RpcStastus - success or failure
+    * The reponseHeader's errDetail,  exceptionClassName and errMsg contains
+    * further details on the error
+    **/
+
   enum RpcStatusProto {
    SUCCESS = 0;  // RPC succeeded
-   ERROR = 1;    // RPC Failed
-   FATAL = 2;    // Fatal error - connection is closed
+   ERROR = 1;    // RPC or error - connection left open for future calls
+   FATAL = 2;    // Fatal error - connection closed
+  }
+
+  enum RpcErrorCodeProto {
+
+   // Non-fatal Rpc error - connection left open for future rpc calls
+   ERROR_APPLICATION = 1;      // RPC Failed - rpc app threw exception
+   ERROR_NO_SUCH_METHOD = 2;   // Rpc error - no such method
+   ERROR_NO_SUCH_PROTOCOL = 3; // Rpc error - no such protocol
+   ERROR_RPC_SERVER  = 4;      // Rpc error on server side
+   ERROR_SERIALIZING_RESPONSE = 5; // error serializign response
+   ERROR_RPC_VERSION_MISMATCH = 6; // Rpc protocol version mismatch
+
+
+   // Fatal Server side Rpc error - connection closed
+   FATAL_UNKNOWN = 10;                   // unknown Fatal error
+   FATAL_UNSUPPORTED_SERIALIZATION = 11; // IPC layer serilization type invalid
+   FATAL_INVALID_RPC_HEADER = 12;        // fields of RpcHeader are invalid
+   FATAL_DESERIALIZING_REQUEST = 13;     // could not deserilize rpc request
+   FATAL_VERSION_MISMATCH = 14;          // Ipc Layer version mismatch
+   FATAL_UNAUTHORIZED = 15;              // Auth failed
   }
 
   required uint32 callId = 1; // callId used in Request
@@ -90,4 +118,5 @@ message RpcResponseHeaderProto {
   optional uint32 serverIpcVersionNum = 3; // Sent if success or fail
   optional string exceptionClassName = 4;  // if request fails
   optional string errorMsg = 5;  // if request fails, often contains strack trace
+  optional RpcErrorCodeProto errorDetail = 6; // in case of error
 }

+ 3 - 0
hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h

@@ -104,6 +104,9 @@ void TaskUsage();
 int Symlink(__in int argc, __in_ecount(argc) wchar_t *argv[]);
 void SymlinkUsage();
 
+int Readlink(__in int argc, __in_ecount(argc) wchar_t *argv[]);
+void ReadlinkUsage();
+
 int SystemInfo();
 void SystemInfoUsage();
 

+ 8 - 0
hadoop-common-project/hadoop-common/src/main/winutils/main.c

@@ -55,6 +55,10 @@ int wmain(__in int argc, __in_ecount(argc) wchar_t* argv[])
   {
     return Symlink(argc - 1, argv + 1);
   }
+  else if (wcscmp(L"readlink", cmd) == 0)
+  {
+    return Readlink(argc - 1, argv + 1);
+  }
   else if (wcscmp(L"task", cmd) == 0)
   {
     return Task(argc - 1, argv + 1);
@@ -105,6 +109,10 @@ The available commands and their usages are:\n\n", program);
   SymlinkUsage();
   fwprintf(stdout, L"\n\n");
 
+  fwprintf(stdout, L"%-10s%s\n\n", L"readlink", L"Print the target of a symbolic link.");
+  ReadlinkUsage();
+  fwprintf(stdout, L"\n\n");
+
   fwprintf(stdout, L"%-15s%s\n\n", L"systeminfo", L"System information.");
   SystemInfoUsage();
   fwprintf(stdout, L"\n\n");

+ 224 - 0
hadoop-common-project/hadoop-common/src/main/winutils/readlink.c

@@ -0,0 +1,224 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+#include "winutils.h"
+
+
+//----------------------------------------------------------------------------
+// The Windows SDK does not include the definition of REPARSE_DATA_BUFFER. To
+// avoid adding a dependency on the WDK we define the structure here.
+// Reference: http://msdn.microsoft.com/en-us/library/ff552012.aspx
+//
+#pragma warning(push)
+#pragma warning(disable: 4201)  // nonstandard extension: nameless struct/union
+#pragma pack(push, 1)
+typedef struct _REPARSE_DATA_BUFFER {
+  ULONG  ReparseTag;
+  USHORT ReparseDataLength;
+  USHORT Reserved;
+  union {
+    struct {
+      USHORT SubstituteNameOffset;
+      USHORT SubstituteNameLength;
+      USHORT PrintNameOffset;
+      USHORT PrintNameLength;
+      ULONG  Flags;
+      WCHAR  PathBuffer[1];
+    } SymbolicLinkReparseBuffer;
+    struct {
+      USHORT SubstituteNameOffset;
+      USHORT SubstituteNameLength;
+      USHORT PrintNameOffset;
+      USHORT PrintNameLength;
+      WCHAR  PathBuffer[1];
+    } MountPointReparseBuffer;
+    struct {
+      UCHAR DataBuffer[1];
+    } GenericReparseBuffer;
+  };
+} REPARSE_DATA_BUFFER, *PREPARSE_DATA_BUFFER;
+#pragma pack(pop)
+#pragma warning(pop)
+
+
+//----------------------------------------------------------------------------
+// Function: Readlink
+//
+// Description:
+//  Prints the target of a symbolic link to stdout.
+//
+//  The return codes and output are modeled after the UNIX readlink command. 
+//  Hence no error messages are printed. Unlike the UNIX readlink, no options
+//  are accepted.
+//
+// Returns:
+//  0: on success
+//  1: on all errors
+//
+// Notes:
+//
+int Readlink(__in int argc, __in_ecount(argc) wchar_t *argv[])
+{
+  DWORD bytesReturned;
+  DWORD bufferSize = 1024;                  // Start off with a 1KB buffer.
+  HANDLE hFile = INVALID_HANDLE_VALUE;
+  PWSTR longLinkName = NULL;
+  PWCHAR printName = NULL;
+  PREPARSE_DATA_BUFFER pReparseData = NULL;
+  USHORT printNameLength;
+  USHORT printNameOffset;
+  DWORD result;
+  BOOLEAN succeeded = FALSE;
+  
+  if (argc != 2)
+  {
+    ReadlinkUsage();
+    goto Cleanup;
+  }
+
+  if (ConvertToLongPath(argv[1], &longLinkName) != ERROR_SUCCESS)
+  {
+    goto Cleanup;
+  }
+
+  // Get a handle to the link to issue the FSCTL.
+  // FILE_FLAG_BACKUP_SEMANTICS is needed to open directories.
+  // FILE_FLAG_OPEN_REPARSE_POINT disables normal reparse point processing
+  // so we can query the symlink.
+  //
+  hFile = CreateFileW(longLinkName,
+                      0,        // no rights needed to issue the FSCTL.
+                      FILE_SHARE_READ | FILE_SHARE_WRITE | FILE_SHARE_DELETE,
+                      NULL,
+                      OPEN_EXISTING,
+                      FILE_FLAG_BACKUP_SEMANTICS | FILE_FLAG_OPEN_REPARSE_POINT,
+                      NULL);
+
+  if (hFile == INVALID_HANDLE_VALUE) 
+  {
+    goto Cleanup;
+  }
+
+  for (;;)
+  {
+    pReparseData = (PREPARSE_DATA_BUFFER) LocalAlloc(LMEM_FIXED, bufferSize);
+
+    if (pReparseData == NULL) 
+    {
+      goto Cleanup;
+    }
+
+    // Issue the FSCTL to query the link information.
+    //
+    result = DeviceIoControl(hFile,
+                             FSCTL_GET_REPARSE_POINT,
+                             NULL,
+                             0,
+                             pReparseData,
+                             bufferSize,
+                             &bytesReturned,
+                             NULL);
+
+    if (result != 0)
+    {
+      // Success!
+      //
+      break;
+    }
+    else if ((GetLastError() == ERROR_INSUFFICIENT_BUFFER) ||
+             (GetLastError() == ERROR_MORE_DATA))
+    {
+      // Retry with a larger buffer.
+      //
+      LocalFree(pReparseData);
+      bufferSize *= 2;
+    }
+    else
+    {
+      // Unrecoverable error.
+      //
+      goto Cleanup;
+    }
+  }
+
+  if (pReparseData->ReparseTag != IO_REPARSE_TAG_SYMLINK) 
+  {
+    // Doesn't look like a symlink.
+    //
+    goto Cleanup;
+  }
+
+  // MSDN does not guarantee that the embedded paths in REPARSE_DATA_BUFFER
+  // will be NULL terminated. So we copy the string to a separate buffer and
+  // NULL terminate it before printing.
+  //
+  printNameLength = pReparseData->SymbolicLinkReparseBuffer.PrintNameLength;
+  printNameOffset = pReparseData->SymbolicLinkReparseBuffer.PrintNameOffset;
+  printName = (PWCHAR) LocalAlloc(LMEM_FIXED, printNameLength + 1);
+
+  if (printName == NULL) 
+  {
+    goto Cleanup;
+  }
+
+  memcpy(
+      printName,
+      pReparseData->SymbolicLinkReparseBuffer.PathBuffer + printNameOffset,
+      printNameLength);
+
+  printName[printNameLength / sizeof(WCHAR)] = L'\0';
+
+  fwprintf(stdout, L"%ls", printName);
+  succeeded = TRUE;
+
+Cleanup:
+  if (hFile != INVALID_HANDLE_VALUE) 
+  {
+    CloseHandle(hFile);
+  }
+
+  if (printName != NULL) 
+  {
+    LocalFree(printName);
+  }
+
+  if (pReparseData != NULL)
+  {
+    LocalFree(pReparseData);
+  }
+
+  if (longLinkName != NULL)
+  {
+    LocalFree(longLinkName);
+  }
+
+  return (succeeded ? EXIT_SUCCESS : EXIT_FAILURE);
+}
+
+void ReadlinkUsage()
+{
+    fwprintf(stdout, L"\
+Usage: readlink [LINKNAME]\n\
+Prints the target of a symbolic link\n\
+The output and returned error codes are similar to the UNIX\n\
+readlink command. However no options are accepted.\n\
+\n\
+0 is returned on success.\n\
+1 is returned for all errors.\n\
+\n");
+}
+

+ 11 - 0
hadoop-common-project/hadoop-common/src/main/winutils/symlink.c

@@ -60,6 +60,17 @@ int Symlink(__in int argc, __in_ecount(argc) wchar_t *argv[])
     goto SymlinkEnd;
   }
 
+  if (wcschr(longLinkName, L'/') != NULL || wcschr(longFileName, L'/') != NULL)
+  {
+    // Reject forward-slash separated paths as they result in unusable symlinks.
+    //
+    fwprintf(stderr,
+      L"Rejecting forward-slash separated path which would result in an "
+      L"unusable symlink: link = %s, target = %s\n", longLinkName, longFileName);
+    ret = FAILURE;
+    goto SymlinkEnd;
+  }
+
   // Check if the the process's access token has the privilege to create
   // symbolic links. Without this step, the call to CreateSymbolicLink() from
   // users have the privilege to create symbolic links will still succeed.

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/winutils/winutils.vcxproj

@@ -160,6 +160,7 @@
     </Link>
   </ItemDefinitionGroup>
   <ItemGroup>
+    <ClCompile Include="readlink.c" />
     <ClCompile Include="symlink.c" />
     <ClCompile Include="systeminfo.c" />
     <ClCompile Include="chmod.c" />
@@ -178,4 +179,4 @@
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />
   <ImportGroup Label="ExtensionTargets">
   </ImportGroup>
-</Project>
+</Project>

+ 34 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java

@@ -36,6 +36,7 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 import java.util.regex.Pattern;
+import static java.util.concurrent.TimeUnit.*;
 
 import junit.framework.TestCase;
 import static org.junit.Assert.assertArrayEquals;
@@ -43,6 +44,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration.IntegerRanges;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.net.NetUtils;
+import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
 import org.codehaus.jackson.map.ObjectMapper; 
 
 public class TestConfiguration extends TestCase {
@@ -51,9 +53,8 @@ public class TestConfiguration extends TestCase {
   final static String CONFIG = new File("./test-config.xml").getAbsolutePath();
   final static String CONFIG2 = new File("./test-config2.xml").getAbsolutePath();
   final static Random RAN = new Random();
-  final static boolean IBMJAVA = System.getProperty("java.vendor").contains("IBM"); 
   final static String XMLHEADER = 
-            IBMJAVA?"<?xml version=\"1.0\" encoding=\"UTF-8\"?><configuration>":
+            IBM_JAVA?"<?xml version=\"1.0\" encoding=\"UTF-8\"?><configuration>":
   "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"no\"?><configuration>";
 
   @Override
@@ -693,6 +694,37 @@ public class TestConfiguration extends TestCase {
     assertTrue(fail);
   }
 
+  public void testTimeDuration() {
+    Configuration conf = new Configuration(false);
+    conf.setTimeDuration("test.time.a", 7L, SECONDS);
+    assertEquals("7s", conf.get("test.time.a"));
+    assertEquals(0L, conf.getTimeDuration("test.time.a", 30, MINUTES));
+    assertEquals(7L, conf.getTimeDuration("test.time.a", 30, SECONDS));
+    assertEquals(7000L, conf.getTimeDuration("test.time.a", 30, MILLISECONDS));
+    assertEquals(7000000L,
+        conf.getTimeDuration("test.time.a", 30, MICROSECONDS));
+    assertEquals(7000000000L,
+        conf.getTimeDuration("test.time.a", 30, NANOSECONDS));
+    conf.setTimeDuration("test.time.b", 1, DAYS);
+    assertEquals("1d", conf.get("test.time.b"));
+    assertEquals(1, conf.getTimeDuration("test.time.b", 1, DAYS));
+    assertEquals(24, conf.getTimeDuration("test.time.b", 1, HOURS));
+    assertEquals(MINUTES.convert(1, DAYS),
+        conf.getTimeDuration("test.time.b", 1, MINUTES));
+
+    // check default
+    assertEquals(30L, conf.getTimeDuration("test.time.X", 30, SECONDS));
+    conf.set("test.time.X", "30");
+    assertEquals(30L, conf.getTimeDuration("test.time.X", 40, SECONDS));
+
+    for (Configuration.ParsedTimeDuration ptd :
+         Configuration.ParsedTimeDuration.values()) {
+      conf.setTimeDuration("test.time.unit", 1, ptd.unit());
+      assertEquals(1 + ptd.suffix(), conf.get("test.time.unit"));
+      assertEquals(1, conf.getTimeDuration("test.time.unit", 2, ptd.unit()));
+    }
+  }
+
   public void testPattern() throws IOException {
     out = new BufferedWriter(new FileWriter(CONFIG));
     startConfig();

+ 0 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextPermissionBase.java

@@ -202,8 +202,6 @@ public abstract class FileContextPermissionBase {
   
   
   /*
-   * Some filesystem like HDFS ignore the "x" bit if the permission.
-   * Others like localFs does not.
    * Override the method below if the file system being tested masks our
    * certain bits for file masks.
    */

+ 40 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFSFileContextSymlink.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileUtil;
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
 import static org.junit.Assert.*;
+import static org.junit.Assume.assumeTrue;
 import org.junit.Test;
 import org.junit.Before;
 
@@ -65,7 +66,44 @@ public class TestLocalFSFileContextSymlink extends FileContextSymlinkBaseTest {
     fc = FileContext.getLocalFSFileContext();
     super.setUp();
   }
-  
+
+  @Override
+  public void testCreateDanglingLink() throws IOException {
+    // Dangling symlinks are not supported on Windows local file system.
+    assumeTrue(!Path.WINDOWS);
+    super.testCreateDanglingLink();
+  }
+
+  @Override
+  public void testCreateFileViaDanglingLinkParent() throws IOException {
+    assumeTrue(!Path.WINDOWS);
+    super.testCreateFileViaDanglingLinkParent();
+  }
+
+  @Override
+  public void testOpenResolvesLinks() throws IOException {
+    assumeTrue(!Path.WINDOWS);
+    super.testOpenResolvesLinks();
+  }
+
+  @Override
+  public void testRecursiveLinks() throws IOException {
+    assumeTrue(!Path.WINDOWS);
+    super.testRecursiveLinks();
+  }
+
+  @Override
+  public void testRenameDirToDanglingSymlink() throws IOException {
+    assumeTrue(!Path.WINDOWS);
+    super.testRenameDirToDanglingSymlink();
+  }
+
+  @Override  
+  public void testStatDanglingLink() throws IOException {
+    assumeTrue(!Path.WINDOWS);
+    super.testStatDanglingLink();
+  }
+
   @Test
   /** lstat a non-existant file using a partially qualified path */
   public void testDanglingLinkFilePartQual() throws IOException {
@@ -87,6 +125,7 @@ public class TestLocalFSFileContextSymlink extends FileContextSymlinkBaseTest {
   @Test
   /** Stat and lstat a dangling link */
   public void testDanglingLink() throws IOException {
+    assumeTrue(!Path.WINDOWS);
     Path fileAbs  = new Path(testBaseDir1()+"/file");    
     Path fileQual = new Path(testURI().toString(), fileAbs);    
     Path link     = new Path(testBaseDir1()+"/linkToFile");

+ 87 - 27
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSecureIOUtils.java

@@ -17,73 +17,133 @@
  */
 package org.apache.hadoop.io;
 
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.nativeio.NativeIO;
-
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import static org.junit.Assume.*;
-import static org.junit.Assert.*;
-import java.io.IOException;
-import java.io.File;
-import java.io.FileOutputStream;
 
 public class TestSecureIOUtils {
-  private static String realOwner, realGroup; 
-  private static final File testFilePath =
-      new File(System.getProperty("test.build.data"), "TestSecureIOContext");
+
+  private static String realOwner, realGroup;
+  private static File testFilePathIs;
+  private static File testFilePathRaf;
+  private static File testFilePathFadis;
+  private static FileSystem fs;
 
   @BeforeClass
   public static void makeTestFile() throws Exception {
-    FileOutputStream fos = new FileOutputStream(testFilePath);
-    fos.write("hello".getBytes("UTF-8"));
-    fos.close();
-
     Configuration conf = new Configuration();
-    FileSystem rawFS = FileSystem.getLocal(conf).getRaw();
-    FileStatus stat = rawFS.getFileStatus(
-      new Path(testFilePath.toString()));
+    fs = FileSystem.getLocal(conf).getRaw();
+    testFilePathIs =
+        new File((new Path("target", TestSecureIOUtils.class.getSimpleName()
+            + "1")).toUri().getRawPath());
+    testFilePathRaf =
+        new File((new Path("target", TestSecureIOUtils.class.getSimpleName()
+            + "2")).toUri().getRawPath());
+    testFilePathFadis =
+        new File((new Path("target", TestSecureIOUtils.class.getSimpleName()
+            + "3")).toUri().getRawPath());
+    for (File f : new File[] { testFilePathIs, testFilePathRaf,
+        testFilePathFadis }) {
+      FileOutputStream fos = new FileOutputStream(f);
+      fos.write("hello".getBytes("UTF-8"));
+      fos.close();
+    }
+
+    FileStatus stat = fs.getFileStatus(
+        new Path(testFilePathIs.toString()));
+    // RealOwner and RealGroup would be same for all three files.
     realOwner = stat.getOwner();
     realGroup = stat.getGroup();
   }
 
-  @Test
+  @Test(timeout = 10000)
   public void testReadUnrestricted() throws IOException {
-    SecureIOUtils.openForRead(testFilePath, null, null).close();
+    SecureIOUtils.openForRead(testFilePathIs, null, null).close();
+    SecureIOUtils.openFSDataInputStream(testFilePathFadis, null, null).close();
+    SecureIOUtils.openForRandomRead(testFilePathRaf, "r", null, null).close();
   }
 
-  @Test
+  @Test(timeout = 10000)
   public void testReadCorrectlyRestrictedWithSecurity() throws IOException {
     SecureIOUtils
-      .openForRead(testFilePath, realOwner, realGroup).close();
+        .openForRead(testFilePathIs, realOwner, realGroup).close();
+    SecureIOUtils
+        .openFSDataInputStream(testFilePathFadis, realOwner, realGroup).close();
+    SecureIOUtils.openForRandomRead(testFilePathRaf, "r", realOwner, realGroup)
+        .close();
   }
 
-  @Test
+  @Test(timeout = 10000)
   public void testReadIncorrectlyRestrictedWithSecurity() throws IOException {
     // this will only run if libs are available
     assumeTrue(NativeIO.isAvailable());
 
     System.out.println("Running test with native libs...");
+    String invalidUser = "InvalidUser";
+
+    // We need to make sure that forceSecure.. call works only if
+    // the file belongs to expectedOwner.
 
+    // InputStream
     try {
       SecureIOUtils
-        .forceSecureOpenForRead(testFilePath, "invalidUser", null).close();
-      fail("Didn't throw expection for wrong ownership!");
+          .forceSecureOpenForRead(testFilePathIs, invalidUser, realGroup)
+          .close();
+      fail("Didn't throw expection for wrong user ownership!");
+
+    } catch (IOException ioe) {
+      // expected
+    }
+
+    // FSDataInputStream
+    try {
+      SecureIOUtils
+          .forceSecureOpenFSDataInputStream(testFilePathFadis, invalidUser,
+              realGroup).close();
+      fail("Didn't throw expection for wrong user ownership!");
+    } catch (IOException ioe) {
+      // expected
+    }
+
+    // RandomAccessFile
+    try {
+      SecureIOUtils
+          .forceSecureOpenForRandomRead(testFilePathRaf, "r", invalidUser,
+              realGroup).close();
+      fail("Didn't throw expection for wrong user ownership!");
     } catch (IOException ioe) {
       // expected
     }
   }
 
-  @Test
+  @Test(timeout = 10000)
   public void testCreateForWrite() throws IOException {
     try {
-      SecureIOUtils.createForWrite(testFilePath, 0777);
-      fail("Was able to create file at " + testFilePath);
+      SecureIOUtils.createForWrite(testFilePathIs, 0777);
+      fail("Was able to create file at " + testFilePathIs);
     } catch (SecureIOUtils.AlreadyExistsException aee) {
       // expected
     }
   }
+
+  @AfterClass
+  public static void removeTestFile() throws Exception {
+    // cleaning files
+    for (File f : new File[] { testFilePathIs, testFilePathRaf,
+        testFilePathFadis }) {
+      f.delete();
+    }
+  }
 }

+ 5 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java

@@ -25,6 +25,7 @@ import java.net.InetSocketAddress;
 import java.net.URISyntaxException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto;
 import org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto;
 import org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto;
 import org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto;
@@ -183,6 +184,8 @@ public class TestProtoBufRpc {
       RemoteException re = (RemoteException)e.getCause();
       RpcServerException rse = (RpcServerException) re
           .unwrapRemoteException(RpcServerException.class);
+      Assert.assertTrue(re.getErrorCode().equals(
+          RpcErrorCodeProto.ERROR_RPC_SERVER));
     }
   }
   
@@ -223,6 +226,8 @@ public class TestProtoBufRpc {
       Assert.assertTrue(re.getClassName().equals(
           URISyntaxException.class.getName()));
       Assert.assertTrue(re.getMessage().contains("testException"));
+      Assert.assertTrue(
+          re.getErrorCode().equals(RpcErrorCodeProto.ERROR_APPLICATION));
     }
   }
 }

+ 8 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureRequestProto;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureResponseProto;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolSignatureProto;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto;
 import org.apache.hadoop.net.NetUtils;
 import org.junit.After;
 import org.junit.Before;
@@ -310,9 +311,13 @@ System.out.println("echo int is NOT supported");
     try {
       proxy.echo(21);
       fail("The call must throw VersionMismatch exception");
-    } catch (IOException ex) {
-      Assert.assertTrue("Expected version mismatch but got " + ex.getMessage(), 
-          ex.getMessage().contains("VersionMismatch"));
+    } catch (RemoteException ex) {
+      Assert.assertEquals(RPC.VersionMismatch.class.getName(), 
+          ex.getClassName());
+      Assert.assertTrue(ex.getErrorCode().equals(
+          RpcErrorCodeProto.ERROR_RPC_VERSION_MISMATCH));
+    }  catch (IOException ex) {
+      fail("Expected version mismatch but got " + ex);
     }
   }
   

+ 118 - 25
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestWinUtils.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.util;
 
 import static org.junit.Assert.*;
+import static org.junit.Assume.assumeTrue;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -32,6 +33,8 @@ import org.apache.hadoop.fs.FileUtil;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import static org.junit.Assume.*;
+import static org.hamcrest.CoreMatchers.*;
 
 /**
  * Test cases for helper Windows winutils.exe utility.
@@ -44,6 +47,8 @@ public class TestWinUtils {
 
   @Before
   public void setUp() {
+    // Not supported on non-Windows platforms
+    assumeTrue(Shell.WINDOWS);
     TEST_DIR.mkdirs();
   }
 
@@ -70,11 +75,6 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testLs() throws IOException {
-    if (!Shell.WINDOWS) {
-      // Not supported on non-Windows platforms
-      return;
-    }
-
     final String content = "6bytes";
     final int contentSize = content.length();
     File testFile = new File(TEST_DIR, "file1");
@@ -104,11 +104,6 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testGroups() throws IOException {
-    if (!Shell.WINDOWS) {
-      // Not supported on non-Windows platforms
-      return;
-    }
-
     String currentUser = System.getProperty("user.name");
 
     // Verify that groups command returns information about the current user
@@ -229,11 +224,6 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testBasicChmod() throws IOException {
-    if (!Shell.WINDOWS) {
-      // Not supported on non-Windows platforms
-      return;
-    }
-
     // - Create a file.
     // - Change mode to 377 so owner does not have read permission.
     // - Verify the owner truly does not have the permissions to read.
@@ -285,11 +275,6 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testChmod() throws IOException {
-    if (!Shell.WINDOWS) {
-      // Not supported on non-Windows platforms
-      return;
-    }
-
     testChmodInternal("7", "-------rwx");
     testChmodInternal("70", "----rwx---");
     testChmodInternal("u-x,g+r,o=g", "-rw-r--r--");
@@ -322,11 +307,6 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testChown() throws IOException {
-    if (!Shell.WINDOWS) {
-      // Not supported on non-Windows platforms
-      return;
-    }
-
     File a = new File(TEST_DIR, "a");
     assertTrue(a.createNewFile());
     String username = System.getProperty("user.name");
@@ -349,4 +329,117 @@ public class TestWinUtils {
     assertTrue(a.delete());
     assertFalse(a.exists());
   }
+
+  @Test (timeout = 30000)
+  public void testSymlinkRejectsForwardSlashesInLink() throws IOException {
+    File newFile = new File(TEST_DIR, "file");
+    assertTrue(newFile.createNewFile());
+    String target = newFile.getPath();
+    String link = new File(TEST_DIR, "link").getPath().replaceAll("\\\\", "/");
+    try {
+      Shell.execCommand(Shell.WINUTILS, "symlink", link, target);
+      fail(String.format("did not receive expected failure creating symlink "
+        + "with forward slashes in link: link = %s, target = %s", link, target));
+    } catch (IOException e) {
+      LOG.info(
+        "Expected: Failed to create symlink with forward slashes in target");
+    }
+  }
+
+  @Test (timeout = 30000)
+  public void testSymlinkRejectsForwardSlashesInTarget() throws IOException {
+    File newFile = new File(TEST_DIR, "file");
+    assertTrue(newFile.createNewFile());
+    String target = newFile.getPath().replaceAll("\\\\", "/");
+    String link = new File(TEST_DIR, "link").getPath();
+    try {
+      Shell.execCommand(Shell.WINUTILS, "symlink", link, target);
+      fail(String.format("did not receive expected failure creating symlink "
+        + "with forward slashes in target: link = %s, target = %s", link, target));
+    } catch (IOException e) {
+      LOG.info(
+        "Expected: Failed to create symlink with forward slashes in target");
+    }
+  }
+
+  @Test (timeout = 30000)
+  public void testReadLink() throws IOException {
+    // Create TEST_DIR\dir1\file1.txt
+    //
+    File dir1 = new File(TEST_DIR, "dir1");
+    assertTrue(dir1.mkdirs());
+
+    File file1 = new File(dir1, "file1.txt");
+    assertTrue(file1.createNewFile());
+
+    File dirLink = new File(TEST_DIR, "dlink");
+    File fileLink = new File(TEST_DIR, "flink");
+
+    // Next create a directory symlink to dir1 and a file
+    // symlink to file1.txt.
+    //
+    Shell.execCommand(
+        Shell.WINUTILS, "symlink", dirLink.toString(), dir1.toString());
+    Shell.execCommand(
+        Shell.WINUTILS, "symlink", fileLink.toString(), file1.toString());
+
+    // Read back the two links and ensure we get what we expected.
+    //
+    String readLinkOutput = Shell.execCommand(Shell.WINUTILS,
+        "readlink",
+        dirLink.toString());
+    assertThat(readLinkOutput, equalTo(dir1.toString()));
+
+    readLinkOutput = Shell.execCommand(Shell.WINUTILS,
+        "readlink",
+        fileLink.toString());
+    assertThat(readLinkOutput, equalTo(file1.toString()));
+
+    // Try a few invalid inputs and verify we get an ExitCodeException for each.
+    //
+    try {
+      // No link name specified.
+      //
+      Shell.execCommand(Shell.WINUTILS, "readlink", "");
+      fail("Failed to get Shell.ExitCodeException when reading bad symlink");
+    } catch (Shell.ExitCodeException ece) {
+      assertThat(ece.getExitCode(), is(1));
+    }
+
+    try {
+      // Bad link name.
+      //
+      Shell.execCommand(Shell.WINUTILS, "readlink", "ThereIsNoSuchLink");
+      fail("Failed to get Shell.ExitCodeException when reading bad symlink");
+    } catch (Shell.ExitCodeException ece) {
+      assertThat(ece.getExitCode(), is(1));
+    }
+
+    try {
+      // Non-symlink directory target.
+      //
+      Shell.execCommand(Shell.WINUTILS, "readlink", dir1.toString());
+      fail("Failed to get Shell.ExitCodeException when reading bad symlink");
+    } catch (Shell.ExitCodeException ece) {
+      assertThat(ece.getExitCode(), is(1));
+    }
+
+    try {
+      // Non-symlink file target.
+      //
+      Shell.execCommand(Shell.WINUTILS, "readlink", file1.toString());
+      fail("Failed to get Shell.ExitCodeException when reading bad symlink");
+    } catch (Shell.ExitCodeException ece) {
+      assertThat(ece.getExitCode(), is(1));
+    }
+
+    try {
+      // Too many parameters.
+      //
+      Shell.execCommand(Shell.WINUTILS, "readlink", "a", "b");
+      fail("Failed to get Shell.ExitCodeException with bad parameters");
+    } catch (Shell.ExitCodeException ece) {
+      assertThat(ece.getExitCode(), is(1));
+    }
+  }
 }

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

@@ -22,6 +22,8 @@ Trunk (Unreleased)
     Azure environments. (See breakdown of tasks below for subtasks and
     contributors)
 
+    HDFS-4659 Support setting execution bit for regular files (Brandon Li via sanjay)
+
   IMPROVEMENTS
 
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.
@@ -620,6 +622,11 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4778. Fixes some issues that the first patch on HDFS-2576 missed.
     (ddas)
 
+    HDFS-4785. Concat operation does not remove concatenated files from
+    InodeMap. (suresh)
+
+    HDFS-4784. NPE in FSDirectory.resolvePath(). (Brandon Li via suresh)
+
 Release 2.0.4-alpha - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 8 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -1234,6 +1234,7 @@ public class FSDirectory implements Closeable {
       
       nodeToRemove.setBlocks(null);
       trgParent.removeChild(nodeToRemove, trgLatestSnapshot, null);
+      inodeMap.remove(nodeToRemove);
       count++;
     }
     
@@ -2684,14 +2685,18 @@ public class FSDirectory implements Closeable {
     try {
       id = Long.valueOf(inodeId);
     } catch (NumberFormatException e) {
-      throw new FileNotFoundException(
-          "File for given inode path does not exist: " + src);
+      throw new FileNotFoundException("Invalid inode path: " + src);
     }
     if (id == INodeId.ROOT_INODE_ID && pathComponents.length == 4) {
       return Path.SEPARATOR;
     }
+    INode inode = fsd.getInode(id);
+    if (inode == null) {
+      throw new FileNotFoundException(
+          "File for given inode path does not exist: " + src);
+    }
     StringBuilder path = id == INodeId.ROOT_INODE_ID ? new StringBuilder()
-        : new StringBuilder(fsd.getInode(id).getFullPathName());
+        : new StringBuilder(inode.getFullPathName());
     for (int i = 4; i < pathComponents.length; i++) {
       path.append(Path.SEPARATOR).append(DFSUtil.bytes2String(pathComponents[i]));
     }

+ 0 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -66,8 +66,6 @@ public class INodeFile extends INodeWithAdditionalFields implements BlockCollect
     return inode.asFile();
   }
 
-  static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
-
   /** Format: [16 bits for replication][48 bits for PreferredBlockSize] */
   private static class HeaderFormat {
     /** Number of bits for Block size */
@@ -167,27 +165,6 @@ public class INodeFile extends INodeWithAdditionalFields implements BlockCollect
     }
   }
 
-  /**
-   * Set the {@link FsPermission} of this {@link INodeFile}.
-   * Since this is a file,
-   * the {@link FsAction#EXECUTE} action, if any, is ignored.
-   */
-  @Override
-  final void setPermission(FsPermission permission) {
-    super.setPermission(permission.applyUMask(UMASK));
-  }
-
-  /**
-   * Set the {@link FsPermission} of this {@link INodeFile}.
-   * Since this is a file,
-   * the {@link FsAction#EXECUTE} action, if any, is ignored.
-   */
-  @Override
-  final INode setPermission(FsPermission permission, Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
-    return super.setPermission(permission.applyUMask(UMASK), latest, inodeMap);
-  }
-
   /** @return the replication factor of the file. */
   public final short getFileReplication(Snapshot snapshot) {
     if (snapshot != null) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java

@@ -63,7 +63,7 @@ public class INodeFileUnderConstruction extends INodeFile implements MutableBloc
                              String clientMachine,
                              DatanodeDescriptor clientNode) {
     this(id, null, replication, modTime, preferredBlockSize, BlockInfo.EMPTY_ARRAY,
-        permissions.applyUMask(UMASK), clientName, clientMachine, clientNode);
+        permissions, clientName, clientMachine, clientNode);
   }
 
   INodeFileUnderConstruction(long id,

+ 0 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsPermission.java

@@ -67,15 +67,4 @@ public class TestFcHdfsPermission extends FileContextPermissionBase {
   public void tearDown() throws Exception {
     super.tearDown();
   }
-  
-  /*
-   * HDFS ignore the "x" bit if the permission.
-   * 
-   */
-  static final FsPermission FILE_MASK_IGNORE_X_BIT = 
-    new FsPermission((short) ~0666);
-  @Override
-  FsPermission getFileMask() {
-    return FILE_MASK_IGNORE_X_BIT;
-  }
 }

+ 2 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java

@@ -141,16 +141,7 @@ public class TestStickyBit {
     hdfs.setPermission(p1, new FsPermission(withSB));
     assertTrue(hdfs.getFileStatus(p1).getPermission().getStickyBit());
 
-    // However, while you can set the sticky bit on files, it has no effect,
-    // following the linux/unix model:
-    //
-    // [user@host test]$ ls -alh
-    // -rw-r--r-- 1 user users 0 Dec 31 01:46 aFile
-    // [user@host test]$ chmod +t aFile
-    // [user@host test]$ ls -alh
-    // -rw-r--r-- 1 user users 0 Dec 31 01:46 aFile
-
-    // Write a file to the fs, try to set its sticky bit, expect to be ignored
+    // Write a file to the fs, try to set its sticky bit
     Path f = new Path(baseDir, "somefile");
     writeFile(hdfs, f);
     assertFalse(hdfs.getFileStatus(f).getPermission().getStickyBit());
@@ -159,7 +150,7 @@ public class TestStickyBit {
 
     hdfs.setPermission(f, new FsPermission(withSB));
 
-    assertFalse(hdfs.getFileStatus(f).getPermission().getStickyBit());
+    assertTrue(hdfs.getFileStatus(f).getPermission().getStickyBit());
   }
 
   @Test

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java

@@ -63,7 +63,6 @@ public class TestDFSPermission {
 
   final private static short MAX_PERMISSION = 511;
   final private static short DEFAULT_UMASK = 022;
-  final private static short FILE_MASK = 0666;
   final private static FsPermission DEFAULT_PERMISSION = 
     FsPermission.createImmutable((short) 0777);
   final static private int NUM_TEST_PERMISSIONS = 
@@ -171,7 +170,7 @@ public class TestDFSPermission {
     uMask = (short) 0111;
     initFileSystem(uMask);
     fs.setPermission(FILE_DIR_PATH, new FsPermission(uMask));
-    short expectedPermission = (short) ((op == OpType.CREATE) ? 0 : 0111);
+    short expectedPermission = (short) 0111;
     checkPermission(FILE_DIR_PATH, expectedPermission, true);
     closeFileSystem();
 
@@ -234,7 +233,7 @@ public class TestDFSPermission {
 
     // get the expected permission
     short expectedPermission = (op == OpType.CREATE) ? (short) (~umask
-        & permissionNum & FILE_MASK) : (short) (~umask & permissionNum);
+        & permissionNum) : (short) (~umask & permissionNum);
 
     // check if permission is correctly set
     checkPermission(name, expectedPermission, delete);

+ 31 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -423,8 +423,27 @@ public class TestINodeFile {
       assertTrue(fs.delete(renamedPath, true));
       inodeCount -= 2;
       assertEquals(inodeCount, fsn.dir.getInodeMapSize());
+      
+      // Create and concat /test/file1 /test/file2
+      // Create /test1/file1 and /test1/file2
+      String file1 = "/test1/file1";
+      String file2 = "/test1/file2";
+      DFSTestUtil.createFile(fs, new Path(file1), 512, (short) 1, 0);
+      DFSTestUtil.createFile(fs, new Path(file2), 512, (short) 1, 0);
+      inodeCount += 3; // test1, file1 and file2 are created
+      expectedLastInodeId += 3;
+      assertEquals(inodeCount, fsn.dir.getInodeMapSize());
+      assertEquals(expectedLastInodeId, fsn.getLastInodeId());
+      // Concat the /test1/file1 /test1/file2 into /test1/file2
+      nnrpc.concat(file2, new String[] {file1});
+      inodeCount--; // file1 and file2 are concatenated to file2
+      assertEquals(inodeCount, fsn.dir.getInodeMapSize());
+      assertEquals(expectedLastInodeId, fsn.getLastInodeId());
+      assertTrue(fs.delete(new Path("/test1"), true));
+      inodeCount -= 2; // test1 and file2 is deleted
+      assertEquals(inodeCount, fsn.dir.getInodeMapSize());
 
-      // Make sure empty editlog can be handled
+      // Make sure editlog is loaded correctly 
       cluster.restartNameNode();
       cluster.waitActive();
       fsn = cluster.getNamesystem();
@@ -862,6 +881,17 @@ public class TestINodeFile {
     components = INode.getPathComponents(testPath);
     resolvedPath = FSDirectory.resolvePath(testPath, components, fsd);
     assertEquals(testPath, resolvedPath);
+    
+    // Test path with nonexistent(deleted or wrong id) inode
+    Mockito.doReturn(null).when(fsd).getInode(Mockito.anyLong());
+    testPath = "/.reserved/.inodes/1234";
+    components = INode.getPathComponents(testPath);
+    try {
+      String realPath = FSDirectory.resolvePath(testPath, components, fsd);
+      fail("Path should not be resolved:" + realPath);
+    } catch (IOException e) {
+      assertTrue(e instanceof FileNotFoundException);
+    }
   }
   
   /**

+ 24 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java

@@ -190,14 +190,28 @@ public class TestPermission {
       catch(java.io.FileNotFoundException e) {
         LOG.info("GOOD: got " + e);
       }
+      
+      // make sure nn can take user specified permission (with default fs
+      // permission umask applied)
+      FSDataOutputStream out = nnfs.create(CHILD_FILE1, new FsPermission(
+          (short) 0777), true, 1024, (short) 1, 1024, null);
+      FileStatus status = nnfs.getFileStatus(CHILD_FILE1);
+      // FS_PERMISSIONS_UMASK_DEFAULT is 0022
+      assertTrue(status.getPermission().toString().equals("rwxr-xr-x"));
+      nnfs.delete(CHILD_FILE1, false);
+      
       // following dir/file creations are legal
       nnfs.mkdirs(CHILD_DIR1);
-      FSDataOutputStream out = nnfs.create(CHILD_FILE1);
+      out = nnfs.create(CHILD_FILE1);
+      status = nnfs.getFileStatus(CHILD_FILE1);
+      assertTrue(status.getPermission().toString().equals("rw-r--r--"));
       byte data[] = new byte[FILE_LEN];
       RAN.nextBytes(data);
       out.write(data);
       out.close();
       nnfs.setPermission(CHILD_FILE1, new FsPermission("700"));
+      status = nnfs.getFileStatus(CHILD_FILE1);
+      assertTrue(status.getPermission().toString().equals("rwx------"));
 
       // following read is legal
       byte dataIn[] = new byte[FILE_LEN];
@@ -208,6 +222,15 @@ public class TestPermission {
         assertEquals(data[i], dataIn[i]);
       }
 
+      // test execution bit support for files
+      nnfs.setPermission(CHILD_FILE1, new FsPermission("755"));
+      status = nnfs.getFileStatus(CHILD_FILE1);
+      assertTrue(status.getPermission().toString().equals("rwxr-xr-x"));
+      nnfs.setPermission(CHILD_FILE1, new FsPermission("744"));
+      status = nnfs.getFileStatus(CHILD_FILE1);
+      assertTrue(status.getPermission().toString().equals("rwxr--r--"));
+      nnfs.setPermission(CHILD_FILE1, new FsPermission("700"));
+      
       ////////////////////////////////////////////////////////////////
       // test illegal file/dir creation
       UserGroupInformation userGroupInfo = 

+ 147 - 147
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml

@@ -8557,7 +8557,7 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file1</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -8786,11 +8786,11 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -8820,11 +8820,11 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -8854,11 +8854,11 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -8888,11 +8888,11 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -8981,19 +8981,19 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file3</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file3</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file4</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file4</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -9014,19 +9014,19 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file3</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file3</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file4</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file4</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -9157,19 +9157,19 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file3</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file3</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file4</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/file4</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -9190,19 +9190,19 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file3</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file3</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file4</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*file4</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -9695,15 +9695,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -9711,15 +9711,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -9750,15 +9750,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -9766,15 +9766,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -9805,15 +9805,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -9821,15 +9821,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -9860,15 +9860,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -9876,15 +9876,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -9915,15 +9915,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -9931,15 +9931,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -9970,15 +9970,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -9986,15 +9986,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -10025,15 +10025,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -10041,15 +10041,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*/dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -10080,15 +10080,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -10096,15 +10096,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -10194,7 +10194,7 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file1</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -10310,11 +10310,11 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -10344,11 +10344,11 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -10405,19 +10405,19 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file3</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file3</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file4</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file4</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -10493,19 +10493,19 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file3</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file3</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file4</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///file4</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -10767,15 +10767,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -10783,15 +10783,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -10822,15 +10822,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -10838,15 +10838,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -10877,15 +10877,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -10893,15 +10893,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -10932,15 +10932,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -10948,15 +10948,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*hdfs:///dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -11013,7 +11013,7 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file1</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -11129,11 +11129,11 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -11163,11 +11163,11 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -11224,19 +11224,19 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file3</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file3</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file4</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file4</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -11312,19 +11312,19 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file3</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file3</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file4</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/file4</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -11586,15 +11586,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -11602,15 +11602,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -11641,15 +11641,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -11657,15 +11657,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -11696,15 +11696,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -11712,15 +11712,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -11751,15 +11751,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir0/file0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -11767,15 +11767,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file1</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file2</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/dir1/file2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-rw-rw-rw-( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/file0</expected-output>
+          <expected-output>^-rwxrwxrwx( )*1( )*[a-zA-z0-9]*( )*supergroup( )*0( )*[0-9]{4,}-[0-9]{2,}-[0-9]{2,} [0-9]{2,}:[0-9]{2,}( )*NAMENODE/dir2/file0</expected-output>
         </comparator>
       </comparators>
     </test>

+ 14 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -215,6 +215,12 @@ Release 2.0.5-beta - UNRELEASED
     MAPREDUCE-5069. add concrete common implementations of
     CombineFileInputFormat (Sangjin Lee via bobby)
 
+    MAPREDUCE-5145. Changed default max-attempts to be more than one for MR jobs
+    inline with YARN. (Zhijie Shen via vinodkv)
+
+    MAPREDUCE-5036. Default shuffle handler port should not be 8080.
+    (Sandy Ryza via tomwhite)
+
   OPTIMIZATIONS
 
     MAPREDUCE-4974. Optimising the LineRecordReader initialize() method 
@@ -366,6 +372,14 @@ Release 2.0.5-beta - UNRELEASED
     versionInfo has parantheses like when running on a git checkout. (Hitesh Shah
     via vinodkv)
 
+    MAPREDUCE-5193. A few MR tests use block sizes which are smaller than the
+    default minimum block size. (Andrew Wang via atm)
+
+    MAPREDUCE-5205. Fixed MR App to load tokens correctly. (vinodkv)
+
+    MAPREDUCE-5204. Handling YarnRemoteException separately from IOException in
+    MR app after YARN-629. (Xuan Gong via vinodkv)
+
 Release 2.0.4-alpha - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -1360,8 +1360,13 @@ public class MRAppMaster extends CompositeService {
       final YarnConfiguration conf, String jobUserName) throws IOException,
       InterruptedException {
     UserGroupInformation.setConfiguration(conf);
+    // Security framework already loaded the tokens into current UGI, just use
+    // them
+    Credentials credentials =
+        UserGroupInformation.getCurrentUser().getCredentials();
     UserGroupInformation appMasterUgi = UserGroupInformation
         .createRemoteUser(jobUserName);
+    appMasterUgi.addCredentials(credentials);
     appMasterUgi.doAs(new PrivilegedExceptionAction<Object>() {
       @Override
       public Object run() throws Exception {

+ 128 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java

@@ -17,18 +17,29 @@
  */
 package org.apache.hadoop.mapreduce.v2.app;
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
 
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import junit.framework.Assert;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.TypeConverter;
@@ -41,13 +52,22 @@ import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -55,13 +75,20 @@ import org.junit.Test;
 public class TestMRAppMaster {
   private static final Log LOG = LogFactory.getLog(TestMRAppMaster.class);
   static String stagingDir = "staging/";
+  private static FileContext localFS = null;
+  private static final File testDir = new File("target",
+    TestMRAppMaster.class.getName() + "-tmpDir").getAbsoluteFile();
   
   @BeforeClass
-  public static void setup() {
+  public static void setup() throws AccessControlException,
+      FileNotFoundException, IllegalArgumentException, IOException {
     //Do not error out if metrics are inited multiple times
     DefaultMetricsSystem.setMiniClusterMode(true);
     File dir = new File(stagingDir);
     stagingDir = dir.getAbsolutePath();
+    localFS = FileContext.getLocalFSFileContext();
+    localFS.delete(new Path(testDir.getAbsolutePath()), true);
+    testDir.mkdir();
   }
   
   @Before
@@ -269,6 +296,100 @@ public class TestMRAppMaster {
     }
   }
 
+  // A dirty hack to modify the env of the current JVM itself - Dirty, but
+  // should be okay for testing.
+  @SuppressWarnings({ "rawtypes", "unchecked" })
+  private static void setNewEnvironmentHack(Map<String, String> newenv)
+      throws Exception {
+    try {
+      Class<?> cl = Class.forName("java.lang.ProcessEnvironment");
+      Field field = cl.getDeclaredField("theEnvironment");
+      field.setAccessible(true);
+      Map<String, String> env = (Map<String, String>) field.get(null);
+      env.clear();
+      env.putAll(newenv);
+      Field ciField = cl.getDeclaredField("theCaseInsensitiveEnvironment");
+      ciField.setAccessible(true);
+      Map<String, String> cienv = (Map<String, String>) ciField.get(null);
+      cienv.clear();
+      cienv.putAll(newenv);
+    } catch (NoSuchFieldException e) {
+      Class[] classes = Collections.class.getDeclaredClasses();
+      Map<String, String> env = System.getenv();
+      for (Class cl : classes) {
+        if ("java.util.Collections$UnmodifiableMap".equals(cl.getName())) {
+          Field field = cl.getDeclaredField("m");
+          field.setAccessible(true);
+          Object obj = field.get(env);
+          Map<String, String> map = (Map<String, String>) obj;
+          map.clear();
+          map.putAll(newenv);
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testMRAppMasterCredentials() throws Exception {
+
+    Logger rootLogger = LogManager.getRootLogger();
+    rootLogger.setLevel(Level.DEBUG);
+
+    // Simulate credentials passed to AM via client->RM->NM
+    Credentials credentials = new Credentials();
+    byte[] identifier = "MyIdentifier".getBytes();
+    byte[] password = "MyPassword".getBytes();
+    Text kind = new Text("MyTokenKind");
+    Text service = new Text("host:port");
+    Token<? extends TokenIdentifier> myToken =
+        new Token<TokenIdentifier>(identifier, password, kind, service);
+    Text tokenAlias = new Text("myToken");
+    credentials.addToken(tokenAlias, myToken);
+    Token<? extends TokenIdentifier> storedToken =
+        credentials.getToken(tokenAlias);
+
+    YarnConfiguration conf = new YarnConfiguration();
+
+    Path tokenFilePath = new Path(testDir.getAbsolutePath(), "tokens-file");
+    Map<String, String> newEnv = new HashMap<String, String>();
+    newEnv.put(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION, tokenFilePath
+      .toUri().getPath());
+    setNewEnvironmentHack(newEnv);
+    credentials.writeTokenStorageFile(tokenFilePath, conf);
+
+    ApplicationId appId = BuilderUtils.newApplicationId(12345, 56);
+    ApplicationAttemptId applicationAttemptId =
+        BuilderUtils.newApplicationAttemptId(appId, 1);
+    ContainerId containerId =
+        BuilderUtils.newContainerId(applicationAttemptId, 546);
+    String userName = UserGroupInformation.getCurrentUser().getShortUserName();
+
+    // Create staging dir, so MRAppMaster doesn't barf.
+    File stagingDir =
+        new File(MRApps.getStagingAreaDir(conf, userName).toString());
+    stagingDir.mkdirs();
+
+    // Set login-user to null as that is how real world MRApp starts with.
+    // This is null is the reason why token-file is read by UGI.
+    UserGroupInformation.setLoginUser(null);
+
+    MRAppMasterTest appMaster =
+        new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
+          System.currentTimeMillis(), 1, false, true);
+    MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
+
+    // Now validate the credentials
+    Credentials appMasterCreds = appMaster.credentials;
+    Assert.assertNotNull(appMasterCreds);
+    Token<? extends TokenIdentifier> usedToken =
+        appMasterCreds.getToken(tokenAlias);
+    Assert.assertNotNull(usedToken);
+    Assert
+      .assertEquals("MyIdentifier", new String(storedToken.getIdentifier()));
+    Assert.assertEquals("MyPassword", new String(storedToken.getPassword()));
+    Assert.assertEquals("MyTokenKind", storedToken.getKind().toString());
+    Assert.assertEquals("host:port", storedToken.getService().toString());
+  }
 }
 
 class MRAppMasterTest extends MRAppMaster {
@@ -280,6 +401,7 @@ class MRAppMasterTest extends MRAppMaster {
   ContainerAllocator mockContainerAllocator;
   CommitterEventHandler mockCommitterEventHandler;
   RMHeartbeatHandler mockRMHeartbeatHandler;
+  Credentials credentials;
 
   public MRAppMasterTest(ApplicationAttemptId applicationAttemptId,
       ContainerId containerId, String host, int port, int httpPort,
@@ -338,7 +460,9 @@ class MRAppMasterTest extends MRAppMaster {
   public void start() {
     if (overrideStart) {
       try {
-        String user = UserGroupInformation.getCurrentUser().getShortUserName();
+        UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+        String user = ugi.getShortUserName();
+        this.credentials = ugi.getCredentials();
         stagingDirPath = MRApps.getStagingAreaDir(conf, user);
       } catch (Exception e) {
         fail(e.getMessage());

+ 5 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java

@@ -90,6 +90,7 @@ import org.junit.Test;
      JobId jobid = recordFactory.newRecordInstance(JobId.class);
      jobid.setAppId(appId);
      ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
+     Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
      MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
          JobStateInternal.RUNNING, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
      appMaster.init(conf);
@@ -116,8 +117,9 @@ import org.junit.Test;
      appId.setId(0);
      attemptId.setApplicationId(appId);
      ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
+     Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
      MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
-         JobStateInternal.REBOOT, 4);
+         JobStateInternal.REBOOT, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
      appMaster.init(conf);
      appMaster.start();
      //shutdown the job, not the lastRetry
@@ -144,7 +146,7 @@ import org.junit.Test;
      attemptId.setApplicationId(appId);
      ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
      MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
-         JobStateInternal.REBOOT, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
+         JobStateInternal.REBOOT, 1); //no retry
      appMaster.init(conf);
      appMaster.start();
      //shutdown the job, is lastRetry
@@ -201,8 +203,7 @@ import org.junit.Test;
      JobId jobid = recordFactory.newRecordInstance(JobId.class);
      jobid.setAppId(appId);
      ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
-     MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
-         MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
+     MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc, 1); //no retry
      appMaster.init(conf);
      //simulate the process being killed
      MRAppMaster.MRAppMasterShutdownHook hook = 

+ 15 - 15
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/impl/pb/client/MRClientProtocolPBClientImpl.java

@@ -98,7 +98,7 @@ import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequ
 import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
 import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
-import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
 
 import com.google.protobuf.ServiceException;
 
@@ -133,7 +133,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol,
     try {
       return new GetJobReportResponsePBImpl(proxy.getJobReport(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -144,7 +144,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol,
     try {
       return new GetTaskReportResponsePBImpl(proxy.getTaskReport(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -155,7 +155,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol,
     try {
       return new GetTaskAttemptReportResponsePBImpl(proxy.getTaskAttemptReport(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -166,7 +166,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol,
     try {
       return new GetCountersResponsePBImpl(proxy.getCounters(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -177,7 +177,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol,
     try {
       return new GetTaskAttemptCompletionEventsResponsePBImpl(proxy.getTaskAttemptCompletionEvents(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -188,7 +188,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol,
     try {
       return new GetTaskReportsResponsePBImpl(proxy.getTaskReports(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -199,7 +199,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol,
     try {
       return new GetDiagnosticsResponsePBImpl(proxy.getDiagnostics(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
   
@@ -212,7 +212,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol,
       return new GetDelegationTokenResponsePBImpl(proxy.getDelegationToken(
           null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
   
@@ -223,7 +223,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol,
     try {
       return new KillJobResponsePBImpl(proxy.killJob(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -234,7 +234,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol,
     try {
       return new KillTaskResponsePBImpl(proxy.killTask(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -245,7 +245,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol,
     try {
       return new KillTaskAttemptResponsePBImpl(proxy.killTaskAttempt(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -256,7 +256,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol,
     try {
       return new FailTaskAttemptResponsePBImpl(proxy.failTaskAttempt(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
  
@@ -269,7 +269,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol,
       return new RenewDelegationTokenResponsePBImpl(proxy.renewDelegationToken(
           null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -283,7 +283,7 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol,
           proxy.cancelDelegationToken(null, requestProto));
 
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 }

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/security/MRDelegationTokenRenewer.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenRenewer;
 import org.apache.hadoop.yarn.api.records.DelegationToken;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
@@ -67,6 +68,8 @@ public class MRDelegationTokenRenewer extends TokenRenewer {
           .newRecord(RenewDelegationTokenRequest.class);
       request.setDelegationToken(dToken);
       return histProxy.renewDelegationToken(request).getNextExpirationTime();
+    } catch (YarnRemoteException e) {
+      throw new IOException(e);
     } finally {
       stopHistoryProxy(histProxy);
     }
@@ -88,6 +91,8 @@ public class MRDelegationTokenRenewer extends TokenRenewer {
           .newRecord(CancelDelegationTokenRequest.class);
       request.setDelegationToken(dToken);
       histProxy.cancelDelegationToken(request);
+    } catch (YarnRemoteException e) {
+      throw new IOException(e);
     } finally {
       stopHistoryProxy(histProxy);
     }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java

@@ -677,6 +677,6 @@ public interface MRJobConfig {
    */
   public static final String MR_AM_MAX_ATTEMPTS = "mapreduce.am.max-attempts";
 
-  public static final int DEFAULT_MR_AM_MAX_ATTEMPTS = 1;
+  public static final int DEFAULT_MR_AM_MAX_ATTEMPTS = 2;
   
 }

+ 4 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml

@@ -773,7 +773,7 @@
 
 <property>
   <name>mapreduce.shuffle.port</name>
-  <value>8080</value>
+  <value>11000</value>
   <description>Default port that the ShuffleHandler will run on. ShuffleHandler 
    is a service run at the NodeManager to facilitate transfers of intermediate 
    Map outputs to requesting Reducers.
@@ -816,10 +816,11 @@
 
 <property>
   <name>mapreduce.am.max-attempts</name>
-  <value>1</value>
+  <value>2</value>
   <description>The maximum number of application attempts. It is a
   application-specific setting. It should not be larger than the global number
-  set by resourcemanager. Otherwise, it will be override.</description>
+  set by resourcemanager. Otherwise, it will be override. The default number is
+  set to 2, to allow at least one retry for AM.</description>
 </property>
 
 <!-- Job Notification Configuration -->

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java

@@ -302,13 +302,13 @@ public class ClientServiceDelegate {
         return methodOb.invoke(getProxy(), args);
       } catch (YarnRemoteException yre) {
         LOG.warn("Exception thrown by remote end.", yre);
-        throw yre;
+        throw new IOException(yre);
       } catch (InvocationTargetException e) {
         if (e.getTargetException() instanceof YarnRemoteException) {
           LOG.warn("Error from remote end: " + e
               .getTargetException().getLocalizedMessage());
           LOG.debug("Tracing remote error ", e.getTargetException());
-          throw (YarnRemoteException) e.getTargetException();
+          throw new IOException(e.getTargetException());
         }
         LOG.debug("Failed to contact AM/History for job " + jobId + 
             " retrying..", e.getTargetException());

+ 66 - 21
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.client.YarnClientImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.util.ProtoUtils;
 
 public class ResourceMgrDelegate extends YarnClientImpl {
@@ -65,11 +66,19 @@ public class ResourceMgrDelegate extends YarnClientImpl {
 
   public TaskTrackerInfo[] getActiveTrackers() throws IOException,
       InterruptedException {
-    return TypeConverter.fromYarnNodes(super.getNodeReports());
+    try {
+      return TypeConverter.fromYarnNodes(super.getNodeReports());
+    } catch (YarnRemoteException e) {
+      throw new IOException(e);
+    }
   }
 
   public JobStatus[] getAllJobs() throws IOException, InterruptedException {
-    return TypeConverter.fromYarnApps(super.getApplicationList(), this.conf);
+    try {
+      return TypeConverter.fromYarnApps(super.getApplicationList(), this.conf);
+    } catch (YarnRemoteException e) {
+      throw new IOException(e);
+    }
   }
 
   public TaskTrackerInfo[] getBlacklistedTrackers() throws IOException,
@@ -81,11 +90,17 @@ public class ResourceMgrDelegate extends YarnClientImpl {
 
   public ClusterMetrics getClusterMetrics() throws IOException,
       InterruptedException {
-    YarnClusterMetrics metrics = super.getYarnClusterMetrics();
-    ClusterMetrics oldMetrics = new ClusterMetrics(1, 1, 1, 1, 1, 1, 
-        metrics.getNumNodeManagers() * 10, metrics.getNumNodeManagers() * 2, 1,
-        metrics.getNumNodeManagers(), 0, 0);
-    return oldMetrics;
+    try {
+      YarnClusterMetrics metrics = super.getYarnClusterMetrics();
+      ClusterMetrics oldMetrics =
+          new ClusterMetrics(1, 1, 1, 1, 1, 1,
+              metrics.getNumNodeManagers() * 10,
+              metrics.getNumNodeManagers() * 2, 1,
+              metrics.getNumNodeManagers(), 0, 0);
+      return oldMetrics;
+    } catch (YarnRemoteException e) {
+      throw new IOException(e);
+    }
   }
 
   InetSocketAddress getConnectAddress() {
@@ -95,8 +110,12 @@ public class ResourceMgrDelegate extends YarnClientImpl {
   @SuppressWarnings("rawtypes")
   public Token getDelegationToken(Text renewer) throws IOException,
       InterruptedException {
-    return ProtoUtils.convertFromProtoFormat(
-      super.getRMDelegationToken(renewer), rmAddress);
+    try {
+      return ProtoUtils.convertFromProtoFormat(
+        super.getRMDelegationToken(renewer), rmAddress);
+    } catch (YarnRemoteException e) {
+      throw new IOException(e);
+    }
   }
 
   public String getFilesystemName() throws IOException, InterruptedException {
@@ -104,36 +123,62 @@ public class ResourceMgrDelegate extends YarnClientImpl {
   }
 
   public JobID getNewJobID() throws IOException, InterruptedException {
-    this.application = super.getNewApplication();
-    this.applicationId = this.application.getApplicationId();
-    return TypeConverter.fromYarn(applicationId);
+    try {
+      this.application = super.getNewApplication();
+      this.applicationId = this.application.getApplicationId();
+      return TypeConverter.fromYarn(applicationId);
+    } catch (YarnRemoteException e) {
+      throw new IOException(e);
+    }
   }
 
   public QueueInfo getQueue(String queueName) throws IOException,
   InterruptedException {
-    org.apache.hadoop.yarn.api.records.QueueInfo queueInfo =
-        super.getQueueInfo(queueName);
-    return (queueInfo == null) ? null : TypeConverter.fromYarn(queueInfo, conf);
+    try {
+      org.apache.hadoop.yarn.api.records.QueueInfo queueInfo =
+          super.getQueueInfo(queueName);
+      return (queueInfo == null) ? null : TypeConverter.fromYarn(queueInfo,
+          conf);
+    } catch (YarnRemoteException e) {
+      throw new IOException(e);
+    }
   }
 
   public QueueAclsInfo[] getQueueAclsForCurrentUser() throws IOException,
       InterruptedException {
-    return TypeConverter.fromYarnQueueUserAclsInfo(super
-      .getQueueAclsInfo());
+    try {
+      return TypeConverter.fromYarnQueueUserAclsInfo(super
+        .getQueueAclsInfo());
+    } catch (YarnRemoteException e) {
+      throw new IOException(e);
+    }
   }
 
   public QueueInfo[] getQueues() throws IOException, InterruptedException {
-    return TypeConverter.fromYarnQueueInfo(super.getAllQueues(), this.conf);
+    try {
+      return TypeConverter.fromYarnQueueInfo(super.getAllQueues(), this.conf);
+    } catch (YarnRemoteException e) {
+      throw new IOException(e);
+    }
   }
 
   public QueueInfo[] getRootQueues() throws IOException, InterruptedException {
-    return TypeConverter.fromYarnQueueInfo(super.getRootQueueInfos(), this.conf);
+    try {
+      return TypeConverter.fromYarnQueueInfo(super.getRootQueueInfos(),
+          this.conf);
+    } catch (YarnRemoteException e) {
+      throw new IOException(e);
+    }
   }
 
   public QueueInfo[] getChildQueues(String parent) throws IOException,
       InterruptedException {
-    return TypeConverter.fromYarnQueueInfo(super.getChildQueueInfos(parent),
-      this.conf);
+    try {
+      return TypeConverter.fromYarnQueueInfo(super.getChildQueueInfos(parent),
+        this.conf);
+    } catch (YarnRemoteException e) {
+      throw new IOException(e);
+    }
   }
 
   public String getStagingAreaDir() throws IOException, InterruptedException {

+ 43 - 19
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java

@@ -80,6 +80,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.client.RMTokenSelector;
@@ -207,10 +208,15 @@ public class YARNRunner implements ClientProtocol {
     GetDelegationTokenRequest request = recordFactory
       .newRecordInstance(GetDelegationTokenRequest.class);
     request.setRenewer(Master.getMasterPrincipal(conf));
-    DelegationToken mrDelegationToken = hsProxy.getDelegationToken(request)
-      .getDelegationToken();
-    return ProtoUtils.convertFromProtoFormat(mrDelegationToken,
-                                             hsProxy.getConnectAddress());
+    DelegationToken mrDelegationToken;
+    try {
+      mrDelegationToken = hsProxy.getDelegationToken(request)
+        .getDelegationToken();
+      return ProtoUtils.convertFromProtoFormat(mrDelegationToken,
+          hsProxy.getConnectAddress());
+    } catch (YarnRemoteException e) {
+      throw new IOException(e);
+    }
   }
 
   @Override
@@ -295,19 +301,25 @@ public class YARNRunner implements ClientProtocol {
       createApplicationSubmissionContext(conf, jobSubmitDir, ts);
 
     // Submit to ResourceManager
-    ApplicationId applicationId = resMgrDelegate.submitApplication(appContext);
-
-    ApplicationReport appMaster = resMgrDelegate
-        .getApplicationReport(applicationId);
-    String diagnostics =
-        (appMaster == null ?
-            "application report is null" : appMaster.getDiagnostics());
-    if (appMaster == null || appMaster.getYarnApplicationState() == YarnApplicationState.FAILED
-        || appMaster.getYarnApplicationState() == YarnApplicationState.KILLED) {
-      throw new IOException("Failed to run job : " +
-        diagnostics);
+    try {
+      ApplicationId applicationId =
+          resMgrDelegate.submitApplication(appContext);
+
+      ApplicationReport appMaster = resMgrDelegate
+          .getApplicationReport(applicationId);
+      String diagnostics =
+          (appMaster == null ?
+              "application report is null" : appMaster.getDiagnostics());
+      if (appMaster == null
+          || appMaster.getYarnApplicationState() == YarnApplicationState.FAILED
+          || appMaster.getYarnApplicationState() == YarnApplicationState.KILLED) {
+        throw new IOException("Failed to run job : " +
+            diagnostics);
+      }
+      return clientCache.getClient(jobId).getJobStatus(jobId);
+    } catch (YarnRemoteException e) {
+      throw new IOException(e);
     }
-    return clientCache.getClient(jobId).getJobStatus(jobId);
   }
 
   private LocalResource createApplicationResource(FileContext fs, Path p, LocalResourceType type)
@@ -552,7 +564,11 @@ public class YARNRunner implements ClientProtocol {
     /* check if the status is not running, if not send kill to RM */
     JobStatus status = clientCache.getClient(arg0).getJobStatus(arg0);
     if (status.getState() != JobStatus.State.RUNNING) {
-      resMgrDelegate.killApplication(TypeConverter.toYarn(arg0).getAppId());
+      try {
+        resMgrDelegate.killApplication(TypeConverter.toYarn(arg0).getAppId());
+      } catch (YarnRemoteException e) {
+        throw new IOException(e);
+      }
       return;
     }
 
@@ -576,7 +592,11 @@ public class YARNRunner implements ClientProtocol {
       LOG.debug("Error when checking for application status", io);
     }
     if (status.getState() != JobStatus.State.KILLED) {
-      resMgrDelegate.killApplication(TypeConverter.toYarn(arg0).getAppId());
+      try {
+        resMgrDelegate.killApplication(TypeConverter.toYarn(arg0).getAppId());
+      } catch (YarnRemoteException e) {
+        throw new IOException(e);
+      }
     }
   }
 
@@ -607,7 +627,11 @@ public class YARNRunner implements ClientProtocol {
   @Override
   public LogParams getLogFileParams(JobID jobID, TaskAttemptID taskAttemptID)
       throws IOException {
-    return clientCache.getClient(jobID).getLogFilePath(jobID, taskAttemptID);
+    try {
+      return clientCache.getClient(jobID).getLogFilePath(jobID, taskAttemptID);
+    } catch (YarnRemoteException e) {
+      throw new IOException(e);
+    }
   }
 
   private static void warnForJavaLibPath(String opts, String component, 

+ 10 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java

@@ -115,8 +115,9 @@ public class TestClientServiceDelegate {
     try {
       clientServiceDelegate.getJobStatus(oldJobId);
       Assert.fail("Invoke should throw exception after retries.");
-    } catch (YarnRemoteException e) {
-      Assert.assertEquals("Job ID doesnot Exist", e.getMessage());
+    } catch (IOException e) {
+      Assert.assertTrue(e.getMessage().contains(
+          "Job ID doesnot Exist"));
     }
   }
 
@@ -198,7 +199,8 @@ public class TestClientServiceDelegate {
   }
 
   @Test
-  public void testReconnectOnAMRestart() throws IOException {
+  public void testReconnectOnAMRestart() throws IOException,
+      YarnRemoteException {
     //test not applicable when AM not reachable
     //as instantiateAMProxy is not called at all
     if(!isAMReachableFromClient) {
@@ -265,7 +267,7 @@ public class TestClientServiceDelegate {
   }
   
   @Test
-  public void testAMAccessDisabled() throws IOException {
+  public void testAMAccessDisabled() throws IOException, YarnRemoteException {
     //test only applicable when AM not reachable
     if(isAMReachableFromClient) {
       return;
@@ -317,7 +319,8 @@ public class TestClientServiceDelegate {
   }
   
   @Test
-  public void testRMDownForJobStatusBeforeGetAMReport() throws IOException {
+  public void testRMDownForJobStatusBeforeGetAMReport() throws IOException,
+      YarnRemoteException {
     Configuration conf = new YarnConfiguration();
     testRMDownForJobStatusBeforeGetAMReport(conf,
         MRJobConfig.DEFAULT_MR_CLIENT_MAX_RETRIES);
@@ -325,7 +328,7 @@ public class TestClientServiceDelegate {
 
   @Test
   public void testRMDownForJobStatusBeforeGetAMReportWithRetryTimes()
-      throws IOException {
+      throws IOException, YarnRemoteException {
     Configuration conf = new YarnConfiguration();
     conf.setInt(MRJobConfig.MR_CLIENT_MAX_RETRIES, 2);
     testRMDownForJobStatusBeforeGetAMReport(conf, conf.getInt(
@@ -335,7 +338,7 @@ public class TestClientServiceDelegate {
   
   @Test
   public void testRMDownRestoreForJobStatusBeforeGetAMReport()
-      throws IOException {
+      throws IOException, YarnRemoteException {
     Configuration conf = new YarnConfiguration();
     conf.setInt(MRJobConfig.MR_CLIENT_MAX_RETRIES, 3);
 

+ 5 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
@@ -47,9 +48,12 @@ public class TestResourceMgrDelegate {
 
   /**
    * Tests that getRootQueues makes a request for the (recursive) child queues
+   * @throws YarnRemoteException
+   * @throws IOException
    */
   @Test
-  public void testGetRootQueues() throws IOException, InterruptedException {
+  public void testGetRootQueues() throws IOException, InterruptedException,
+      YarnRemoteException {
     final ClientRMProtocol applicationsManager = Mockito.mock(ClientRMProtocol.class);
     GetQueueInfoResponse response = Mockito.mock(GetQueueInfoResponse.class);
     org.apache.hadoop.yarn.api.records.QueueInfo queueInfo =

+ 2 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRJobClient.java

@@ -40,7 +40,6 @@ import org.apache.hadoop.mapreduce.tools.CLI;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 
 /**
  test CLI class. CLI class implemented  the Tool interface. 
@@ -155,7 +154,7 @@ public class TestMRJobClient extends ClusterMapReduceTestCase {
     try {
       runTool(conf, jc, new String[] { "-fail-task", taid.toString() }, out);
       fail(" this task should field");
-    } catch (YarnRemoteException e) {
+    } catch (IOException e) {
       // task completed !
       assertTrue(e.getMessage().contains("_0001_m_000000_1"));
     }
@@ -175,7 +174,7 @@ public class TestMRJobClient extends ClusterMapReduceTestCase {
     try {
       runTool(conf, jc, new String[] { "-kill-task", taid.toString() }, out);
       fail(" this task should be killed");
-    } catch (YarnRemoteException e) {
+    } catch (IOException e) {
       // task completed
       assertTrue(e.getMessage().contains("_0001_m_000000_1"));
     }

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java

@@ -62,7 +62,8 @@ public class TestJHSSecurity {
   private static final Log LOG = LogFactory.getLog(TestJHSSecurity.class);
   
   @Test
-  public void testDelegationToken() throws IOException, InterruptedException {
+  public void testDelegationToken() throws IOException, InterruptedException,
+      YarnRemoteException {
 
     Logger rootLogger = LogManager.getRootLogger();
     rootLogger.setLevel(Level.DEBUG);

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.util.BuilderUtils;
@@ -114,7 +115,7 @@ public class TestMRJobsWithHistoryService {
 
   @Test (timeout = 30000)
   public void testJobHistoryData() throws IOException, InterruptedException,
-      AvroRemoteException, ClassNotFoundException {
+      AvroRemoteException, ClassNotFoundException, YarnRemoteException {
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
       LOG.info("MRAppJar " + MiniMRYarnCluster.APPJAR
           + " not found. Not running test.");

+ 29 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/resources/hdfs-site.xml

@@ -0,0 +1,29 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+  <!-- Disable min block size since most tests use tiny blocks -->
+  <property>
+    <name>dfs.namenode.fs-limits.min-block-size</name>
+    <value>0</value>
+  </property>
+
+</configuration>

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java

@@ -153,7 +153,7 @@ public class ShuffleHandler extends AbstractService
     new JobTokenSecretManager();
 
   public static final String SHUFFLE_PORT_CONFIG_KEY = "mapreduce.shuffle.port";
-  public static final int DEFAULT_SHUFFLE_PORT = 8080;
+  public static final int DEFAULT_SHUFFLE_PORT = 11000;
 
   public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY =
     "mapreduce.shuffle.ssl.file.buffer.size";

+ 12 - 0
hadoop-yarn-project/CHANGES.txt

@@ -106,6 +106,9 @@ Release 2.0.5-beta - UNRELEASED
     favour of the copy present in the container token field. 
     (Vinod Kumar Vavilapalli via sseth)
 
+    YARN-629. Make YarnRemoteException not be rooted at IOException. (Xuan Gong
+    via vinodkv)
+
   NEW FEATURES
 
     YARN-482. FS: Extend SchedulingMode to intermediate queues. 
@@ -201,6 +204,15 @@ Release 2.0.5-beta - UNRELEASED
     to separate out various validation checks depending on whether they rely on
     RM configuration or not. (Zhijie Shen via vinodkv)
 
+    YARN-618. Modified RM_INVALID_IDENTIFIER to be -1 instead of zero. (Jian He
+    via vinodkv)
+
+    YARN-625. Move the utility method unwrapAndThrowException from
+    YarnRemoteExceptionPBImpl to RPCUtil. (Siddharth Seth via vinodkv)
+
+    YARN-645. Moved RMDelegationTokenSecretManager from yarn-server-common to
+    yarn-server-resourcemanager where it really belongs. (Jian He via vinodkv)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/YarnRemoteException.java

@@ -18,11 +18,10 @@
 
 package org.apache.hadoop.yarn.exceptions;
 
-import java.io.IOException;
 import java.io.PrintStream;
 import java.io.PrintWriter;
 
-public abstract class YarnRemoteException extends IOException {
+public abstract class YarnRemoteException extends Exception {
   private static final long serialVersionUID = 1L;
   
   public YarnRemoteException() {

+ 1 - 26
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/impl/pb/YarnRemoteExceptionPBImpl.java

@@ -109,30 +109,5 @@ public class YarnRemoteExceptionPBImpl extends YarnRemoteException {
     }
     viaProto = false;
   }
-  
-  /**
-   * Utility method that unwraps and throws appropriate exception. 
-   * @param se ServiceException
-   * @throws YarnRemoteException
-   * @throws UndeclaredThrowableException
-   */
-  public static YarnRemoteException unwrapAndThrowException(ServiceException se) 
-     throws UndeclaredThrowableException {
-    if (se.getCause() instanceof RemoteException) {
-      try {
-        throw ((RemoteException) se.getCause())
-            .unwrapRemoteException(YarnRemoteExceptionPBImpl.class);
-      } catch (YarnRemoteException ex) {
-        return ex;
-      } catch (IOException e1) {
-        throw new UndeclaredThrowableException(e1);
-      }
-    } else if (se.getCause() instanceof YarnRemoteException) {
-      return (YarnRemoteException)se.getCause();
-    } else if (se.getCause() instanceof UndeclaredThrowableException) {
-      throw (UndeclaredThrowableException)se.getCause();
-    } else {
-      throw new UndeclaredThrowableException(se);
-    }
-  }
+
 }

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java

@@ -312,8 +312,9 @@ public class Client extends YarnClientImpl {
    * Main run function for the client
    * @return true if application completed successfully
    * @throws IOException
+   * @throws YarnRemoteException
    */
-  public boolean run() throws IOException {
+  public boolean run() throws IOException, YarnRemoteException {
 
     LOG.info("Running Client");
     start();

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java

@@ -271,7 +271,7 @@ public class UnmanagedAMLauncher {
     amProc.destroy();
   }
   
-  public boolean run() throws IOException {
+  public boolean run() throws IOException, YarnRemoteException {
     LOG.info("Starting Client");
     
     // Connect to ResourceManager

+ 9 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/RMAdmin.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RefreshServiceAclsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
@@ -187,7 +188,7 @@ public class RMAdmin extends Configured implements Tool {
     return adminProtocol;
   }
   
-  private int refreshQueues() throws IOException {
+  private int refreshQueues() throws IOException, YarnRemoteException {
     // Refresh the queue properties
     RMAdminProtocol adminProtocol = createAdminProtocol();
     RefreshQueuesRequest request = 
@@ -196,7 +197,7 @@ public class RMAdmin extends Configured implements Tool {
     return 0;
   }
 
-  private int refreshNodes() throws IOException {
+  private int refreshNodes() throws IOException, YarnRemoteException {
     // Refresh the nodes
     RMAdminProtocol adminProtocol = createAdminProtocol();
     RefreshNodesRequest request = 
@@ -205,7 +206,8 @@ public class RMAdmin extends Configured implements Tool {
     return 0;
   }
   
-  private int refreshUserToGroupsMappings() throws IOException {
+  private int refreshUserToGroupsMappings() throws IOException,
+      YarnRemoteException {
     // Refresh the user-to-groups mappings
     RMAdminProtocol adminProtocol = createAdminProtocol();
     RefreshUserToGroupsMappingsRequest request = 
@@ -214,7 +216,8 @@ public class RMAdmin extends Configured implements Tool {
     return 0;
   }
   
-  private int refreshSuperUserGroupsConfiguration() throws IOException {
+  private int refreshSuperUserGroupsConfiguration() throws IOException,
+      YarnRemoteException {
     // Refresh the super-user groups
     RMAdminProtocol adminProtocol = createAdminProtocol();
     RefreshSuperUserGroupsConfigurationRequest request = 
@@ -223,7 +226,7 @@ public class RMAdmin extends Configured implements Tool {
     return 0;
   }
   
-  private int refreshAdminAcls() throws IOException {
+  private int refreshAdminAcls() throws IOException, YarnRemoteException {
     // Refresh the admin acls
     RMAdminProtocol adminProtocol = createAdminProtocol();
     RefreshAdminAclsRequest request = 
@@ -232,7 +235,7 @@ public class RMAdmin extends Configured implements Tool {
     return 0;
   }
   
-  private int refreshServiceAcls() throws IOException {
+  private int refreshServiceAcls() throws IOException, YarnRemoteException {
     // Refresh the service acls
     RMAdminProtocol adminProtocol = createAdminProtocol();
     RefreshServiceAclsRequest request = 

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/AMRMProtocolPBClientImpl.java

@@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationMaste
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterResponsePBImpl;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
-import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationMasterRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterRequestProto;
@@ -74,7 +74,7 @@ public class AMRMProtocolPBClientImpl implements AMRMProtocol, Closeable {
     try {
       return new AllocateResponsePBImpl(proxy.allocate(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -87,7 +87,7 @@ public class AMRMProtocolPBClientImpl implements AMRMProtocol, Closeable {
       return new FinishApplicationMasterResponsePBImpl(
         proxy.finishApplicationMaster(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -100,7 +100,7 @@ public class AMRMProtocolPBClientImpl implements AMRMProtocol, Closeable {
       return new RegisterApplicationMasterResponsePBImpl(
         proxy.registerApplicationMaster(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 }

+ 13 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ClientRMProtocolPBClientImpl.java

@@ -79,7 +79,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenRe
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
-import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllApplicationsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationReportRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterMetricsRequestProto;
@@ -120,7 +120,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
       return new KillApplicationResponsePBImpl(proxy.forceKillApplication(null,
         requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -133,7 +133,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
       return new GetApplicationReportResponsePBImpl(proxy.getApplicationReport(
         null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -146,7 +146,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
       return new GetClusterMetricsResponsePBImpl(proxy.getClusterMetrics(null,
         requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -159,7 +159,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
       return new GetNewApplicationResponsePBImpl(proxy.getNewApplication(null,
         requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -172,7 +172,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
       return new SubmitApplicationResponsePBImpl(proxy.submitApplication(null,
         requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -185,7 +185,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
       return new GetAllApplicationsResponsePBImpl(proxy.getAllApplications(
         null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -199,7 +199,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
       return new GetClusterNodesResponsePBImpl(proxy.getClusterNodes(null,
         requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -212,7 +212,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
       return new GetQueueInfoResponsePBImpl(proxy.getQueueInfo(null,
         requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -225,7 +225,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
       return new GetQueueUserAclsInfoResponsePBImpl(proxy.getQueueUserAcls(
         null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -238,7 +238,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
       return new GetDelegationTokenResponsePBImpl(proxy.getDelegationToken(
         null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -251,7 +251,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
       return new RenewDelegationTokenResponsePBImpl(proxy.renewDelegationToken(
           null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -265,7 +265,7 @@ public class ClientRMProtocolPBClientImpl implements ClientRMProtocol,
           proxy.cancelDelegationToken(null, requestProto));
 
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 }

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagerPBClientImpl.java

@@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerRequestPB
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerResponsePBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
-import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerRequestProto;
@@ -93,7 +93,7 @@ public class ContainerManagerPBClientImpl implements ContainerManager,
       return new GetContainerStatusResponsePBImpl(proxy.getContainerStatus(
         null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -106,7 +106,7 @@ public class ContainerManagerPBClientImpl implements ContainerManager,
       return new StartContainerResponsePBImpl(proxy.startContainer(null,
         requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -119,7 +119,7 @@ public class ContainerManagerPBClientImpl implements ContainerManager,
       return new StopContainerResponsePBImpl(proxy.stopContainer(null,
         requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 }

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java

@@ -53,7 +53,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshSuperUserGroups
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshUserToGroupsMappingsRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RefreshUserToGroupsMappingsResponsePBImpl;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
-import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetGroupsForUserRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetGroupsForUserResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsRequestProto;
@@ -94,7 +94,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol, Closeable {
       return new RefreshQueuesResponsePBImpl(
           proxy.refreshQueues(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -107,7 +107,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol, Closeable {
       return new RefreshNodesResponsePBImpl(
           proxy.refreshNodes(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -121,7 +121,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol, Closeable {
       return new RefreshSuperUserGroupsConfigurationResponsePBImpl(
           proxy.refreshSuperUserGroupsConfiguration(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -134,7 +134,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol, Closeable {
       return new RefreshUserToGroupsMappingsResponsePBImpl(
           proxy.refreshUserToGroupsMappings(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -147,7 +147,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol, Closeable {
       return new RefreshAdminAclsResponsePBImpl(
           proxy.refreshAdminAcls(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -160,7 +160,7 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol, Closeable {
       return new RefreshServiceAclsResponsePBImpl(proxy.refreshServiceAcls(
           null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 

+ 50 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/RPCUtil.java

@@ -18,11 +18,19 @@
 
 package org.apache.hadoop.yarn.ipc;
 
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.UndeclaredThrowableException;
+
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
 import org.apache.hadoop.yarn.factories.YarnRemoteExceptionFactory;
 import org.apache.hadoop.yarn.factory.providers.YarnRemoteExceptionFactoryProvider;
 
+import com.google.protobuf.ServiceException;
+
 public class RPCUtil {
 
 
@@ -52,4 +60,46 @@ public class RPCUtil {
       (e.getRemoteTrace() == null ? "" : "\n StackTrace: " + e.getRemoteTrace()) + 
       (e.getCause() == null ? "" : "\n Caused by: " + toString(e.getCause()));
   }
+  
+  /**
+   * Utility method that unwraps and throws appropriate exception.
+   * 
+   * @param se ServiceException
+   * @throws YarnRemoteException
+   * @throws UndeclaredThrowableException
+   */
+  public static YarnRemoteException unwrapAndThrowException(ServiceException se)
+      throws UndeclaredThrowableException {
+    if (se.getCause() instanceof RemoteException) {
+      try {
+        RemoteException re = (RemoteException) se.getCause();
+        Class<?> realClass = Class.forName(re.getClassName());
+        //YarnRemoteException is not rooted as IOException.
+        //Do the explicitly check if it is YarnRemoteException
+        if (YarnRemoteException.class.isAssignableFrom(realClass)) {
+          Constructor<? extends YarnRemoteException> cn =
+              realClass.asSubclass(YarnRemoteException.class).getConstructor(
+                  String.class);
+          cn.setAccessible(true);
+          YarnRemoteException ex = cn.newInstance(re.getMessage());
+          ex.initCause(re);
+          return ex;
+        } else {
+          throw ((RemoteException) se.getCause())
+              .unwrapRemoteException(YarnRemoteExceptionPBImpl.class);
+        }
+      } catch (IOException e1) {
+        throw new UndeclaredThrowableException(e1);
+      } catch (Exception ex) {
+        throw new UndeclaredThrowableException(
+            (RemoteException) se.getCause());
+      }
+    } else if (se.getCause() instanceof YarnRemoteException) {
+      return (YarnRemoteException) se.getCause();
+    } else if (se.getCause() instanceof UndeclaredThrowableException) {
+      throw (UndeclaredThrowableException) se.getCause();
+    } else {
+      throw new UndeclaredThrowableException(se);
+    }
+  }
 }

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/RMDelegationTokenIdentifier.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.records.DelegationToken;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
@@ -105,6 +106,8 @@ public class RMDelegationTokenIdentifier extends AbstractDelegationTokenIdentifi
               Records.newRecord(RenewDelegationTokenRequest.class);
           request.setDelegationToken(convertToProtoToken(token));
           return rmClient.renewDelegationToken(request).getNextExpirationTime();
+        } catch (YarnRemoteException e) {
+          throw new IOException(e);
         } finally {
           RPC.stopProxy(rmClient);
         }
@@ -125,6 +128,8 @@ public class RMDelegationTokenIdentifier extends AbstractDelegationTokenIdentifi
               Records.newRecord(CancelDelegationTokenRequest.class);
           request.setDelegationToken(convertToProtoToken(token));
           rmClient.cancelDelegationToken(request);
+        } catch (YarnRemoteException e) {
+          throw new IOException(e);
         } finally {
           RPC.stopProxy(rmClient);
         }

+ 6 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerConstants.java

@@ -20,5 +20,10 @@ package org.apache.hadoop.yarn.server.api;
 
 public interface ResourceManagerConstants {
 
-  public static final long RM_INVALID_IDENTIFIER = 0;
+  /**
+   * This states the invalid identifier of Resource Manager. This is used as a
+   * default value for initializing RM identifier. Currently, RM is using time
+   * stamp as RM identifier.
+   */
+  public static final long RM_INVALID_IDENTIFIER = -1;
 }

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceTrackerPBClientImpl.java

@@ -25,7 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
-import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.RegisterNodeManagerRequestProto;
 import org.apache.hadoop.yarn.server.api.ResourceTracker;
@@ -58,7 +58,7 @@ private ResourceTrackerPB proxy;
     try {
       return new RegisterNodeManagerResponsePBImpl(proxy.registerNodeManager(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 
@@ -69,7 +69,7 @@ private ResourceTrackerPB proxy;
     try {
       return new NodeHeartbeatResponsePBImpl(proxy.nodeHeartbeat(null, requestProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/impl/pb/client/LocalizationProtocolPBClientImpl.java

@@ -25,7 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
-import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerStatusProto;
 import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocol;
 import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocolPB;
@@ -62,7 +62,7 @@ public class LocalizationProtocolPBClientImpl implements LocalizationProtocol,
       return new LocalizerHeartbeatResponsePBImpl(
           proxy.heartbeat(null, statusProto));
     } catch (ServiceException e) {
-      throw YarnRemoteExceptionPBImpl.unwrapAndThrowException(e);
+      throw RPCUtil.unwrapAndThrowException(e);
     }
   }
 

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.ContainerManager;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;

+ 7 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager;
 import org.junit.After;
 
@@ -73,7 +74,8 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
   }
 
   @Override
-  public void testContainerSetup() throws IOException, InterruptedException {
+  public void testContainerSetup() throws IOException, InterruptedException,
+      YarnRemoteException {
     // Don't run the test if the binary is not available.
     if (!shouldRunTest()) {
       LOG.info("LCE binary path is not passed. Not running the test");
@@ -96,7 +98,7 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
 
   @Override
   public void testContainerLaunchAndStop() throws IOException,
-      InterruptedException {
+      InterruptedException, YarnRemoteException {
     // Don't run the test if the binary is not available.
     if (!shouldRunTest()) {
       LOG.info("LCE binary path is not passed. Not running the test");
@@ -108,7 +110,7 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
   
   @Override
   public void testContainerLaunchAndExitSuccess() throws IOException,
-      InterruptedException {
+      InterruptedException, YarnRemoteException {
     // Don't run the test if the binary is not available.
     if (!shouldRunTest()) {
       LOG.info("LCE binary path is not passed. Not running the test");
@@ -120,7 +122,7 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
 
   @Override
   public void testContainerLaunchAndExitFailure() throws IOException,
-      InterruptedException {
+      InterruptedException, YarnRemoteException {
     // Don't run the test if the binary is not available.
     if (!shouldRunTest()) {
       LOG.info("LCE binary path is not passed. Not running the test");
@@ -132,7 +134,7 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
   
   @Override
   public void testLocalFilesCleanup() throws InterruptedException,
-      IOException {
+      IOException, YarnRemoteException {
     // Don't run the test if the binary is not available.
     if (!shouldRunTest()) {
       LOG.info("LCE binary path is not passed. Not running the test");

+ 10 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.api.ResourceTracker;
@@ -58,10 +59,11 @@ public class TestEventFlow {
       TestEventFlow.class.getName() + "-localLogDir").getAbsoluteFile();
   private static File remoteLogDir = new File("target",
       TestEventFlow.class.getName() + "-remoteLogDir").getAbsoluteFile();
+  private static final long SIMULATED_RM_IDENTIFIER = 1234;
 
   @Test
   public void testSuccessfulContainerLaunch() throws InterruptedException,
-      IOException {
+      IOException, YarnRemoteException {
 
     FileContext localFS = FileContext.getLocalFSFileContext();
 
@@ -100,6 +102,11 @@ public class TestEventFlow {
       protected void startStatusUpdater() {
         return; // Don't start any updating thread.
       }
+
+      @Override
+      public long getRMIdentifier() {
+        return SIMULATED_RM_IDENTIFIER;
+      }
     };
 
     DummyContainerManager containerManager =
@@ -124,6 +131,8 @@ public class TestEventFlow {
     when(mockContainer.getId()).thenReturn(cID);
     when(mockContainer.getResource()).thenReturn(recordFactory
         .newRecordInstance(Resource.class));
+    when(mockContainer.getRMIdentifer()).thenReturn(SIMULATED_RM_IDENTIFIER);
+
     launchContext.setUser("testing");
     StartContainerRequest request = 
         recordFactory.newRecordInstance(StartContainerRequest.class);

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
@@ -87,7 +88,8 @@ public class TestNodeManagerReboot {
   }
 
   @Test(timeout = 20000)
-  public void testClearLocalDirWhenNodeReboot() throws IOException {
+  public void testClearLocalDirWhenNodeReboot() throws IOException,
+      YarnRemoteException {
     nm = new MyNodeManager();
     nm.start();
     // create files under fileCache

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java

@@ -84,7 +84,7 @@ public class TestNodeManagerResync {
   @SuppressWarnings("unchecked")
   @Test
   public void testKillContainersOnResync() throws IOException,
-      InterruptedException {
+      InterruptedException, YarnRemoteException {
     NodeManager nm = new TestNodeManager1();
     YarnConfiguration conf = createNMConfig();
     nm.init(conf);
@@ -112,7 +112,7 @@ public class TestNodeManagerResync {
   @SuppressWarnings("unchecked")
   @Test
   public void testBlockNewContainerRequestsOnStartAndResync()
-      throws IOException, InterruptedException {
+      throws IOException, InterruptedException, YarnRemoteException {
     NodeManager nm = new TestNodeManager2();
     YarnConfiguration conf = createNMConfig();
     nm.init(conf);

+ 4 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java

@@ -54,6 +54,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
@@ -97,7 +98,8 @@ public class TestNodeManagerShutdown {
   }
   
   @Test
-  public void testKillContainersOnShutdown() throws IOException {
+  public void testKillContainersOnShutdown() throws IOException,
+      YarnRemoteException {
     NodeManager nm = getNodeManager();
     nm.init(createNMConfig());
     nm.start();
@@ -144,7 +146,7 @@ public class TestNodeManagerShutdown {
 
   public static void startContainer(NodeManager nm, ContainerId cId,
       FileContext localFS, File scriptFileDir, File processStartFile)
-      throws IOException {
+      throws IOException, YarnRemoteException {
     ContainerManagerImpl containerManager = nm.getContainerManager();
     File scriptFile =
         createUnhaltingScriptFile(cId, scriptFileDir, processStartFile);

+ 7 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java

@@ -98,6 +98,7 @@ public abstract class BaseContainerManagerTest {
   protected String user = "nobody";
   protected NodeHealthCheckerService nodeHealthChecker;
   protected LocalDirsHandlerService dirsHandler;
+  protected final long DUMMY_RM_IDENTIFIER = 1234;
 
   protected NodeStatusUpdater nodeStatusUpdater = new NodeStatusUpdaterImpl(
       context, new AsyncDispatcher(), null, metrics) {
@@ -110,6 +111,12 @@ public abstract class BaseContainerManagerTest {
     protected void startStatusUpdater() {
       return; // Don't start any updating thread.
     }
+
+    @Override
+    public long getRMIdentifier() {
+      // There is no real RM registration, simulate and set RMIdentifier
+      return DUMMY_RM_IDENTIFIER;
+    }
   };
 
   protected ContainerManagerImpl containerManager = null;

+ 17 - 25
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java

@@ -26,7 +26,6 @@ import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
@@ -58,12 +57,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
-import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
-import org.apache.hadoop.yarn.server.nodemanager.LocalRMInterface;
-import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
-import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
@@ -116,7 +111,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
   }
 
   @Test
-  public void testContainerSetup() throws IOException, InterruptedException {
+  public void testContainerSetup() throws IOException, InterruptedException,
+      YarnRemoteException {
 
     containerManager.start();
 
@@ -161,6 +157,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     when(mockContainer.getNodeId()).thenReturn(context.getNodeId());
     when(mockContainer.getNodeHttpAddress()).thenReturn(
         context.getNodeId().getHost() + ":12345");
+    when(mockContainer.getRMIdentifer()).thenReturn(super.DUMMY_RM_IDENTIFIER);
 
     StartContainerRequest startRequest = 
         recordFactory.newRecordInstance(StartContainerRequest.class);
@@ -206,7 +203,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
   @Test
   public void testContainerLaunchAndStop() throws IOException,
-      InterruptedException {
+      InterruptedException, YarnRemoteException {
     containerManager.start();
 
     File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
@@ -259,6 +256,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     when(mockContainer.getNodeId()).thenReturn(context.getNodeId());
     when(mockContainer.getNodeHttpAddress()).thenReturn(
         context.getNodeId().getHost() + ":12345");
+    when(mockContainer.getRMIdentifer()).thenReturn(super.DUMMY_RM_IDENTIFIER);
 
     StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
     startRequest.setContainerLaunchContext(containerLaunchContext);
@@ -312,7 +310,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
       DefaultContainerExecutor.containerIsAlive(pid));
   }
   
-  private void testContainerLaunchAndExit(int exitCode) throws IOException, InterruptedException {
+  private void testContainerLaunchAndExit(int exitCode) throws IOException,
+      InterruptedException, YarnRemoteException {
 
 	  File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
 	  PrintWriter fileWriter = new PrintWriter(scriptFile);
@@ -370,6 +369,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
     when(mockContainer.getNodeId()).thenReturn(context.getNodeId());
     when(mockContainer.getNodeHttpAddress()).thenReturn(
         context.getNodeId().getHost() + ":12345");
+    when(mockContainer.getRMIdentifer()).thenReturn(super.DUMMY_RM_IDENTIFIER);
+
 	  StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
 	  startRequest.setContainerLaunchContext(containerLaunchContext);
 	  startRequest.setContainer(mockContainer);
@@ -390,7 +391,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
   }
   
   @Test
-  public void testContainerLaunchAndExitSuccess() throws IOException, InterruptedException {
+  public void testContainerLaunchAndExitSuccess() throws IOException,
+      InterruptedException, YarnRemoteException {
 	  containerManager.start();
 	  int exitCode = 0; 
 
@@ -400,7 +402,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
   }
 
   @Test
-  public void testContainerLaunchAndExitFailure() throws IOException, InterruptedException {
+  public void testContainerLaunchAndExitFailure() throws IOException,
+      InterruptedException, YarnRemoteException {
 	  containerManager.start();
 	  int exitCode = 50; 
 
@@ -411,7 +414,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
   
   @Test
   public void testLocalFilesCleanup() throws InterruptedException,
-      IOException {
+      IOException, YarnRemoteException {
     // Real del service
     delSrvc = new DeletionService(exec);
     delSrvc.init(conf);
@@ -470,6 +473,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     when(mockContainer.getNodeId()).thenReturn(context.getNodeId());
     when(mockContainer.getNodeHttpAddress()).thenReturn(
         context.getNodeId().getHost() + ":12345");
+    when(mockContainer.getRMIdentifer()).thenReturn(super.DUMMY_RM_IDENTIFIER);
 
 //    containerLaunchContext.command = new ArrayList<CharSequence>();
 
@@ -541,19 +545,6 @@ public class TestContainerManager extends BaseContainerManagerTest {
   @Test
   public void testContainerLaunchFromPreviousRM() throws IOException,
       InterruptedException {
-    // There is no real RM registration, simulate and set RMIdentifier
-    NodeStatusUpdater nodeStatusUpdater = mock(NodeStatusUpdater.class);
-    when(nodeStatusUpdater.getRMIdentifier()).thenReturn((long) 1234);
-    containerManager =
-        new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
-          metrics, new ApplicationACLsManager(conf), dirsHandler) {
-          @Override
-          public void setBlockNewContainerRequests(
-              boolean blockNewContainerRequests) {
-            // do nothing
-          }
-        };
-    containerManager.init(conf);
     containerManager.start();
 
     ContainerLaunchContext containerLaunchContext =
@@ -595,7 +586,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
     // Construct the Container with a RMIdentifier within current RM
     Container mockContainer2 = mock(Container.class);
     when(mockContainer2.getId()).thenReturn(cId2);
-    when(mockContainer2.getRMIdentifer()).thenReturn((long) 1234);
+    when(mockContainer2.getRMIdentifer()).thenReturn(super.DUMMY_RM_IDENTIFIER);
+
     when(mockContainer2.getResource()).thenReturn(mockResource);
     StartContainerRequest startRequest2 =
         recordFactory.newRecordInstance(StartContainerRequest.class);

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java

@@ -180,6 +180,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     when(mockContainer.getNodeId()).thenReturn(context.getNodeId());
     when(mockContainer.getNodeHttpAddress()).thenReturn(
         context.getNodeId().getHost() + ":12345");
+    when(mockContainer.getRMIdentifer()).thenReturn(super.DUMMY_RM_IDENTIFIER);
 
     Map<String, String> userSetEnv = new HashMap<String, String>();
     userSetEnv.put(Environment.CONTAINER_ID.name(), "user_set_container_id");
@@ -376,6 +377,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     when(mockContainer.getNodeId()).thenReturn(context.getNodeId());
     when(mockContainer.getNodeHttpAddress()).thenReturn(
         context.getNodeId().getHost() + ":12345");
+    when(mockContainer.getRMIdentifer()).thenReturn(super.DUMMY_RM_IDENTIFIER);
 
     containerLaunchContext.setUser(user);
 

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java

@@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat;
@@ -663,7 +664,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
 
   @Test
   public void testLogAggregationForRealContainerLaunch() throws IOException,
-      InterruptedException {
+      InterruptedException, YarnRemoteException {
 
     this.containerManager.start();
 
@@ -691,6 +692,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     ContainerId cId = BuilderUtils.newContainerId(appAttemptId, 0);
 
     when(mockContainer.getId()).thenReturn(cId);
+    when(mockContainer.getRMIdentifer()).thenReturn(super.DUMMY_RM_IDENTIFIER);
 
     containerLaunchContext.setUser(this.user);
 

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java

@@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
@@ -177,7 +178,7 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
 
   @Test
   public void testContainerKillOnMemoryOverflow() throws IOException,
-      InterruptedException {
+      InterruptedException, YarnRemoteException {
 
     if (!ProcfsBasedProcessTree.isAvailable()) {
       return;
@@ -216,6 +217,8 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
     when(mockContainer.getNodeId()).thenReturn(context.getNodeId());
     when(mockContainer.getNodeHttpAddress()).thenReturn(
         context.getNodeId().getHost() + ":12345");
+    when(mockContainer.getRMIdentifer()).thenReturn(
+      super.DUMMY_RM_IDENTIFIER);
     containerLaunchContext.setUser(user);
 
     URL resource_alpha =

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java

@@ -79,7 +79,6 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
-import org.apache.hadoop.yarn.server.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
@@ -87,6 +86,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.service.AbstractService;

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java

@@ -43,7 +43,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.server.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
@@ -69,6 +68,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.ApplicationTokenSe
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebApp;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.webproxy.AppReportFetcher;

+ 5 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java

@@ -50,6 +50,7 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
@@ -98,7 +99,7 @@ public class AMLauncher implements Runnable {
     containerMgrProxy = getContainerMgrProxy(masterContainerID);
   }
   
-  private void launch() throws IOException {
+  private void launch() throws IOException, YarnRemoteException {
     connect();
     ContainerId masterContainerID = masterContainer.getId();
     ApplicationSubmissionContext applicationContext =
@@ -116,7 +117,7 @@ public class AMLauncher implements Runnable {
         + " for AM " + application.getAppAttemptId());
   }
   
-  private void cleanup() throws IOException {
+  private void cleanup() throws IOException, YarnRemoteException {
     connect();
     ContainerId containerId = masterContainer.getId();
     StopContainerRequest stopRequest = 
@@ -256,6 +257,8 @@ public class AMLauncher implements Runnable {
         cleanup();
       } catch(IOException ie) {
         LOG.info("Error cleaning master ", ie);
+      } catch (YarnRemoteException e) {
+        LOG.info("Error cleaning master ", e);
       }
       break;
     default:

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/RMDelegationTokenSecretManager.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMDelegationTokenSecretManager.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.server;
+package org.apache.hadoop.yarn.server.resourcemanager.security;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -44,7 +44,7 @@ public class RMDelegationTokenSecretManager
    *        for expired tokens
    */
   public RMDelegationTokenSecretManager(long delegationKeyUpdateInterval,
-                                      long delegationTokenMaxLifetime, 
+                                      long delegationTokenMaxLifetime,
                                       long delegationTokenRenewInterval,
                                       long delegationTokenRemoverScanInterval) {
     super(delegationKeyUpdateInterval, delegationTokenMaxLifetime,

+ 7 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.Task.State;
@@ -127,7 +128,7 @@ public class Application {
     return used;
   }
   
-  public synchronized void submit() throws IOException {
+  public synchronized void submit() throws IOException, YarnRemoteException {
     ApplicationSubmissionContext context = recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
     context.setApplicationId(this.applicationId);
     context.getAMContainerSpec().setUser(this.user);
@@ -201,7 +202,8 @@ public class Application {
     addResourceRequest(priority, requests, ResourceRequest.ANY, capability);
   }
   
-  public synchronized void finishTask(Task task) throws IOException {
+  public synchronized void finishTask(Task task) throws IOException,
+      YarnRemoteException {
     Set<Task> tasks = this.tasks.get(task.getPriority());
     if (!tasks.remove(task)) {
       throw new IllegalStateException(
@@ -288,7 +290,7 @@ public class Application {
   }
   
   public synchronized void assign(List<Container> containers) 
-  throws IOException {
+  throws IOException, YarnRemoteException {
     
     int numContainers = containers.size();
     // Schedule in priority order
@@ -307,12 +309,12 @@ public class Application {
         assignedContainers + "/" + numContainers);
   }
   
-  public synchronized void schedule() throws IOException {
+  public synchronized void schedule() throws IOException, YarnRemoteException {
     assign(getResources());
   }
   
   private synchronized void assign(Priority priority, NodeType type, 
-      List<Container> containers) throws IOException {
+      List<Container> containers) throws IOException, YarnRemoteException {
     for (Iterator<Container> i=containers.iterator(); i.hasNext();) {
       Container container = i.next();
       String host = container.getNodeId().toString();

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java

@@ -44,7 +44,6 @@ import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
-import org.apache.hadoop.yarn.server.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
@@ -59,6 +58,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java

@@ -81,7 +81,7 @@ public class NodeManager implements ContainerManager {
   public NodeManager(String hostName, int containerManagerPort, int httpPort,
       String rackName, Resource capability,
       ResourceTrackerService resourceTrackerService, RMContext rmContext)
-      throws IOException {
+      throws IOException, YarnRemoteException {
     this.containerManagerAddress = hostName + ":" + containerManagerPort;
     this.nodeHttpAddress = hostName + ":" + httpPort;
     this.rackName = rackName;
@@ -144,7 +144,7 @@ public class NodeManager implements ContainerManager {
     }
     return containerStatuses;
   }
-  public void heartbeat() throws IOException {
+  public void heartbeat() throws IOException, YarnRemoteException {
     NodeStatus nodeStatus = 
       org.apache.hadoop.yarn.server.resourcemanager.NodeManager.createNodeStatus(
           nodeId, getContainerStatuses(containers));

+ 15 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java

@@ -65,13 +65,13 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
-import org.apache.hadoop.yarn.server.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
@@ -205,15 +205,19 @@ public class TestClientRMService {
       owner.doAs(new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
-          checkTokenRenewal(owner, other);
-          return null;
+          try {
+            checkTokenRenewal(owner, other);
+            return null;
+          } catch (YarnRemoteException ex) {
+            Assert.assertEquals(ex.getMessage(),
+                "Client " + owner.getUserName() +
+                " tries to renew a token with renewer specified as " +
+                other.getUserName());
+            throw ex;
+          }
         }
       });
-    } catch (YarnRemoteException e) {
-      Assert.assertEquals(e.getMessage(),
-          "Client " + owner.getUserName() +
-          " tries to renew a token with renewer specified as " +
-          other.getUserName());
+    } catch (Exception e) {
       return;
     }
     Assert.fail("renew should have failed");
@@ -232,7 +236,7 @@ public class TestClientRMService {
   }
 
   private void checkTokenRenewal(UserGroupInformation owner,
-      UserGroupInformation renewer) throws IOException {
+      UserGroupInformation renewer) throws IOException, YarnRemoteException {
     RMDelegationTokenIdentifier tokenIdentifier =
         new RMDelegationTokenIdentifier(
             new Text(owner.getUserName()), new Text(renewer.getUserName()), null);
@@ -312,7 +316,8 @@ public class TestClientRMService {
   
   @Test(timeout=4000)
   public void testConcurrentAppSubmit()
-      throws IOException, InterruptedException, BrokenBarrierException {
+      throws IOException, InterruptedException, BrokenBarrierException,
+      YarnRemoteException {
     YarnScheduler yarnScheduler = mockYarnScheduler();
     RMContext rmContext = mock(RMContext.class);
     mockRMContext(yarnScheduler, rmContext);

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java

@@ -52,8 +52,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
-import org.apache.hadoop.yarn.server.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.ProtoUtils;
@@ -72,7 +72,8 @@ public class TestClientRMTokens {
   }
   
   @Test
-  public void testDelegationToken() throws IOException, InterruptedException {
+  public void testDelegationToken() throws IOException, InterruptedException,
+      YarnRemoteException {
     
     final YarnConfiguration conf = new YarnConfiguration();
     conf.set(YarnConfiguration.RM_PRINCIPAL, "testuser/localhost@apache.org");

+ 5 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.junit.After;
@@ -59,7 +60,8 @@ public class TestResourceManager {
 
   private org.apache.hadoop.yarn.server.resourcemanager.NodeManager
       registerNode(String hostName, int containerManagerPort, int httpPort,
-          String rackName, Resource capability) throws IOException {
+          String rackName, Resource capability) throws IOException,
+          YarnRemoteException {
     return new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
         hostName, containerManagerPort, httpPort, rackName, capability,
         resourceManager.getResourceTrackerService(), resourceManager
@@ -67,7 +69,8 @@ public class TestResourceManager {
   }
 
 //  @Test
-  public void testResourceAllocation() throws IOException {
+  public void testResourceAllocation() throws IOException,
+      YarnRemoteException {
     LOG.info("--- START: testResourceAllocation ---");
         
     final int memory = 4 * 1024;

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.event.InlineDispatcher;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
@@ -88,7 +89,7 @@ public class TestRMNMRPCResponseId {
   }
 
   @Test
-  public void testRPCResponseId() throws IOException {
+  public void testRPCResponseId() throws IOException, YarnRemoteException {
     String node = "localhost";
     Resource capability = BuilderUtils.newResource(1024, 1);
     RegisterNodeManagerRequest request = recordFactory.newRecordInstance(RegisterNodeManagerRequest.class);

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.server.resourcemanager.Application;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
@@ -101,7 +102,7 @@ public class TestCapacityScheduler {
   private org.apache.hadoop.yarn.server.resourcemanager.NodeManager
       registerNode(String hostName, int containerManagerPort, int httpPort,
           String rackName, Resource capability)
-          throws IOException {
+          throws IOException, YarnRemoteException {
     return new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
         hostName, containerManagerPort, httpPort, rackName, capability,
         resourceManager.getResourceTrackerService(), resourceManager

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.InlineDispatcher;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.Application;
@@ -84,7 +85,8 @@ public class TestFifoScheduler {
   
   private org.apache.hadoop.yarn.server.resourcemanager.NodeManager
       registerNode(String hostName, int containerManagerPort, int nmHttpPort,
-          String rackName, Resource capability) throws IOException {
+          String rackName, Resource capability) throws IOException,
+          YarnRemoteException {
     return new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
         hostName, containerManagerPort, nmHttpPort, rackName, capability,
         resourceManager.getResourceTrackerService(), resourceManager

Alguns ficheiros não foram mostrados porque muitos ficheiros mudaram neste diff