浏览代码

Merge trunk to HDFS-4685.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4685@1553226 13f79535-47bb-0310-9956-ffa450edef68
Chris Nauroth 11 年之前
父节点
当前提交
eaa5321619
共有 49 个文件被更改,包括 1058 次插入338 次删除
  1. 6 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 4 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
  3. 3 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java
  4. 0 15
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
  5. 11 8
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/source/JvmMetrics.java
  6. 16 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java
  7. 13 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
  8. 13 3
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  9. 13 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  10. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirective.java
  11. 25 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
  12. 71 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
  13. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  14. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
  15. 9 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java
  16. 134 51
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
  17. 26 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java
  18. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  19. 22 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  20. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  21. 138 61
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
  22. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  23. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/ShortCircuitLocalReads.apt.vm
  24. 10 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
  25. 185 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
  26. 72 71
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
  27. 34 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCacheAdminConf.xml
  28. 6 0
      hadoop-mapreduce-project/CHANGES.txt
  29. 18 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
  30. 15 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
  31. 5 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
  32. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java
  33. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java
  34. 15 15
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptInfo.java
  35. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskInfo.java
  36. 20 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java
  37. 6 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java
  38. 7 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java
  39. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/TaskReport.java
  40. 12 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/TaskReportPBImpl.java
  41. 25 17
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java
  42. 26 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestBlocks.java
  43. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesAttempts.java
  44. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesTasks.java
  45. 36 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecutionWithMRApp.java
  46. 16 1
      hadoop-yarn-project/CHANGES.txt
  47. 0 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java
  48. 5 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java
  49. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm

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

@@ -404,6 +404,9 @@ Release 2.4.0 - UNRELEASED
 
     HADOOP-10164. Allow UGI to login with a known Subject (bobby)
 
+    HADOOP-10169. Remove the unnecessary synchronized in JvmMetrics class.
+    (Liang Xie via jing9) 
+
   OPTIMIZATIONS
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
@@ -562,6 +565,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-10087. UserGroupInformation.getGroupNames() fails to return primary
     group first when JniBasedUnixGroupsMappingWithFallback is used (cmccabe)
 
+    HADOOP-10175. Har files system authority should preserve userinfo.
+    (Chuan Liu via cnauroth)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 4 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java

@@ -294,6 +294,10 @@ public class HarFileSystem extends FileSystem {
   private String getHarAuth(URI underLyingUri) {
     String auth = underLyingUri.getScheme() + "-";
     if (underLyingUri.getHost() != null) {
+      if (underLyingUri.getUserInfo() != null) {
+        auth += underLyingUri.getUserInfo();
+        auth += "@";
+      }
       auth += underLyingUri.getHost();
       if (underLyingUri.getPort() != -1) {
         auth += ":";

+ 3 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java

@@ -305,12 +305,13 @@ public class HttpServer implements FilterContainer {
         }
       }
 
-      if (endpoints.size() == 0) {
+      if (endpoints.size() == 0 && connector == null) {
         throw new HadoopIllegalArgumentException("No endpoints specified");
       }
 
       if (hostName == null) {
-        hostName = endpoints.get(0).getHost();
+        hostName = endpoints.size() == 0 ? connector.getHost() : endpoints.get(
+            0).getHost();
       }
       
       if (this.conf == null) {

+ 0 - 15
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java

@@ -69,13 +69,6 @@ public class RetryPolicies {
    */
   public static final RetryPolicy RETRY_FOREVER = new RetryForever();
 
-  /**
-   * <p>
-   * Keep failing over forever
-   * </p>
-   */
-  public static final RetryPolicy FAILOVER_FOREVER = new FailoverForever();
-
   /**
    * <p>
    * Keep trying a limited number of times, waiting a fixed time between attempts,
@@ -173,14 +166,6 @@ public class RetryPolicies {
       return RetryAction.RETRY;
     }
   }
-
-  static class FailoverForever implements RetryPolicy {
-    @Override
-    public RetryAction shouldRetry(Exception e, int retries, int failovers,
-        boolean isIdempotentOrAtMostOnce) throws Exception {
-      return RetryAction.FAILOVER_AND_RETRY;
-    }
-  }
   
   /**
    * Retry up to maxRetries.

+ 11 - 8
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/source/JvmMetrics.java

@@ -24,10 +24,8 @@ import java.lang.management.MemoryUsage;
 import java.lang.management.ThreadInfo;
 import java.lang.management.ThreadMXBean;
 import java.lang.management.GarbageCollectorMXBean;
-import java.util.Map;
 import java.util.List;
-
-import com.google.common.collect.Maps;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.log.metrics.EventCounter;
@@ -67,7 +65,8 @@ public class JvmMetrics implements MetricsSource {
       ManagementFactory.getGarbageCollectorMXBeans();
   final ThreadMXBean threadMXBean = ManagementFactory.getThreadMXBean();
   final String processName, sessionId;
-  final Map<String, MetricsInfo[]> gcInfoCache = Maps.newHashMap();
+  final ConcurrentHashMap<String, MetricsInfo[]> gcInfoCache =
+      new ConcurrentHashMap<String, MetricsInfo[]>();
 
   JvmMetrics(String processName, String sessionId) {
     this.processName = processName;
@@ -123,13 +122,17 @@ public class JvmMetrics implements MetricsSource {
       .addCounter(GcTimeMillis, timeMillis);
   }
 
-  private synchronized MetricsInfo[] getGcInfo(String gcName) {
+  private MetricsInfo[] getGcInfo(String gcName) {
     MetricsInfo[] gcInfo = gcInfoCache.get(gcName);
     if (gcInfo == null) {
       gcInfo = new MetricsInfo[2];
-      gcInfo[0] = Interns.info("GcCount"+ gcName, "GC Count for "+ gcName);
-      gcInfo[1] = Interns.info("GcTimeMillis"+ gcName, "GC Time for "+ gcName);
-      gcInfoCache.put(gcName, gcInfo);
+      gcInfo[0] = Interns.info("GcCount" + gcName, "GC Count for " + gcName);
+      gcInfo[1] = Interns
+          .info("GcTimeMillis" + gcName, "GC Time for " + gcName);
+      MetricsInfo[] previousGcInfo = gcInfoCache.putIfAbsent(gcName, gcInfo);
+      if (previousGcInfo != null) {
+        return previousGcInfo;
+      }
     }
     return gcInfo;
   }

+ 16 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java

@@ -258,6 +258,22 @@ public class TestHarFileSystemBasics {
                  0, expectedFileNames.size());
   }
 
+  @Test
+  public void testMakeQualifiedPath() throws Exception {
+    // Construct a valid har file system path with authority that
+    // contains userinfo and port. The userinfo and port are useless
+    // in local fs uri. They are only used to verify har file system
+    // can correctly preserve the information for the underlying file system.
+    String harPathWithUserinfo = "har://file-user:passwd@localhost:80"
+        + harPath.toUri().getPath().toString();
+    Path path = new Path(harPathWithUserinfo);
+    Path qualifiedPath = path.getFileSystem(conf).makeQualified(path);
+    assertTrue(String.format(
+        "The qualified path (%s) did not match the expected path (%s).",
+        qualifiedPath.toString(), harPathWithUserinfo),
+        qualifiedPath.toString().equals(harPathWithUserinfo));
+  }
+
   // ========== Negative:
 
   @Test

+ 13 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java

@@ -66,6 +66,8 @@ import org.mockito.internal.util.reflection.Whitebox;
 import org.mortbay.jetty.Connector;
 import org.mortbay.util.ajax.JSON;
 
+import static org.mockito.Mockito.*;
+
 public class TestHttpServer extends HttpServerFunctionalTest {
   static final Log LOG = LogFactory.getLog(TestHttpServer.class);
   private static HttpServer server;
@@ -588,4 +590,15 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     assertEquals(conn.getHeaderField("Expires"), conn.getHeaderField("Date"));
   }
 
+  /**
+   * HTTPServer.Builder should proceed if a external connector is available.
+   */
+  @Test
+  public void testHttpServerBuilderWithExternalConnector() throws Exception {
+    Connector c = mock(Connector.class);
+    doReturn("localhost").when(c).getHost();
+    HttpServer s = new HttpServer.Builder().setName("test").setConnector(c)
+        .build();
+    s.stop();
+  }
 }

+ 13 - 3
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -241,6 +241,8 @@ Trunk (Unreleased)
     HDFS-5431. Support cachepool-based limit management in path-based caching
     (awang via cmccabe)
 
+    HDFS-5636. Enforce a max TTL per cache pool. (awang via cmccabe)
+
   OPTIMIZATIONS
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
 
@@ -763,6 +765,9 @@ Release 2.4.0 - UNRELEASED
     HDFS-5540. Fix intermittent failure in TestBlocksWithNotEnoughRacks.
     (Binglin Chang via junping_du)
 
+    HDFS-2933. Improve DataNode Web UI Index Page. (Vivek Ganesan via
+    Arpit Agarwal)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
@@ -770,6 +775,8 @@ Release 2.4.0 - UNRELEASED
     HDFS-5341. Reduce fsdataset lock duration during directory scanning.
     (Qus-Jiawei via kihwal)
 
+    HDFS-5681. renewLease should not hold fsn write lock. (daryn via Kihwal)
+
   BUG FIXES
 
     HDFS-5034.  Remove debug prints from GetFileLinkInfo (Andrew Wang via Colin
@@ -820,6 +827,12 @@ Release 2.4.0 - UNRELEASED
 
     HDFS-5676. fix inconsistent synchronization of CachingStrategy (cmccabe)
 
+    HDFS-5691. Fix typo in ShortCircuitLocalRead document.
+    (Akira Ajisaka via suresh)
+
+    HDFS-5690. DataNode fails to start in secure mode when dfs.http.policy equals to 
+    HTTP_ONLY. (Haohui Mai via jing9)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -1132,9 +1145,6 @@ Release 2.1.1-beta - 2013-09-23
     HDFS-5047. Supress logging of full stack trace of quota and lease
     exceptions. (Robert Parker via kihwal)
 
-    HDFS-2933. Improve DataNode Web UI Index Page. (Vivek Ganesan via
-    Arpit Agarwal)
-
     HDFS-5111. Remove duplicated error message for snapshot commands when 
     processing invalid arguments. (jing9)
 

+ 13 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -1546,7 +1546,11 @@ public class DFSUtil {
    * Converts a time duration in milliseconds into DDD:HH:MM:SS format.
    */
   public static String durationToString(long durationMs) {
-    Preconditions.checkArgument(durationMs >= 0, "Invalid negative duration");
+    boolean negative = false;
+    if (durationMs < 0) {
+      negative = true;
+      durationMs = -durationMs;
+    }
     // Chop off the milliseconds
     long durationSec = durationMs / 1000;
     final int secondsPerMinute = 60;
@@ -1559,7 +1563,12 @@ public class DFSUtil {
     final long minutes = durationSec / secondsPerMinute;
     durationSec -= minutes * secondsPerMinute;
     final long seconds = durationSec;
-    return String.format("%03d:%02d:%02d:%02d", days, hours, minutes, seconds);
+    final long milliseconds = durationMs % 1000;
+    String format = "%03d:%02d:%02d:%02d.%03d";
+    if (negative)  {
+      format = "-" + format;
+    }
+    return String.format(format, days, hours, minutes, seconds, milliseconds);
   }
 
   /**
@@ -1571,9 +1580,9 @@ public class DFSUtil {
           + ": too short");
     }
     String ttlString = relTime.substring(0, relTime.length()-1);
-    int ttl;
+    long ttl;
     try {
-      ttl = Integer.parseInt(ttlString);
+      ttl = Long.parseLong(ttlString);
     } catch (NumberFormatException e) {
       throw new IOException("Unable to parse relative time value of " + relTime
           + ": " + ttlString + " is not a number");

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirective.java

@@ -52,6 +52,14 @@ public final class CacheDirective implements IntrusiveCollection.Element {
   private Element prev;
   private Element next;
 
+  public CacheDirective(CacheDirectiveInfo info) {
+    this(
+        info.getId(),
+        info.getPath().toUri().getPath(),
+        info.getReplication(),
+        info.getExpiration().getAbsoluteMillis());
+  }
+
   public CacheDirective(long id, String path,
       short replication, long expiryTime) {
     Preconditions.checkArgument(id > 0);

+ 25 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java

@@ -26,6 +26,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSUtil;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Describes a path-based cache directive.
  */
@@ -138,11 +140,22 @@ public class CacheDirectiveInfo {
    */
   public static class Expiration {
 
-    /** Denotes a CacheDirectiveInfo that never expires **/
-    public static final int EXPIRY_NEVER = -1;
+    /**
+     * The maximum value we accept for a relative expiry.
+     */
+    public static final long MAX_RELATIVE_EXPIRY_MS =
+        Long.MAX_VALUE / 4; // This helps prevent weird overflow bugs
+
+    /**
+     * An relative Expiration that never expires.
+     */
+    public static final Expiration NEVER = newRelative(MAX_RELATIVE_EXPIRY_MS);
 
     /**
      * Create a new relative Expiration.
+     * <p>
+     * Use {@link Expiration#NEVER} to indicate an Expiration that never
+     * expires.
      * 
      * @param ms how long until the CacheDirective expires, in milliseconds
      * @return A relative Expiration
@@ -153,6 +166,9 @@ public class CacheDirectiveInfo {
 
     /**
      * Create a new absolute Expiration.
+     * <p>
+     * Use {@link Expiration#NEVER} to indicate an Expiration that never
+     * expires.
      * 
      * @param date when the CacheDirective expires
      * @return An absolute Expiration
@@ -163,6 +179,9 @@ public class CacheDirectiveInfo {
 
     /**
      * Create a new absolute Expiration.
+     * <p>
+     * Use {@link Expiration#NEVER} to indicate an Expiration that never
+     * expires.
      * 
      * @param ms when the CacheDirective expires, in milliseconds since the Unix
      *          epoch.
@@ -176,6 +195,10 @@ public class CacheDirectiveInfo {
     private final boolean isRelative;
 
     private Expiration(long ms, boolean isRelative) {
+      if (isRelative) {
+        Preconditions.checkArgument(ms <= MAX_RELATIVE_EXPIRY_MS,
+            "Expiration time is too far in the future!");
+      }
       this.ms = ms;
       this.isRelative = isRelative;
     }

+ 71 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
 
 /**
  * CachePoolInfo describes a cache pool.
@@ -42,6 +43,20 @@ import org.apache.hadoop.fs.permission.FsPermission;
 public class CachePoolInfo {
   public static final Log LOG = LogFactory.getLog(CachePoolInfo.class);
 
+  /**
+   * Indicates that the pool does not have a maximum relative expiry.
+   */
+  public static final long RELATIVE_EXPIRY_NEVER =
+      Expiration.MAX_RELATIVE_EXPIRY_MS;
+  /**
+   * Default max relative expiry for cache pools.
+   */
+  public static final long DEFAULT_MAX_RELATIVE_EXPIRY =
+      RELATIVE_EXPIRY_NEVER;
+
+  public static final long LIMIT_UNLIMITED = Long.MAX_VALUE;
+  public static final long DEFAULT_LIMIT = LIMIT_UNLIMITED;
+
   final String poolName;
 
   @Nullable
@@ -56,14 +71,24 @@ public class CachePoolInfo {
   @Nullable
   Long limit;
 
+  @Nullable
+  Long maxRelativeExpiryMs;
+
   public CachePoolInfo(String poolName) {
     this.poolName = poolName;
   }
-  
+
+  /**
+   * @return Name of the pool.
+   */
   public String getPoolName() {
     return poolName;
   }
 
+  /**
+   * @return The owner of the pool. Along with the group and mode, determines
+   *         who has access to view and modify the pool.
+   */
   public String getOwnerName() {
     return ownerName;
   }
@@ -73,6 +98,10 @@ public class CachePoolInfo {
     return this;
   }
 
+  /**
+   * @return The group of the pool. Along with the owner and mode, determines
+   *         who has access to view and modify the pool.
+   */
   public String getGroupName() {
     return groupName;
   }
@@ -81,7 +110,11 @@ public class CachePoolInfo {
     this.groupName = groupName;
     return this;
   }
-  
+
+  /**
+   * @return Unix-style permissions of the pool. Along with the owner and group,
+   *         determines who has access to view and modify the pool.
+   */
   public FsPermission getMode() {
     return mode;
   }
@@ -91,6 +124,10 @@ public class CachePoolInfo {
     return this;
   }
 
+  /**
+   * @return The maximum aggregate number of bytes that can be cached by
+   *         directives in this pool.
+   */
   public Long getLimit() {
     return limit;
   }
@@ -100,6 +137,26 @@ public class CachePoolInfo {
     return this;
   }
 
+  /**
+   * @return The maximum relative expiration of directives of this pool in
+   *         milliseconds
+   */
+  public Long getMaxRelativeExpiryMs() {
+    return maxRelativeExpiryMs;
+  }
+
+  /**
+   * Set the maximum relative expiration of directives of this pool in
+   * milliseconds.
+   * 
+   * @param ms in milliseconds
+   * @return This builder, for call chaining.
+   */
+  public CachePoolInfo setMaxRelativeExpiryMs(Long ms) {
+    this.maxRelativeExpiryMs = ms;
+    return this;
+  }
+
   public String toString() {
     return new StringBuilder().append("{").
       append("poolName:").append(poolName).
@@ -108,6 +165,7 @@ public class CachePoolInfo {
       append(", mode:").append((mode == null) ? "null" :
           String.format("0%03o", mode.toShort())).
       append(", limit:").append(limit).
+      append(", maxRelativeExpiryMs:").append(maxRelativeExpiryMs).
       append("}").toString();
   }
   
@@ -125,6 +183,7 @@ public class CachePoolInfo {
         append(groupName, other.groupName).
         append(mode, other.mode).
         append(limit, other.limit).
+        append(maxRelativeExpiryMs, other.maxRelativeExpiryMs).
         isEquals();
   }
 
@@ -136,6 +195,7 @@ public class CachePoolInfo {
         append(groupName).
         append(mode).
         append(limit).
+        append(maxRelativeExpiryMs).
         hashCode();
   }
 
@@ -146,6 +206,15 @@ public class CachePoolInfo {
     if ((info.getLimit() != null) && (info.getLimit() < 0)) {
       throw new InvalidRequestException("Limit is negative.");
     }
+    if (info.getMaxRelativeExpiryMs() != null) {
+      long maxRelativeExpiryMs = info.getMaxRelativeExpiryMs();
+      if (maxRelativeExpiryMs < 0l) {
+        throw new InvalidRequestException("Max relative expiry is negative.");
+      }
+      if (maxRelativeExpiryMs > Expiration.MAX_RELATIVE_EXPIRY_MS) {
+        throw new InvalidRequestException("Max relative expiry is too big.");
+      }
+    }
     validateName(info.poolName);
   }
 

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -1839,6 +1839,9 @@ public class PBHelper {
     if (info.getLimit() != null) {
       builder.setLimit(info.getLimit());
     }
+    if (info.getMaxRelativeExpiryMs() != null) {
+      builder.setMaxRelativeExpiry(info.getMaxRelativeExpiryMs());
+    }
     return builder.build();
   }
 
@@ -1858,6 +1861,9 @@ public class PBHelper {
     if (proto.hasLimit())  {
       info.setLimit(proto.getLimit());
     }
+    if (proto.hasMaxRelativeExpiry()) {
+      info.setMaxRelativeExpiryMs(proto.getMaxRelativeExpiry());
+    }
     return info;
   }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java

@@ -365,7 +365,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       if (directive.getExpiryTime() > 0 && directive.getExpiryTime() <= now) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Skipping directive id " + directive.getId()
-              + " because it has expired (" + directive.getExpiryTime() + ">="
+              + " because it has expired (" + directive.getExpiryTime() + "<="
               + now + ")");
         }
         continue;

+ 9 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java

@@ -87,6 +87,7 @@ public class SecureDataNodeStarter implements Daemon {
   public static SecureResources getSecureResources(Configuration conf)
       throws Exception {
     HttpConfig.Policy policy = DFSUtil.getHttpPolicy(conf);
+    boolean isSecure = UserGroupInformation.isSecurityEnabled();
 
     // Obtain secure port for data streaming to datanode
     InetSocketAddress streamingAddr  = DataNode.getStreamingAddr(conf);
@@ -106,6 +107,11 @@ public class SecureDataNodeStarter implements Daemon {
               + ss.getLocalPort());
     }
 
+    if (ss.getLocalPort() > 1023 && isSecure) {
+      throw new RuntimeException(
+        "Cannot start secure datanode with unprivileged RPC ports");
+    }
+
     System.err.println("Opened streaming server at " + streamingAddr);
 
     // Bind a port for the web server. The code intends to bind HTTP server to
@@ -126,9 +132,9 @@ public class SecureDataNodeStarter implements Daemon {
       System.err.println("Successfully obtained privileged resources (streaming port = "
           + ss + " ) (http listener port = " + listener.getConnection() +")");
 
-      if ((ss.getLocalPort() > 1023 || listener.getPort() > 1023) &&
-          UserGroupInformation.isSecurityEnabled()) {
-        throw new RuntimeException("Cannot start secure datanode with unprivileged ports");
+      if (listener.getPort() > 1023 && isSecure) {
+        throw new RuntimeException(
+            "Cannot start secure datanode with unprivileged HTTP ports");
       }
       System.err.println("Opened info server at " + infoSocAddr);
     }

+ 134 - 51
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java

@@ -32,6 +32,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Date;
 import java.util.EnumSet;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -55,6 +56,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirective;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
@@ -322,27 +324,48 @@ public final class CacheManager {
    * {@link CacheDirectiveInfo.Expiration}. This converts a relative Expiration
    * into an absolute time based on the local clock.
    * 
-   * @param directive from which to get the expiry time
-   * @param defaultValue to use if Expiration is not set
-   * @return Absolute expiry time in milliseconds since Unix epoch
-   * @throws InvalidRequestException if the Expiration is invalid
-   */
-  private static long validateExpiryTime(CacheDirectiveInfo directive,
-      long defaultValue) throws InvalidRequestException {
-    long expiryTime;
-    CacheDirectiveInfo.Expiration expiration = directive.getExpiration();
-    if (expiration != null) {
-      if (expiration.getMillis() < 0) {
-        throw new InvalidRequestException("Cannot set a negative expiration: "
-            + expiration.getMillis());
-      }
-      // Converts a relative duration into an absolute time based on the local
-      // clock
-      expiryTime = expiration.getAbsoluteMillis();
+   * @param info to validate.
+   * @param maxRelativeExpiryTime of the info's pool.
+   * @return the expiration time, or the pool's max absolute expiration if the
+   *         info's expiration was not set.
+   * @throws InvalidRequestException if the info's Expiration is invalid.
+   */
+  private static long validateExpiryTime(CacheDirectiveInfo info,
+      long maxRelativeExpiryTime) throws InvalidRequestException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Validating directive " + info
+          + " pool maxRelativeExpiryTime " + maxRelativeExpiryTime);
+    }
+    final long now = new Date().getTime();
+    final long maxAbsoluteExpiryTime = now + maxRelativeExpiryTime;
+    if (info == null || info.getExpiration() == null) {
+      return maxAbsoluteExpiryTime;
+    }
+    Expiration expiry = info.getExpiration();
+    if (expiry.getMillis() < 0l) {
+      throw new InvalidRequestException("Cannot set a negative expiration: "
+          + expiry.getMillis());
+    }
+    long relExpiryTime, absExpiryTime;
+    if (expiry.isRelative()) {
+      relExpiryTime = expiry.getMillis();
+      absExpiryTime = now + relExpiryTime;
     } else {
-      expiryTime = defaultValue;
+      absExpiryTime = expiry.getMillis();
+      relExpiryTime = absExpiryTime - now;
     }
-    return expiryTime;
+    // Need to cap the expiry so we don't overflow a long when doing math
+    if (relExpiryTime > Expiration.MAX_RELATIVE_EXPIRY_MS) {
+      throw new InvalidRequestException("Expiration "
+          + expiry.toString() + " is too far in the future!");
+    }
+    // Fail if the requested expiry is greater than the max
+    if (relExpiryTime > maxRelativeExpiryTime) {
+      throw new InvalidRequestException("Expiration " + expiry.toString()
+          + " exceeds the max relative expiration time of "
+          + maxRelativeExpiryTime + " ms.");
+    }
+    return absExpiryTime;
   }
 
   /**
@@ -357,6 +380,9 @@ public final class CacheManager {
   private void checkLimit(CachePool pool, String path,
       short replication) throws InvalidRequestException {
     CacheDirectiveStats stats = computeNeeded(path, replication);
+    if (pool.getLimit() == CachePoolInfo.LIMIT_UNLIMITED) {
+      return;
+    }
     if (pool.getBytesNeeded() + (stats.getBytesNeeded() * replication) > pool
         .getLimit()) {
       throw new InvalidRequestException("Caching path " + path + " of size "
@@ -461,17 +487,13 @@ public final class CacheManager {
   }
 
   /**
-   * To be called only from the edit log loading code
+   * Adds a directive, skipping most error checking. This should only be called
+   * internally in special scenarios like edit log replay.
    */
   CacheDirectiveInfo addDirectiveFromEditLog(CacheDirectiveInfo directive)
       throws InvalidRequestException {
     long id = directive.getId();
-    CacheDirective entry =
-        new CacheDirective(
-            directive.getId(),
-            directive.getPath().toUri().getPath(),
-            directive.getReplication(),
-            directive.getExpiration().getAbsoluteMillis());
+    CacheDirective entry = new CacheDirective(directive);
     CachePool pool = cachePools.get(directive.getPool());
     addInternal(entry, pool);
     if (nextDirectiveId <= id) {
@@ -490,8 +512,7 @@ public final class CacheManager {
       checkWritePermission(pc, pool);
       String path = validatePath(info);
       short replication = validateReplication(info, (short)1);
-      long expiryTime = validateExpiryTime(info,
-          CacheDirectiveInfo.Expiration.EXPIRY_NEVER);
+      long expiryTime = validateExpiryTime(info, pool.getMaxRelativeExpiryMs());
       // Do quota validation if required
       if (!flags.contains(CacheFlag.FORCE)) {
         // Can't kick and wait if caching is disabled
@@ -513,6 +534,56 @@ public final class CacheManager {
     return directive.toInfo();
   }
 
+  /**
+   * Factory method that makes a new CacheDirectiveInfo by applying fields in a
+   * CacheDirectiveInfo to an existing CacheDirective.
+   * 
+   * @param info with some or all fields set.
+   * @param defaults directive providing default values for unset fields in
+   *          info.
+   * 
+   * @return new CacheDirectiveInfo of the info applied to the defaults.
+   */
+  private static CacheDirectiveInfo createFromInfoAndDefaults(
+      CacheDirectiveInfo info, CacheDirective defaults) {
+    // Initialize the builder with the default values
+    CacheDirectiveInfo.Builder builder =
+        new CacheDirectiveInfo.Builder(defaults.toInfo());
+    // Replace default with new value if present
+    if (info.getPath() != null) {
+      builder.setPath(info.getPath());
+    }
+    if (info.getReplication() != null) {
+      builder.setReplication(info.getReplication());
+    }
+    if (info.getPool() != null) {
+      builder.setPool(info.getPool());
+    }
+    if (info.getExpiration() != null) {
+      builder.setExpiration(info.getExpiration());
+    }
+    return builder.build();
+  }
+
+  /**
+   * Modifies a directive, skipping most error checking. This is for careful
+   * internal use only. modifyDirective can be non-deterministic since its error
+   * checking depends on current system time, which poses a problem for edit log
+   * replay.
+   */
+  void modifyDirectiveFromEditLog(CacheDirectiveInfo info)
+      throws InvalidRequestException {
+    // Check for invalid IDs.
+    Long id = info.getId();
+    if (id == null) {
+      throw new InvalidRequestException("Must supply an ID.");
+    }
+    CacheDirective prevEntry = getById(id);
+    CacheDirectiveInfo newInfo = createFromInfoAndDefaults(info, prevEntry);
+    removeInternal(prevEntry);
+    addInternal(new CacheDirective(newInfo), getCachePool(newInfo.getPool()));
+  }
+
   public void modifyDirective(CacheDirectiveInfo info,
       FSPermissionChecker pc, EnumSet<CacheFlag> flags) throws IOException {
     assert namesystem.hasWriteLock();
@@ -527,33 +598,38 @@ public final class CacheManager {
       }
       CacheDirective prevEntry = getById(id);
       checkWritePermission(pc, prevEntry.getPool());
-      String path = prevEntry.getPath();
-      if (info.getPath() != null) {
-        path = validatePath(info);
-      }
 
-      short replication = prevEntry.getReplication();
-      replication = validateReplication(info, replication);
-
-      long expiryTime = prevEntry.getExpiryTime();
-      expiryTime = validateExpiryTime(info, expiryTime);
-
-      CachePool pool = prevEntry.getPool();
-      if (info.getPool() != null) {
-        pool = getCachePool(validatePoolName(info));
-        checkWritePermission(pc, pool);
+      // Fill in defaults
+      CacheDirectiveInfo infoWithDefaults =
+          createFromInfoAndDefaults(info, prevEntry);
+      CacheDirectiveInfo.Builder builder =
+          new CacheDirectiveInfo.Builder(infoWithDefaults);
+
+      // Do validation
+      validatePath(infoWithDefaults);
+      validateReplication(infoWithDefaults, (short)-1);
+      // Need to test the pool being set here to avoid rejecting a modify for a
+      // directive that's already been forced into a pool
+      CachePool srcPool = prevEntry.getPool();
+      CachePool destPool = getCachePool(validatePoolName(infoWithDefaults));
+      if (!srcPool.getPoolName().equals(destPool.getPoolName())) {
+        checkWritePermission(pc, destPool);
         if (!flags.contains(CacheFlag.FORCE)) {
-          // Can't kick and wait if caching is disabled
-          if (monitor != null) {
-            monitor.waitForRescan();
-          }
-          checkLimit(pool, path, replication);
+          checkLimit(destPool, infoWithDefaults.getPath().toUri().getPath(),
+              infoWithDefaults.getReplication());
         }
       }
+      // Verify the expiration against the destination pool
+      validateExpiryTime(infoWithDefaults, destPool.getMaxRelativeExpiryMs());
+
+      // Indicate changes to the CRM
+      if (monitor != null) {
+        monitor.setNeedsRescan();
+      }
+
+      // Validation passed
       removeInternal(prevEntry);
-      CacheDirective newEntry =
-          new CacheDirective(id, path, replication, expiryTime);
-      addInternal(newEntry, pool);
+      addInternal(new CacheDirective(builder.build()), destPool);
     } catch (IOException e) {
       LOG.warn("modifyDirective of " + idString + " failed: ", e);
       throw e;
@@ -562,7 +638,7 @@ public final class CacheManager {
         info+ ".");
   }
 
-  public void removeInternal(CacheDirective directive)
+  private void removeInternal(CacheDirective directive)
       throws InvalidRequestException {
     assert namesystem.hasWriteLock();
     // Remove the corresponding entry in directivesByPath.
@@ -734,6 +810,13 @@ public final class CacheManager {
           monitor.setNeedsRescan();
         }
       }
+      if (info.getMaxRelativeExpiryMs() != null) {
+        final Long maxRelativeExpiry = info.getMaxRelativeExpiryMs();
+        pool.setMaxRelativeExpiryMs(maxRelativeExpiry);
+        bld.append(prefix).append("set maxRelativeExpiry to "
+            + maxRelativeExpiry);
+        prefix = "; ";
+      }
       if (prefix.isEmpty()) {
         bld.append("no changes.");
       }

+ 26 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java

@@ -49,8 +49,6 @@ import com.google.common.base.Preconditions;
 public final class CachePool {
   public static final Log LOG = LogFactory.getLog(CachePool.class);
 
-  public static final long DEFAULT_LIMIT = Long.MAX_VALUE;
-
   @Nonnull
   private final String poolName;
 
@@ -76,6 +74,12 @@ public final class CachePool {
    */
   private long limit;
 
+  /**
+   * Maximum duration that a CacheDirective in this pool remains valid,
+   * in milliseconds.
+   */
+  private long maxRelativeExpiryMs;
+
   private long bytesNeeded;
   private long bytesCached;
   private long filesNeeded;
@@ -122,9 +126,12 @@ public final class CachePool {
     FsPermission mode = (info.getMode() == null) ? 
         FsPermission.getCachePoolDefault() : info.getMode();
     long limit = info.getLimit() == null ?
-        DEFAULT_LIMIT : info.getLimit();
+        CachePoolInfo.DEFAULT_LIMIT : info.getLimit();
+    long maxRelativeExpiry = info.getMaxRelativeExpiryMs() == null ?
+        CachePoolInfo.DEFAULT_MAX_RELATIVE_EXPIRY :
+        info.getMaxRelativeExpiryMs();
     return new CachePool(info.getPoolName(),
-        ownerName, groupName, mode, limit);
+        ownerName, groupName, mode, limit, maxRelativeExpiry);
   }
 
   /**
@@ -134,11 +141,11 @@ public final class CachePool {
   static CachePool createFromInfo(CachePoolInfo info) {
     return new CachePool(info.getPoolName(),
         info.getOwnerName(), info.getGroupName(),
-        info.getMode(), info.getLimit());
+        info.getMode(), info.getLimit(), info.getMaxRelativeExpiryMs());
   }
 
   CachePool(String poolName, String ownerName, String groupName,
-      FsPermission mode, long limit) {
+      FsPermission mode, long limit, long maxRelativeExpiry) {
     Preconditions.checkNotNull(poolName);
     Preconditions.checkNotNull(ownerName);
     Preconditions.checkNotNull(groupName);
@@ -148,6 +155,7 @@ public final class CachePool {
     this.groupName = groupName;
     this.mode = new FsPermission(mode);
     this.limit = limit;
+    this.maxRelativeExpiryMs = maxRelativeExpiry;
   }
 
   public String getPoolName() {
@@ -190,6 +198,15 @@ public final class CachePool {
     return this;
   }
 
+  public long getMaxRelativeExpiryMs() {
+    return maxRelativeExpiryMs;
+  }
+
+  public CachePool setMaxRelativeExpiryMs(long expiry) {
+    this.maxRelativeExpiryMs = expiry;
+    return this;
+  }
+
   /**
    * Get either full or partial information about this CachePool.
    *
@@ -207,7 +224,8 @@ public final class CachePool {
     return info.setOwnerName(ownerName).
         setGroupName(groupName).
         setMode(new FsPermission(mode)).
-        setLimit(limit);
+        setLimit(limit).
+        setMaxRelativeExpiryMs(maxRelativeExpiryMs);
   }
 
   /**
@@ -300,6 +318,7 @@ public final class CachePool {
         append(", groupName:").append(groupName).
         append(", mode:").append(mode).
         append(", limit:").append(limit).
+        append(", maxRelativeExpiryMs:").append(maxRelativeExpiryMs).
         append(" }").toString();
   }
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -652,8 +652,8 @@ public class FSEditLogLoader {
     case OP_MODIFY_CACHE_DIRECTIVE: {
       ModifyCacheDirectiveInfoOp modifyOp =
           (ModifyCacheDirectiveInfoOp) op;
-      fsNamesys.getCacheManager().modifyDirective(
-          modifyOp.directive, null, EnumSet.of(CacheFlag.FORCE));
+      fsNamesys.getCacheManager().modifyDirectiveFromEditLog(
+          modifyOp.directive);
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
       }

+ 22 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -604,18 +604,22 @@ public class FSImageSerialization {
     final String groupName = info.getGroupName();
     final Long limit = info.getLimit();
     final FsPermission mode = info.getMode();
+    final Long maxRelativeExpiry = info.getMaxRelativeExpiryMs();
 
-    boolean hasOwner, hasGroup, hasMode, hasLimit;
+    boolean hasOwner, hasGroup, hasMode, hasLimit, hasMaxRelativeExpiry;
     hasOwner = ownerName != null;
     hasGroup = groupName != null;
     hasMode = mode != null;
     hasLimit = limit != null;
+    hasMaxRelativeExpiry = maxRelativeExpiry != null;
 
     int flags =
         (hasOwner ? 0x1 : 0) |
         (hasGroup ? 0x2 : 0) |
         (hasMode  ? 0x4 : 0) |
-        (hasLimit ? 0x8 : 0);
+        (hasLimit ? 0x8 : 0) |
+        (hasMaxRelativeExpiry ? 0x10 : 0);
+
     writeInt(flags, out);
 
     if (hasOwner) {
@@ -630,6 +634,9 @@ public class FSImageSerialization {
     if (hasLimit) {
       writeLong(limit, out);
     }
+    if (hasMaxRelativeExpiry) {
+      writeLong(maxRelativeExpiry, out);
+    }
   }
 
   public static CachePoolInfo readCachePoolInfo(DataInput in)
@@ -649,7 +656,10 @@ public class FSImageSerialization {
     if ((flags & 0x8) != 0) {
       info.setLimit(readLong(in));
     }
-    if ((flags & ~0xF) != 0) {
+    if ((flags & 0x10) != 0) {
+      info.setMaxRelativeExpiryMs(readLong(in));
+    }
+    if ((flags & ~0x1F) != 0) {
       throw new IOException("Unknown flag in CachePoolInfo: " + flags);
     }
     return info;
@@ -663,6 +673,7 @@ public class FSImageSerialization {
     final String groupName = info.getGroupName();
     final Long limit = info.getLimit();
     final FsPermission mode = info.getMode();
+    final Long maxRelativeExpiry = info.getMaxRelativeExpiryMs();
 
     if (ownerName != null) {
       XMLUtils.addSaxString(contentHandler, "OWNERNAME", ownerName);
@@ -677,6 +688,10 @@ public class FSImageSerialization {
       XMLUtils.addSaxString(contentHandler, "LIMIT",
           Long.toString(limit));
     }
+    if (maxRelativeExpiry != null) {
+      XMLUtils.addSaxString(contentHandler, "MAXRELATIVEEXPIRY",
+          Long.toString(maxRelativeExpiry));
+    }
   }
 
   public static CachePoolInfo readCachePoolInfo(Stanza st)
@@ -695,6 +710,10 @@ public class FSImageSerialization {
     if (st.hasChildren("LIMIT")) {
       info.setLimit(Long.parseLong(st.getValue("LIMIT")));
     }
+    if (st.hasChildren("MAXRELATIVEEXPIRY")) {
+      info.setMaxRelativeExpiryMs(
+          Long.parseLong(st.getValue("MAXRELATIVEEXPIRY")));
+    }
     return info;
   }
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -4001,13 +4001,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
   void renewLease(String holder) throws IOException {
     checkOperation(OperationCategory.WRITE);
-    writeLock();
+    readLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot renew lease for " + holder);
       leaseManager.renewLease(holder);
     } finally {
-      writeUnlock();
+      readUnlock();
     }
   }
 

+ 138 - 61
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java

@@ -35,14 +35,12 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolStats;
-import org.apache.hadoop.hdfs.server.namenode.CachePool;
 import org.apache.hadoop.hdfs.tools.TableListing.Justification;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 
@@ -120,6 +118,23 @@ public class CacheAdmin extends Configured implements Tool {
     return listing;
   }
 
+  /**
+   * Parses a time-to-live value from a string
+   * @return The ttl in milliseconds
+   * @throws IOException if it could not be parsed
+   */
+  private static Long parseTtlString(String maxTtlString) throws IOException {
+    Long maxTtl = null;
+    if (maxTtlString != null) {
+      if (maxTtlString.equalsIgnoreCase("never")) {
+        maxTtl = CachePoolInfo.RELATIVE_EXPIRY_NEVER;
+      } else {
+        maxTtl = DFSUtil.parseRelativeTime(maxTtlString);
+      }
+    }
+    return maxTtl;
+  }
+
   interface Command {
     String getName();
     String getShortUsage();
@@ -154,7 +169,7 @@ public class CacheAdmin extends Configured implements Tool {
       listing.addRow("<replication>", "The cache replication factor to use. " +
           "Defaults to 1.");
       listing.addRow("<time-to-live>", "How long the directive is " +
-          "valid. Can be specified in minutes, hours, and days via e.g. " +
+          "valid. Can be specified in minutes, hours, and days, e.g. " +
           "30m, 4h, 2d. Valid units are [smhd]." +
           " If unspecified, the directive never expires.");
       return getShortUsage() + "\n" +
@@ -309,7 +324,7 @@ public class CacheAdmin extends Configured implements Tool {
           "added. You must have write permission on the cache pool "
           + "in order to move a directive into it. (optional)");
       listing.addRow("<time-to-live>", "How long the directive is " +
-          "valid. Can be specified in minutes, hours, and days via e.g. " +
+          "valid. Can be specified in minutes, hours, and days, e.g. " +
           "30m, 4h, 2d. Valid units are [smhd]." +
           " If unspecified, the directive never expires.");
       return getShortUsage() + "\n" +
@@ -419,22 +434,27 @@ public class CacheAdmin extends Configured implements Tool {
         System.err.println("Usage is " + getShortUsage());
         return 1;
       }
-      DistributedFileSystem dfs = getDFS(conf);
-      RemoteIterator<CacheDirectiveEntry> iter =
-          dfs.listCacheDirectives(
-              new CacheDirectiveInfo.Builder().
-                  setPath(new Path(path)).build());
       int exitCode = 0;
-      while (iter.hasNext()) {
-        CacheDirectiveEntry entry = iter.next();
-        try {
-          dfs.removeCacheDirective(entry.getInfo().getId());
-          System.out.println("Removed cache directive " +
-              entry.getInfo().getId());
-        } catch (IOException e) {
-          System.err.println(prettifyException(e));
-          exitCode = 2;
+      try {
+        DistributedFileSystem dfs = getDFS(conf);
+        RemoteIterator<CacheDirectiveEntry> iter =
+            dfs.listCacheDirectives(
+                new CacheDirectiveInfo.Builder().
+                    setPath(new Path(path)).build());
+        while (iter.hasNext()) {
+          CacheDirectiveEntry entry = iter.next();
+          try {
+            dfs.removeCacheDirective(entry.getInfo().getId());
+            System.out.println("Removed cache directive " +
+                entry.getInfo().getId());
+          } catch (IOException e) {
+            System.err.println(prettifyException(e));
+            exitCode = 2;
+          }
         }
+      } catch (IOException e) {
+        System.err.println(prettifyException(e));
+        exitCode = 2;
       }
       if (exitCode == 0) {
         System.out.println("Removed every cache directive with path " +
@@ -500,41 +520,46 @@ public class CacheAdmin extends Configured implements Tool {
                     addField("FILES_CACHED", Justification.RIGHT);
       }
       TableListing tableListing = tableBuilder.build();
-
-      DistributedFileSystem dfs = getDFS(conf);
-      RemoteIterator<CacheDirectiveEntry> iter =
-          dfs.listCacheDirectives(builder.build());
-      int numEntries = 0;
-      while (iter.hasNext()) {
-        CacheDirectiveEntry entry = iter.next();
-        CacheDirectiveInfo directive = entry.getInfo();
-        CacheDirectiveStats stats = entry.getStats();
-        List<String> row = new LinkedList<String>();
-        row.add("" + directive.getId());
-        row.add(directive.getPool());
-        row.add("" + directive.getReplication());
-        String expiry;
-        if (directive.getExpiration().getMillis() ==
-            CacheDirectiveInfo.Expiration.EXPIRY_NEVER) {
-          expiry = "never";
-        } else {
-          expiry = directive.getExpiration().toString();
+      try {
+        DistributedFileSystem dfs = getDFS(conf);
+        RemoteIterator<CacheDirectiveEntry> iter =
+            dfs.listCacheDirectives(builder.build());
+        int numEntries = 0;
+        while (iter.hasNext()) {
+          CacheDirectiveEntry entry = iter.next();
+          CacheDirectiveInfo directive = entry.getInfo();
+          CacheDirectiveStats stats = entry.getStats();
+          List<String> row = new LinkedList<String>();
+          row.add("" + directive.getId());
+          row.add(directive.getPool());
+          row.add("" + directive.getReplication());
+          String expiry;
+          // This is effectively never, round for nice printing
+          if (directive.getExpiration().getMillis() >
+              Expiration.MAX_RELATIVE_EXPIRY_MS / 2) {
+            expiry = "never";
+          } else {
+            expiry = directive.getExpiration().toString();
+          }
+          row.add(expiry);
+          row.add(directive.getPath().toUri().getPath());
+          if (printStats) {
+            row.add("" + stats.getBytesNeeded());
+            row.add("" + stats.getBytesCached());
+            row.add("" + stats.getFilesNeeded());
+            row.add("" + stats.getFilesCached());
+          }
+          tableListing.addRow(row.toArray(new String[0]));
+          numEntries++;
         }
-        row.add(expiry);
-        row.add(directive.getPath().toUri().getPath());
-        if (printStats) {
-          row.add("" + stats.getBytesNeeded());
-          row.add("" + stats.getBytesCached());
-          row.add("" + stats.getFilesNeeded());
-          row.add("" + stats.getFilesCached());
+        System.out.print(String.format("Found %d entr%s\n",
+            numEntries, numEntries == 1 ? "y" : "ies"));
+        if (numEntries > 0) {
+          System.out.print(tableListing);
         }
-        tableListing.addRow(row.toArray(new String[0]));
-        numEntries++;
-      }
-      System.out.print(String.format("Found %d entr%s\n",
-          numEntries, numEntries == 1 ? "y" : "ies"));
-      if (numEntries > 0) {
-        System.out.print(tableListing);
+      } catch (IOException e) {
+        System.err.println(prettifyException(e));
+        return 2;
       }
       return 0;
     }
@@ -552,7 +577,8 @@ public class CacheAdmin extends Configured implements Tool {
     @Override
     public String getShortUsage() {
       return "[" + NAME + " <name> [-owner <owner>] " +
-          "[-group <group>] [-mode <mode>] [-limit <limit>]]\n";
+          "[-group <group>] [-mode <mode>] [-limit <limit>] " +
+          "[-maxttl <maxTtl>]\n";
     }
 
     @Override
@@ -571,7 +597,11 @@ public class CacheAdmin extends Configured implements Tool {
       listing.addRow("<limit>", "The maximum number of bytes that can be " +
           "cached by directives in this pool, in aggregate. By default, " +
           "no limit is set.");
-
+      listing.addRow("<maxTtl>", "The maximum allowed time-to-live for " +
+          "directives being added to the pool. This can be specified in " +
+          "seconds, minutes, hours, and days, e.g. 120s, 30m, 4h, 2d. " +
+          "Valid units are [smhd]. By default, no maximum is set. " +
+          "This can also be manually specified by \"never\".");
       return getShortUsage() + "\n" +
           "Add a new cache pool.\n\n" + 
           listing.toString();
@@ -605,6 +635,18 @@ public class CacheAdmin extends Configured implements Tool {
         long limit = Long.parseLong(limitString);
         info.setLimit(limit);
       }
+      String maxTtlString = StringUtils.popOptionWithArgument("-maxTtl", args);
+      try {
+        Long maxTtl = parseTtlString(maxTtlString);
+        if (maxTtl != null) {
+          info.setMaxRelativeExpiryMs(maxTtl);
+        }
+      } catch (IOException e) {
+        System.err.println(
+            "Error while parsing maxTtl value: " + e.getMessage());
+        return 1;
+      }
+
       if (!args.isEmpty()) {
         System.err.print("Can't understand arguments: " +
           Joiner.on(" ").join(args) + "\n");
@@ -615,7 +657,8 @@ public class CacheAdmin extends Configured implements Tool {
       try {
         dfs.addCachePool(info);
       } catch (IOException e) {
-        throw new RemoteException(e.getClass().getName(), e.getMessage());
+        System.err.println(prettifyException(e));
+        return 2;
       }
       System.out.println("Successfully added cache pool " + name + ".");
       return 0;
@@ -632,7 +675,8 @@ public class CacheAdmin extends Configured implements Tool {
     @Override
     public String getShortUsage() {
       return "[" + getName() + " <name> [-owner <owner>] " +
-          "[-group <group>] [-mode <mode>] [-limit <limit>]]\n";
+          "[-group <group>] [-mode <mode>] [-limit <limit>] " +
+          "[-maxTtl <maxTtl>]]\n";
     }
 
     @Override
@@ -645,6 +689,8 @@ public class CacheAdmin extends Configured implements Tool {
       listing.addRow("<mode>", "Unix-style permissions of the pool in octal.");
       listing.addRow("<limit>", "Maximum number of bytes that can be cached " +
           "by this pool.");
+      listing.addRow("<maxTtl>", "The maximum allowed time-to-live for " +
+          "directives being added to the pool.");
 
       return getShortUsage() + "\n" +
           WordUtils.wrap("Modifies the metadata of an existing cache pool. " +
@@ -663,6 +709,15 @@ public class CacheAdmin extends Configured implements Tool {
       String limitString = StringUtils.popOptionWithArgument("-limit", args);
       Long limit = (limitString == null) ?
           null : Long.parseLong(limitString);
+      String maxTtlString = StringUtils.popOptionWithArgument("-maxTtl", args);
+      Long maxTtl = null;
+      try {
+        maxTtl = parseTtlString(maxTtlString);
+      } catch (IOException e) {
+        System.err.println(
+            "Error while parsing maxTtl value: " + e.getMessage());
+        return 1;
+      }
       String name = StringUtils.popFirstNonOption(args);
       if (name == null) {
         System.err.println("You must specify a name when creating a " +
@@ -693,6 +748,10 @@ public class CacheAdmin extends Configured implements Tool {
         info.setLimit(limit);
         changed = true;
       }
+      if (maxTtl != null) {
+        info.setMaxRelativeExpiryMs(maxTtl);
+        changed = true;
+      }
       if (!changed) {
         System.err.println("You must specify at least one attribute to " +
             "change in the cache pool.");
@@ -702,7 +761,8 @@ public class CacheAdmin extends Configured implements Tool {
       try {
         dfs.modifyCachePool(info);
       } catch (IOException e) {
-        throw new RemoteException(e.getClass().getName(), e.getMessage());
+        System.err.println(prettifyException(e));
+        return 2;
       }
       System.out.print("Successfully modified cache pool " + name);
       String prefix = " to have ";
@@ -722,6 +782,9 @@ public class CacheAdmin extends Configured implements Tool {
         System.out.print(prefix + "limit " + limit);
         prefix = " and ";
       }
+      if (maxTtl != null) {
+        System.out.print(prefix + "max time-to-live " + maxTtlString);
+      }
       System.out.print("\n");
       return 0;
     }
@@ -765,7 +828,8 @@ public class CacheAdmin extends Configured implements Tool {
       try {
         dfs.removeCachePool(name);
       } catch (IOException e) {
-        throw new RemoteException(e.getClass().getName(), e.getMessage());
+        System.err.println(prettifyException(e));
+        return 2;
       }
       System.out.println("Successfully removed cache pool " + name + ".");
       return 0;
@@ -813,7 +877,8 @@ public class CacheAdmin extends Configured implements Tool {
           addField("OWNER", Justification.LEFT).
           addField("GROUP", Justification.LEFT).
           addField("MODE", Justification.LEFT).
-          addField("LIMIT", Justification.RIGHT);
+          addField("LIMIT", Justification.RIGHT).
+          addField("MAXTTL", Justification.RIGHT);
       if (printStats) {
         builder.
             addField("BYTES_NEEDED", Justification.RIGHT).
@@ -837,12 +902,23 @@ public class CacheAdmin extends Configured implements Tool {
             row.add(info.getMode() != null ? info.getMode().toString() : null);
             Long limit = info.getLimit();
             String limitString;
-            if (limit != null && limit.equals(CachePool.DEFAULT_LIMIT)) {
+            if (limit != null && limit.equals(CachePoolInfo.LIMIT_UNLIMITED)) {
               limitString = "unlimited";
             } else {
               limitString = "" + limit;
             }
             row.add(limitString);
+            Long maxTtl = info.getMaxRelativeExpiryMs();
+            String maxTtlString = null;
+
+            if (maxTtl != null) {
+              if (maxTtl.longValue() == CachePoolInfo.RELATIVE_EXPIRY_NEVER) {
+                maxTtlString  = "never";
+              } else {
+                maxTtlString = DFSUtil.durationToString(maxTtl);
+              }
+            }
+            row.add(maxTtlString);
             if (printStats) {
               CachePoolStats stats = entry.getStats();
               row.add(Long.toString(stats.getBytesNeeded()));
@@ -859,7 +935,8 @@ public class CacheAdmin extends Configured implements Tool {
           }
         }
       } catch (IOException e) {
-        throw new RemoteException(e.getClass().getName(), e.getMessage());
+        System.err.println(prettifyException(e));
+        return 2;
       }
       System.out.print(String.format("Found %d result%s.\n", numResults,
           (numResults == 1 ? "" : "s")));

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -435,6 +435,7 @@ message CachePoolInfoProto {
   optional string groupName = 3;
   optional int32 mode = 4;
   optional int64 limit = 5;
+  optional int64 maxRelativeExpiry = 6;
 }
 
 message CachePoolStatsProto {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/ShortCircuitLocalReads.apt.vm

@@ -73,7 +73,7 @@ HDFS Short-Circuit Local Reads
 
   This configuration parameter turns on short-circuit local reads.
 
-  * dfs.client.read.shortcircuit.skip.checkusm
+  * dfs.client.read.shortcircuit.skip.checksum
 
   If this configuration parameter is set, short-circuit local reads will skip
   checksums.  This is normally not recommended, but it may be useful for

+ 10 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java

@@ -62,7 +62,6 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Shell;
 import org.junit.Assume;
 import org.junit.Before;
@@ -730,16 +729,15 @@ public class TestDFSUtil {
 
   @Test(timeout=1000)
   public void testDurationToString() throws Exception {
-    assertEquals("000:00:00:00", DFSUtil.durationToString(0));
-    try {
-      DFSUtil.durationToString(-199);
-    } catch (IllegalArgumentException e) {
-      GenericTestUtils.assertExceptionContains("Invalid negative duration", e);
-    }
-    assertEquals("001:01:01:01",
+    assertEquals("000:00:00:00.000", DFSUtil.durationToString(0));
+    assertEquals("001:01:01:01.000",
         DFSUtil.durationToString(((24*60*60)+(60*60)+(60)+1)*1000));
-    assertEquals("000:23:59:59",
-        DFSUtil.durationToString(((23*60*60)+(59*60)+(59))*1000));
+    assertEquals("000:23:59:59.999",
+        DFSUtil.durationToString(((23*60*60)+(59*60)+(59))*1000+999));
+    assertEquals("-001:01:01:01.000",
+        DFSUtil.durationToString(-((24*60*60)+(60*60)+(60)+1)*1000));
+    assertEquals("-000:23:59:59.574",
+        DFSUtil.durationToString(-(((23*60*60)+(59*60)+(59))*1000+574)));
   }
 
   @Test(timeout=5000)
@@ -763,7 +761,7 @@ public class TestDFSUtil {
     assertEquals(61*60*1000, DFSUtil.parseRelativeTime("61m"));
     assertEquals(0, DFSUtil.parseRelativeTime("0s"));
     assertEquals(25*60*60*1000, DFSUtil.parseRelativeTime("25h"));
-    assertEquals(4*24*60*60*1000, DFSUtil.parseRelativeTime("4d"));
-    assertEquals(999*24*60*60*1000, DFSUtil.parseRelativeTime("999d"));
+    assertEquals(4*24*60*60*1000l, DFSUtil.parseRelativeTime("4d"));
+    assertEquals(999*24*60*60*1000l, DFSUtil.parseRelativeTime("999d"));
   }
 }

+ 185 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java

@@ -23,6 +23,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMOR
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS;
+import static org.apache.hadoop.hdfs.protocol.CachePoolInfo.RELATIVE_EXPIRY_NEVER;
+import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -137,6 +139,8 @@ public class TestCacheDirectives {
     NativeIO.POSIX.setCacheManipulator(new NoMlockCacheManipulator());
     LogManager.getLogger(CacheReplicationMonitor.class.getName()).setLevel(
         Level.TRACE);
+    LogManager.getLogger(CacheManager.class.getName()).setLevel(
+        Level.TRACE);
   }
 
   @After
@@ -1189,4 +1193,185 @@ public class TestCacheDirectives {
         new CacheDirectiveInfo.Builder().setPool(inadequate.getPoolName())
             .setPath(path1).build(), EnumSet.of(CacheFlag.FORCE));
   }
+
+  @Test(timeout=30000)
+  public void testMaxRelativeExpiry() throws Exception {
+    // Test that negative and really big max expirations can't be set during add
+    try {
+      dfs.addCachePool(new CachePoolInfo("failpool").setMaxRelativeExpiryMs(-1l));
+      fail("Added a pool with a negative max expiry.");
+    } catch (InvalidRequestException e) {
+      GenericTestUtils.assertExceptionContains("negative", e);
+    }
+    try {
+      dfs.addCachePool(new CachePoolInfo("failpool")
+          .setMaxRelativeExpiryMs(Long.MAX_VALUE - 1));
+      fail("Added a pool with too big of a max expiry.");
+    } catch (InvalidRequestException e) {
+      GenericTestUtils.assertExceptionContains("too big", e);
+    }
+    // Test that setting a max relative expiry on a pool works
+    CachePoolInfo coolPool = new CachePoolInfo("coolPool");
+    final long poolExpiration = 1000 * 60 * 10l;
+    dfs.addCachePool(coolPool.setMaxRelativeExpiryMs(poolExpiration));
+    RemoteIterator<CachePoolEntry> poolIt = dfs.listCachePools();
+    CachePoolInfo listPool = poolIt.next().getInfo();
+    assertFalse("Should only be one pool", poolIt.hasNext());
+    assertEquals("Expected max relative expiry to match set value",
+        poolExpiration, listPool.getMaxRelativeExpiryMs().longValue());
+    // Test that negative and really big max expirations can't be modified
+    try {
+      dfs.addCachePool(coolPool.setMaxRelativeExpiryMs(-1l));
+      fail("Added a pool with a negative max expiry.");
+    } catch (InvalidRequestException e) {
+      assertExceptionContains("negative", e);
+    }
+    try {
+      dfs.modifyCachePool(coolPool
+          .setMaxRelativeExpiryMs(CachePoolInfo.RELATIVE_EXPIRY_NEVER+1));
+      fail("Added a pool with too big of a max expiry.");
+    } catch (InvalidRequestException e) {
+      assertExceptionContains("too big", e);
+    }
+    // Test that adding a directives without an expiration uses the pool's max
+    CacheDirectiveInfo defaultExpiry = new CacheDirectiveInfo.Builder()
+        .setPath(new Path("/blah"))
+        .setPool(coolPool.getPoolName())
+        .build();
+    dfs.addCacheDirective(defaultExpiry);
+    RemoteIterator<CacheDirectiveEntry> dirIt =
+        dfs.listCacheDirectives(defaultExpiry);
+    CacheDirectiveInfo listInfo = dirIt.next().getInfo();
+    assertFalse("Should only have one entry in listing", dirIt.hasNext());
+    long listExpiration = listInfo.getExpiration().getAbsoluteMillis()
+        - new Date().getTime();
+    assertTrue("Directive expiry should be approximately the pool's max expiry",
+        Math.abs(listExpiration - poolExpiration) < 10*1000);
+    // Test that the max is enforced on add for relative and absolute
+    CacheDirectiveInfo.Builder builder = new CacheDirectiveInfo.Builder()
+        .setPath(new Path("/lolcat"))
+        .setPool(coolPool.getPoolName());
+    try {
+      dfs.addCacheDirective(builder
+          .setExpiration(Expiration.newRelative(poolExpiration+1))
+          .build());
+      fail("Added a directive that exceeds pool's max relative expiration");
+    } catch (InvalidRequestException e) {
+      assertExceptionContains("exceeds the max relative expiration", e);
+    }
+    try {
+      dfs.addCacheDirective(builder
+          .setExpiration(Expiration.newAbsolute(
+              new Date().getTime() + poolExpiration + (10*1000)))
+          .build());
+      fail("Added a directive that exceeds pool's max relative expiration");
+    } catch (InvalidRequestException e) {
+      assertExceptionContains("exceeds the max relative expiration", e);
+    }
+    // Test that max is enforced on modify for relative and absolute Expirations
+    try {
+      dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder(defaultExpiry)
+          .setId(listInfo.getId())
+          .setExpiration(Expiration.newRelative(poolExpiration+1))
+          .build());
+      fail("Modified a directive to exceed pool's max relative expiration");
+    } catch (InvalidRequestException e) {
+      assertExceptionContains("exceeds the max relative expiration", e);
+    }
+    try {
+      dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder(defaultExpiry)
+          .setId(listInfo.getId())
+          .setExpiration(Expiration.newAbsolute(
+              new Date().getTime() + poolExpiration + (10*1000)))
+          .build());
+      fail("Modified a directive to exceed pool's max relative expiration");
+    } catch (InvalidRequestException e) {
+      assertExceptionContains("exceeds the max relative expiration", e);
+    }
+    // Test some giant limit values with add
+    try {
+      dfs.addCacheDirective(builder
+          .setExpiration(Expiration.newRelative(
+              Long.MAX_VALUE))
+          .build());
+      fail("Added a directive with a gigantic max value");
+    } catch (IllegalArgumentException e) {
+      assertExceptionContains("is too far in the future", e);
+    }
+    try {
+      dfs.addCacheDirective(builder
+          .setExpiration(Expiration.newAbsolute(
+              Long.MAX_VALUE))
+          .build());
+      fail("Added a directive with a gigantic max value");
+    } catch (InvalidRequestException e) {
+      assertExceptionContains("is too far in the future", e);
+    }
+    // Test some giant limit values with modify
+    try {
+      dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder(defaultExpiry)
+          .setId(listInfo.getId())
+          .setExpiration(Expiration.NEVER)
+          .build());
+      fail("Modified a directive to exceed pool's max relative expiration");
+    } catch (InvalidRequestException e) {
+      assertExceptionContains("exceeds the max relative expiration", e);
+    }
+    try {
+      dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder(defaultExpiry)
+          .setId(listInfo.getId())
+          .setExpiration(Expiration.newAbsolute(
+              Long.MAX_VALUE))
+          .build());
+      fail("Modified a directive to exceed pool's max relative expiration");
+    } catch (InvalidRequestException e) {
+      assertExceptionContains("is too far in the future", e);
+    }
+    // Test that the max is enforced on modify correctly when changing pools
+    CachePoolInfo destPool = new CachePoolInfo("destPool");
+    dfs.addCachePool(destPool.setMaxRelativeExpiryMs(poolExpiration / 2));
+    try {
+      dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder(defaultExpiry)
+          .setId(listInfo.getId())
+          .setPool(destPool.getPoolName())
+          .build());
+      fail("Modified a directive to a pool with a lower max expiration");
+    } catch (InvalidRequestException e) {
+      assertExceptionContains("exceeds the max relative expiration", e);
+    }
+    dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder(defaultExpiry)
+        .setId(listInfo.getId())
+        .setPool(destPool.getPoolName())
+        .setExpiration(Expiration.newRelative(poolExpiration / 2))
+        .build());
+    dirIt = dfs.listCacheDirectives(new CacheDirectiveInfo.Builder()
+        .setPool(destPool.getPoolName())
+        .build());
+    listInfo = dirIt.next().getInfo();
+    listExpiration = listInfo.getExpiration().getAbsoluteMillis()
+        - new Date().getTime();
+    assertTrue("Unexpected relative expiry " + listExpiration
+        + " expected approximately " + poolExpiration/2,
+        Math.abs(poolExpiration/2 - listExpiration) < 10*1000);
+    // Test that cache pool and directive expiry can be modified back to never
+    dfs.modifyCachePool(destPool
+        .setMaxRelativeExpiryMs(CachePoolInfo.RELATIVE_EXPIRY_NEVER));
+    poolIt = dfs.listCachePools();
+    listPool = poolIt.next().getInfo();
+    while (!listPool.getPoolName().equals(destPool.getPoolName())) {
+      listPool = poolIt.next().getInfo();
+    }
+    assertEquals("Expected max relative expiry to match set value",
+        CachePoolInfo.RELATIVE_EXPIRY_NEVER,
+        listPool.getMaxRelativeExpiryMs().longValue());
+    dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder()
+        .setId(listInfo.getId())
+        .setExpiration(Expiration.newRelative(RELATIVE_EXPIRY_NEVER))
+        .build());
+    // Test modifying close to the limit
+    dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder()
+        .setId(listInfo.getId())
+        .setExpiration(Expiration.newRelative(RELATIVE_EXPIRY_NEVER - 1))
+        .build());
+  }
 }

+ 72 - 71
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml

@@ -13,8 +13,8 @@
       <TXID>2</TXID>
       <DELEGATION_KEY>
         <KEY_ID>1</KEY_ID>
-        <EXPIRY_DATE>1387701670577</EXPIRY_DATE>
-        <KEY>7bb5467995769b59</KEY>
+        <EXPIRY_DATE>1388171826188</EXPIRY_DATE>
+        <KEY>c7d869c22c8afce1</KEY>
       </DELEGATION_KEY>
     </DATA>
   </RECORD>
@@ -24,8 +24,8 @@
       <TXID>3</TXID>
       <DELEGATION_KEY>
         <KEY_ID>2</KEY_ID>
-        <EXPIRY_DATE>1387701670580</EXPIRY_DATE>
-        <KEY>a5a3a2755e36827b</KEY>
+        <EXPIRY_DATE>1388171826191</EXPIRY_DATE>
+        <KEY>a3c41446507dfca9</KEY>
       </DELEGATION_KEY>
     </DATA>
   </RECORD>
@@ -37,17 +37,17 @@
       <INODEID>16386</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1387010471220</MTIME>
-      <ATIME>1387010471220</ATIME>
+      <MTIME>1387480626844</MTIME>
+      <ATIME>1387480626844</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_1147796111_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>7</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -59,8 +59,8 @@
       <INODEID>0</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1387010471276</MTIME>
-      <ATIME>1387010471220</ATIME>
+      <MTIME>1387480626885</MTIME>
+      <ATIME>1387480626844</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -78,8 +78,8 @@
       <LENGTH>0</LENGTH>
       <SRC>/file_create_u\0001;F431</SRC>
       <DST>/file_moved</DST>
-      <TIMESTAMP>1387010471286</TIMESTAMP>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <TIMESTAMP>1387480626894</TIMESTAMP>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>9</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -89,8 +89,8 @@
       <TXID>7</TXID>
       <LENGTH>0</LENGTH>
       <PATH>/file_moved</PATH>
-      <TIMESTAMP>1387010471299</TIMESTAMP>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <TIMESTAMP>1387480626905</TIMESTAMP>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>10</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -101,7 +101,7 @@
       <LENGTH>0</LENGTH>
       <INODEID>16387</INODEID>
       <PATH>/directory_mkdir</PATH>
-      <TIMESTAMP>1387010471312</TIMESTAMP>
+      <TIMESTAMP>1387480626917</TIMESTAMP>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
@@ -136,7 +136,7 @@
       <TXID>12</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot1</SNAPSHOTNAME>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>15</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -147,7 +147,7 @@
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
       <SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>16</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -157,7 +157,7 @@
       <TXID>14</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot2</SNAPSHOTNAME>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>17</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -169,17 +169,17 @@
       <INODEID>16388</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1387010471373</MTIME>
-      <ATIME>1387010471373</ATIME>
+      <MTIME>1387480626978</MTIME>
+      <ATIME>1387480626978</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_1147796111_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>18</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -191,8 +191,8 @@
       <INODEID>0</INODEID>
       <PATH>/file_create_u\0001;F431</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1387010471380</MTIME>
-      <ATIME>1387010471373</ATIME>
+      <MTIME>1387480626985</MTIME>
+      <ATIME>1387480626978</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -253,9 +253,9 @@
       <LENGTH>0</LENGTH>
       <SRC>/file_create_u\0001;F431</SRC>
       <DST>/file_moved</DST>
-      <TIMESTAMP>1387010471428</TIMESTAMP>
+      <TIMESTAMP>1387480627035</TIMESTAMP>
       <OPTIONS>NONE</OPTIONS>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>25</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -267,17 +267,17 @@
       <INODEID>16389</INODEID>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1387010471438</MTIME>
-      <ATIME>1387010471438</ATIME>
+      <MTIME>1387480627043</MTIME>
+      <ATIME>1387480627043</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_1147796111_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>27</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -388,8 +388,8 @@
       <INODEID>0</INODEID>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1387010471540</MTIME>
-      <ATIME>1387010471438</ATIME>
+      <MTIME>1387480627148</MTIME>
+      <ATIME>1387480627043</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -423,17 +423,17 @@
       <INODEID>16390</INODEID>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1387010471547</MTIME>
-      <ATIME>1387010471547</ATIME>
+      <MTIME>1387480627155</MTIME>
+      <ATIME>1387480627155</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_1147796111_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>40</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -544,8 +544,8 @@
       <INODEID>0</INODEID>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1387010471588</MTIME>
-      <ATIME>1387010471547</ATIME>
+      <MTIME>1387480627193</MTIME>
+      <ATIME>1387480627155</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -579,17 +579,17 @@
       <INODEID>16391</INODEID>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1387010471595</MTIME>
-      <ATIME>1387010471595</ATIME>
+      <MTIME>1387480627200</MTIME>
+      <ATIME>1387480627200</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_1147796111_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>52</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -700,8 +700,8 @@
       <INODEID>0</INODEID>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1387010471651</MTIME>
-      <ATIME>1387010471595</ATIME>
+      <MTIME>1387480627238</MTIME>
+      <ATIME>1387480627200</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -733,12 +733,12 @@
       <TXID>56</TXID>
       <LENGTH>0</LENGTH>
       <TRG>/file_concat_target</TRG>
-      <TIMESTAMP>1387010471663</TIMESTAMP>
+      <TIMESTAMP>1387480627246</TIMESTAMP>
       <SOURCES>
         <SOURCE1>/file_concat_0</SOURCE1>
         <SOURCE2>/file_concat_1</SOURCE2>
       </SOURCES>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>63</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -750,14 +750,14 @@
       <INODEID>16392</INODEID>
       <PATH>/file_symlink</PATH>
       <VALUE>/file_concat_target</VALUE>
-      <MTIME>1387010471674</MTIME>
-      <ATIME>1387010471674</ATIME>
+      <MTIME>1387480627255</MTIME>
+      <ATIME>1387480627255</ATIME>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>511</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>64</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -771,11 +771,11 @@
         <OWNER>andrew</OWNER>
         <RENEWER>JobTracker</RENEWER>
         <REALUSER></REALUSER>
-        <ISSUE_DATE>1387010471682</ISSUE_DATE>
-        <MAX_DATE>1387615271682</MAX_DATE>
+        <ISSUE_DATE>1387480627262</ISSUE_DATE>
+        <MAX_DATE>1388085427262</MAX_DATE>
         <MASTER_KEY_ID>2</MASTER_KEY_ID>
       </DELEGATION_TOKEN_IDENTIFIER>
-      <EXPIRY_TIME>1387096871682</EXPIRY_TIME>
+      <EXPIRY_TIME>1387567027262</EXPIRY_TIME>
     </DATA>
   </RECORD>
   <RECORD>
@@ -788,11 +788,11 @@
         <OWNER>andrew</OWNER>
         <RENEWER>JobTracker</RENEWER>
         <REALUSER></REALUSER>
-        <ISSUE_DATE>1387010471682</ISSUE_DATE>
-        <MAX_DATE>1387615271682</MAX_DATE>
+        <ISSUE_DATE>1387480627262</ISSUE_DATE>
+        <MAX_DATE>1388085427262</MAX_DATE>
         <MASTER_KEY_ID>2</MASTER_KEY_ID>
       </DELEGATION_TOKEN_IDENTIFIER>
-      <EXPIRY_TIME>1387096871717</EXPIRY_TIME>
+      <EXPIRY_TIME>1387567027281</EXPIRY_TIME>
     </DATA>
   </RECORD>
   <RECORD>
@@ -805,8 +805,8 @@
         <OWNER>andrew</OWNER>
         <RENEWER>JobTracker</RENEWER>
         <REALUSER></REALUSER>
-        <ISSUE_DATE>1387010471682</ISSUE_DATE>
-        <MAX_DATE>1387615271682</MAX_DATE>
+        <ISSUE_DATE>1387480627262</ISSUE_DATE>
+        <MAX_DATE>1388085427262</MAX_DATE>
         <MASTER_KEY_ID>2</MASTER_KEY_ID>
       </DELEGATION_TOKEN_IDENTIFIER>
     </DATA>
@@ -820,7 +820,8 @@
       <GROUPNAME>andrew</GROUPNAME>
       <MODE>493</MODE>
       <LIMIT>9223372036854775807</LIMIT>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <MAXRELATIVEEXPIRY>2305843009213693951</MAXRELATIVEEXPIRY>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>68</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -833,7 +834,7 @@
       <GROUPNAME>party</GROUPNAME>
       <MODE>448</MODE>
       <LIMIT>1989</LIMIT>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>69</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -845,8 +846,8 @@
       <PATH>/bar</PATH>
       <REPLICATION>1</REPLICATION>
       <POOL>poolparty</POOL>
-      <EXPIRATION>-1</EXPIRATION>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <EXPIRATION>2305844396694321272</EXPIRATION>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>70</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -856,7 +857,7 @@
       <TXID>64</TXID>
       <ID>1</ID>
       <PATH>/bar2</PATH>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>71</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -865,7 +866,7 @@
     <DATA>
       <TXID>65</TXID>
       <ID>1</ID>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>72</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -874,7 +875,7 @@
     <DATA>
       <TXID>66</TXID>
       <POOLNAME>poolparty</POOLNAME>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>73</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -886,17 +887,17 @@
       <INODEID>16393</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1387010471802</MTIME>
-      <ATIME>1387010471802</ATIME>
+      <MTIME>1387480627356</MTIME>
+      <ATIME>1387480627356</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-52011019_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_1147796111_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>508263bb-692e-4439-8738-ff89b8b03923</RPC_CLIENTID>
+      <RPC_CLIENTID>a90261a0-3759-4480-ba80-e10c9ae331e6</RPC_CLIENTID>
       <RPC_CALLID>74</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -953,7 +954,7 @@
     <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <DATA>
       <TXID>73</TXID>
-      <LEASEHOLDER>DFSClient_NONMAPREDUCE_-52011019_1</LEASEHOLDER>
+      <LEASEHOLDER>DFSClient_NONMAPREDUCE_1147796111_1</LEASEHOLDER>
       <PATH>/hard-lease-recovery-test</PATH>
       <NEWHOLDER>HDFS_NameNode</NEWHOLDER>
     </DATA>
@@ -966,8 +967,8 @@
       <INODEID>0</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1387010474126</MTIME>
-      <ATIME>1387010471802</ATIME>
+      <MTIME>1387480629729</MTIME>
+      <ATIME>1387480627356</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>

+ 34 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCacheAdminConf.xml

@@ -417,11 +417,11 @@
         </comparator>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>bar   alice  alicegroup  rwxr-xr-x   unlimited             0             0                0             0             0</expected-output>
+          <expected-output>bar   alice  alicegroup  rwxr-xr-x   unlimited   never             0             0                0             0             0</expected-output>
         </comparator>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>foo   bob    bob         rw-rw-r--   unlimited             0             0                0             0             0</expected-output>
+          <expected-output>foo   bob    bob         rw-rw-r--   unlimited   never             0             0                0             0             0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -457,5 +457,37 @@
         </comparator>
       </comparators>
     </test>
+
+    <test> <!--Tested -->
+      <description>Testing pool max ttl settings</description>
+      <test-commands>
+        <cache-admin-command>-addPool pool1 -owner andrew -group andrew</cache-admin-command>
+        <cache-admin-command>-addPool pool2 -owner andrew -group andrew -maxTtl 999d</cache-admin-command>
+        <cache-admin-command>-modifyPool pool2 -maxTtl never</cache-admin-command>
+        <cache-admin-command>-addPool pool3 -owner andrew -group andrew -maxTtl 4h</cache-admin-command>
+        <cache-admin-command>-listPools</cache-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <cache-admin-command>-removePool pool1</cache-admin-command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Found 3 results</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>pool1  andrew  andrew  rwxr-xr-x   unlimited             never</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>pool2  andrew  andrew  rwxr-xr-x   unlimited             never</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>pool3  andrew  andrew  rwxr-xr-x   unlimited  000:04:00:00.000</expected-output>
+        </comparator>
+      </comparators>
+    </test>
   </tests>
 </configuration>

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

@@ -187,6 +187,12 @@ Release 2.4.0 - UNRELEASED
     MAPREDUCE-5052. Job History UI and web services confusing job start time and
     job submit time (Chen He via jeagles)
 
+    MAPREDUCE-5692. Add explicit diagnostics when a task attempt is killed due
+    to speculative execution (Gera Shegalov via Sandy Ryza)
+
+    MAPREDUCE-5550. Task Status message (reporter.setStatus) not shown in UI
+    with Hadoop 2.0 (Gera Shegalov via Sandy Ryza)
+
   OPTIMIZATIONS
 
     MAPREDUCE-5484. YarnChild unnecessarily loads job conf twice (Sandy Ryza)

+ 18 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

@@ -1552,6 +1552,12 @@ public abstract class TaskAttemptImpl implements
         TaskAttemptEvent event) {
       //set the finish time
       taskAttempt.setFinishTime();
+
+      if (event instanceof TaskAttemptKillEvent) {
+        taskAttempt.addDiagnosticInfo(
+            ((TaskAttemptKillEvent) event).getMessage());
+      }
+
       //send the deallocate event to ContainerAllocator
       taskAttempt.eventHandler.handle(
           new ContainerAllocatorEvent(taskAttempt.attemptId,
@@ -1855,6 +1861,12 @@ public abstract class TaskAttemptImpl implements
         LOG.debug("Not generating HistoryFinish event since start event not " +
             "generated for taskAttempt: " + taskAttempt.getID());
       }
+
+      if (event instanceof TaskAttemptKillEvent) {
+        taskAttempt.addDiagnosticInfo(
+            ((TaskAttemptKillEvent) event).getMessage());
+      }
+
 //      taskAttempt.logAttemptFinishedEvent(TaskAttemptStateInternal.KILLED); Not logging Map/Reduce attempts in case of failure.
       taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
           taskAttempt.attemptId,
@@ -1872,6 +1884,12 @@ public abstract class TaskAttemptImpl implements
       // for it
       taskAttempt.taskAttemptListener.unregister(
           taskAttempt.attemptId, taskAttempt.jvmID);
+
+      if (event instanceof TaskAttemptKillEvent) {
+        taskAttempt.addDiagnosticInfo(
+            ((TaskAttemptKillEvent) event).getMessage());
+      }
+
       taskAttempt.reportedStatus.progress = 1.0f;
       taskAttempt.updateProgressSplits();
       //send the cleanup event to containerLauncher

+ 15 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java

@@ -69,6 +69,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.JobMapTaskRescheduledEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskAttemptCompletedEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptKillEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptRecoverEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
@@ -100,6 +101,7 @@ import com.google.common.annotations.VisibleForTesting;
 public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
 
   private static final Log LOG = LogFactory.getLog(TaskImpl.class);
+  private static final String SPECULATION = "Speculation: ";
 
   protected final JobConf conf;
   protected final Path jobFile;
@@ -374,11 +376,15 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
     TaskReport report = recordFactory.newRecordInstance(TaskReport.class);
     readLock.lock();
     try {
+      TaskAttempt bestAttempt = selectBestAttempt();
       report.setTaskId(taskId);
       report.setStartTime(getLaunchTime());
       report.setFinishTime(getFinishTime());
       report.setTaskState(getState());
-      report.setProgress(getProgress());
+      report.setProgress(bestAttempt == null ? 0f : bestAttempt.getProgress());
+      report.setStatus(bestAttempt == null
+          ? ""
+          : bestAttempt.getReport().getStateString());
 
       for (TaskAttempt attempt : attempts.values()) {
         if (TaskAttemptState.RUNNING.equals(attempt.getState())) {
@@ -398,7 +404,9 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
 
       // Add a copy of counters as the last step so that their lifetime on heap
       // is as small as possible.
-      report.setCounters(TypeConverter.toYarn(getCounters()));
+      report.setCounters(TypeConverter.toYarn(bestAttempt == null
+          ? TaskAttemptImpl.EMPTY_COUNTERS
+          : bestAttempt.getCounters()));
 
       return report;
     } finally {
@@ -906,8 +914,8 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
         LOG.info(task.commitAttempt
             + " already given a go for committing the task output, so killing "
             + attemptID);
-        task.eventHandler.handle(new TaskAttemptEvent(
-            attemptID, TaskAttemptEventType.TA_KILL));
+        task.eventHandler.handle(new TaskAttemptKillEvent(attemptID,
+            SPECULATION + task.commitAttempt + " committed first!"));
       }
     }
   }
@@ -932,9 +940,8 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
             //  other reasons.
             !attempt.isFinished()) {
           LOG.info("Issuing kill to other attempt " + attempt.getID());
-          task.eventHandler.handle(
-              new TaskAttemptEvent(attempt.getID(), 
-                  TaskAttemptEventType.TA_KILL));
+          task.eventHandler.handle(new TaskAttemptKillEvent(attempt.getID(),
+              SPECULATION + task.successfulAttempt + " succeeded first!"));
         }
       }
       task.finished(TaskStateInternal.SUCCEEDED);
@@ -1199,8 +1206,7 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
   private void killUnfinishedAttempt(TaskAttempt attempt, String logMsg) {
     if (attempt != null && !attempt.isFinished()) {
       eventHandler.handle(
-          new TaskAttemptEvent(attempt.getID(),
-              TaskAttemptEventType.TA_KILL));
+          new TaskAttemptKillEvent(attempt.getID(), logMsg));
     }
   }
 

+ 5 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java

@@ -63,6 +63,7 @@ public class TaskPage extends AppView {
             th(".id", "Attempt").
             th(".progress", "Progress").
             th(".state", "State").
+            th(".status", "Status").
             th(".node", "Node").
             th(".logs", "Logs").
             th(".tsh", "Started").
@@ -84,6 +85,7 @@ public class TaskPage extends AppView {
         .append(ta.getId()).append("\",\"")
         .append(progress).append("\",\"")
         .append(ta.getState().toString()).append("\",\"")
+        .append(ta.getStatus()).append("\",\"")
 
         .append(nodeHttpAddr == null ? "N/A" :
           "<a class='nodelink' href='" + MRWebAppUtil.getYARNWebappScheme() + nodeHttpAddr + "'>"
@@ -144,13 +146,13 @@ public class TaskPage extends AppView {
     .append("\n,aoColumnDefs:[\n")
 
     //logs column should not filterable (it includes container ID which may pollute searches)
-    .append("\n{'aTargets': [ 4 ]")
+    .append("\n{'aTargets': [ 5 ]")
     .append(", 'bSearchable': false }")
 
-    .append("\n, {'sType':'numeric', 'aTargets': [ 5, 6")
+    .append("\n, {'sType':'numeric', 'aTargets': [ 6, 7")
     .append(" ], 'mRender': renderHadoopDate }")
 
-    .append("\n, {'sType':'numeric', 'aTargets': [ 7")
+    .append("\n, {'sType':'numeric', 'aTargets': [ 8")
     .append(" ], 'mRender': renderHadoopElapsedTime }]")
 
     // Sort by id upon page load

+ 2 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java

@@ -59,6 +59,7 @@ public class TasksBlock extends HtmlBlock {
           tr().
             th("Task").
             th("Progress").
+            th("Status").
             th("State").
             th("Start Time").
             th("Finish Time").
@@ -81,6 +82,7 @@ public class TasksBlock extends HtmlBlock {
       .append(join(pct, '%')).append("'> ").append("<div class='")
       .append(C_PROGRESSBAR_VALUE).append("' style='")
       .append(join("width:", pct, '%')).append("'> </div> </div>\",\"")
+      .append(info.getStatus()).append("\",\"")
 
       .append(info.getState()).append("\",\"")
       .append(info.getStartTime()).append("\",\"")

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java

@@ -50,10 +50,10 @@ public class TasksPage extends AppView {
       .append(", 'mRender': parseHadoopProgress }")
 
 
-      .append("\n, {'sType':'numeric', 'aTargets': [3, 4]")
+      .append("\n, {'sType':'numeric', 'aTargets': [4, 5]")
       .append(", 'mRender': renderHadoopDate }")
 
-      .append("\n, {'sType':'numeric', 'aTargets': [5]")
+      .append("\n, {'sType':'numeric', 'aTargets': [6]")
       .append(", 'mRender': renderHadoopElapsedTime }]")
 
       // Sort by id upon page load

+ 15 - 15
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptInfo.java

@@ -25,6 +25,7 @@ import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlSeeAlso;
 import javax.xml.bind.annotation.XmlTransient;
 
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptReport;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
@@ -45,6 +46,7 @@ public class TaskAttemptInfo {
   protected String id;
   protected String rack;
   protected TaskAttemptState state;
+  protected String status;
   protected String nodeHttpAddress;
   protected String diagnostics;
   protected String type;
@@ -61,29 +63,23 @@ public class TaskAttemptInfo {
   }
 
   public TaskAttemptInfo(TaskAttempt ta, TaskType type, Boolean isRunning) {
+    final TaskAttemptReport report = ta.getReport();
     this.type = type.toString();
     this.id = MRApps.toString(ta.getID());
     this.nodeHttpAddress = ta.getNodeHttpAddress();
-    this.startTime = ta.getLaunchTime();
-    this.finishTime = ta.getFinishTime();
-    this.assignedContainerId = ConverterUtils.toString(ta
-        .getAssignedContainerID());
-    this.assignedContainer = ta.getAssignedContainerID();
-    this.progress = ta.getProgress() * 100;
-    this.state = ta.getState();
+    this.startTime = report.getStartTime();
+    this.finishTime = report.getFinishTime();
+    this.assignedContainerId = ConverterUtils.toString(report.getContainerId());
+    this.assignedContainer = report.getContainerId();
+    this.progress = report.getProgress() * 100;
+    this.status = report.getStateString();
+    this.state = report.getTaskAttemptState();
     this.elapsedTime = Times
         .elapsed(this.startTime, this.finishTime, isRunning);
     if (this.elapsedTime == -1) {
       this.elapsedTime = 0;
     }
-    List<String> diagnostics = ta.getDiagnostics();
-    if (diagnostics != null && !diagnostics.isEmpty()) {
-      StringBuffer b = new StringBuffer();
-      for (String diag : diagnostics) {
-        b.append(diag);
-      }
-      this.diagnostics = b.toString();
-    }
+    this.diagnostics = report.getDiagnosticInfo();
     this.rack = ta.getNodeRackName();
   }
 
@@ -99,6 +95,10 @@ public class TaskAttemptInfo {
     return this.state.toString();
   }
 
+  public String getStatus() {
+    return status;
+  }
+
   public String getId() {
     return this.id;
   }

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

@@ -43,6 +43,7 @@ public class TaskInfo {
   protected TaskState state;
   protected String type;
   protected String successfulAttempt;
+  protected String status;
 
   @XmlTransient
   int taskNum;
@@ -66,6 +67,7 @@ public class TaskInfo {
       this.elapsedTime = 0;
     }
     this.progress = report.getProgress() * 100;
+    this.status =  report.getStatus();
     this.id = MRApps.toString(task.getID());
     this.taskNum = task.getID().getId();
     this.successful = getSuccessfulAttempt(task);
@@ -121,4 +123,7 @@ public class TaskInfo {
     return null;
   }
 
+  public String getStatus() {
+    return status;
+  }
 }

+ 20 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java

@@ -174,22 +174,37 @@ public class MockJobs extends MockApps {
     report.setFinishTime(System.currentTimeMillis()
         + (int) (Math.random() * DT) + 1);
     report.setProgress((float) Math.random());
+    report.setStatus("Moving average: " + Math.random());
     report.setCounters(TypeConverter.toYarn(newCounters()));
     report.setTaskState(TASK_STATES.next());
     return report;
   }
 
   public static TaskAttemptReport newTaskAttemptReport(TaskAttemptId id) {
+    ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(
+        id.getTaskId().getJobId().getAppId(), 0);
+    ContainerId containerId = ContainerId.newInstance(appAttemptId, 0);
     TaskAttemptReport report = Records.newRecord(TaskAttemptReport.class);
     report.setTaskAttemptId(id);
     report
         .setStartTime(System.currentTimeMillis() - (int) (Math.random() * DT));
     report.setFinishTime(System.currentTimeMillis()
         + (int) (Math.random() * DT) + 1);
+
+    if (id.getTaskId().getTaskType() == TaskType.REDUCE) {
+      report.setShuffleFinishTime(
+          (report.getFinishTime() + report.getStartTime()) / 2);
+      report.setSortFinishTime(
+          (report.getFinishTime() + report.getShuffleFinishTime()) / 2);
+    }
+
     report.setPhase(PHASES.next());
     report.setTaskAttemptState(TASK_ATTEMPT_STATES.next());
     report.setProgress((float) Math.random());
     report.setCounters(TypeConverter.toYarn(newCounters()));
+    report.setContainerId(containerId);
+    report.setDiagnosticInfo(DIAGS.next());
+    report.setStateString("Moving average " + Math.random());
     return report;
   }
 
@@ -230,8 +245,6 @@ public class MockJobs extends MockApps {
     taid.setTaskId(tid);
     taid.setId(i);
     final TaskAttemptReport report = newTaskAttemptReport(taid);
-    final List<String> diags = Lists.newArrayList();
-    diags.add(DIAGS.next());
     return new TaskAttempt() {
       @Override
       public NodeId getNodeId() throws UnsupportedOperationException{
@@ -250,12 +263,12 @@ public class MockJobs extends MockApps {
 
       @Override
       public long getLaunchTime() {
-        return 0;
+        return report.getStartTime();
       }
 
       @Override
       public long getFinishTime() {
-        return 0;
+        return report.getFinishTime();
       }
 
       @Override
@@ -313,7 +326,7 @@ public class MockJobs extends MockApps {
 
       @Override
       public List<String> getDiagnostics() {
-        return diags;
+        return Lists.newArrayList(report.getDiagnosticInfo());
       }
 
       @Override
@@ -323,12 +336,12 @@ public class MockJobs extends MockApps {
 
       @Override
       public long getShuffleFinishTime() {
-        return 0;
+        return report.getShuffleFinishTime();
       }
 
       @Override
       public long getSortFinishTime() {
-        return 0;
+        return report.getSortFinishTime();
       }
 
       @Override

+ 6 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java

@@ -1,3 +1,4 @@
+/**
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -425,9 +426,9 @@ public class TestAMWebServicesAttempts extends JerseyTest {
   public void verifyAMTaskAttempt(JSONObject info, TaskAttempt att,
       TaskType ttype) throws JSONException {
     if (ttype == TaskType.REDUCE) {
-      assertEquals("incorrect number of elements", 16, info.length());
+      assertEquals("incorrect number of elements", 17, info.length());
     } else {
-      assertEquals("incorrect number of elements", 11, info.length());
+      assertEquals("incorrect number of elements", 12, info.length());
     }
 
     verifyTaskAttemptGeneric(att, ttype, info.getString("id"),
@@ -532,11 +533,11 @@ public class TestAMWebServicesAttempts extends JerseyTest {
     assertEquals("mergeFinishTime wrong", ta.getSortFinishTime(),
         mergeFinishTime);
     assertEquals("elapsedShuffleTime wrong",
-        ta.getLaunchTime() - ta.getShuffleFinishTime(), elapsedShuffleTime);
+        ta.getShuffleFinishTime() - ta.getLaunchTime(), elapsedShuffleTime);
     assertEquals("elapsedMergeTime wrong",
-        ta.getShuffleFinishTime() - ta.getSortFinishTime(), elapsedMergeTime);
+        ta.getSortFinishTime() - ta.getShuffleFinishTime(), elapsedMergeTime);
     assertEquals("elapsedReduceTime wrong",
-        ta.getSortFinishTime() - ta.getFinishTime(), elapsedReduceTime);
+        ta.getFinishTime() - ta.getSortFinishTime(), elapsedReduceTime);
   }
 
   @Test

+ 7 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java

@@ -525,12 +525,13 @@ public class TestAMWebServicesTasks extends JerseyTest {
 
   public void verifyAMSingleTask(JSONObject info, Task task)
       throws JSONException {
-    assertEquals("incorrect number of elements", 8, info.length());
+    assertEquals("incorrect number of elements", 9, info.length());
 
     verifyTaskGeneric(task, info.getString("id"), info.getString("state"),
         info.getString("type"), info.getString("successfulAttempt"),
         info.getLong("startTime"), info.getLong("finishTime"),
-        info.getLong("elapsedTime"), (float) info.getDouble("progress"));
+        info.getLong("elapsedTime"), (float) info.getDouble("progress"),
+        info.getString("status"));
   }
 
   public void verifyAMTask(JSONArray arr, Job job, String type)
@@ -555,7 +556,7 @@ public class TestAMWebServicesTasks extends JerseyTest {
 
   public void verifyTaskGeneric(Task task, String id, String state,
       String type, String successfulAttempt, long startTime, long finishTime,
-      long elapsedTime, float progress) {
+      long elapsedTime, float progress, String status) {
 
     TaskId taskid = task.getID();
     String tid = MRApps.toString(taskid);
@@ -572,6 +573,7 @@ public class TestAMWebServicesTasks extends JerseyTest {
     assertEquals("finishTime wrong", report.getFinishTime(), finishTime);
     assertEquals("elapsedTime wrong", finishTime - startTime, elapsedTime);
     assertEquals("progress wrong", report.getProgress() * 100, progress, 1e-3f);
+    assertEquals("status wrong", report.getStatus(), status);
   }
 
   public void verifyAMSingleTaskXML(Element element, Task task) {
@@ -582,7 +584,8 @@ public class TestAMWebServicesTasks extends JerseyTest {
         WebServicesTestUtils.getXmlLong(element, "startTime"),
         WebServicesTestUtils.getXmlLong(element, "finishTime"),
         WebServicesTestUtils.getXmlLong(element, "elapsedTime"),
-        WebServicesTestUtils.getXmlFloat(element, "progress"));
+        WebServicesTestUtils.getXmlFloat(element, "progress"),
+        WebServicesTestUtils.getXmlString(element, "status"));
   }
 
   public void verifyAMTaskXML(NodeList nodes, Job job) {

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/TaskReport.java

@@ -24,10 +24,10 @@ public interface TaskReport {
   public abstract TaskId getTaskId();
   public abstract TaskState getTaskState();
   public abstract float getProgress();
+  public abstract String getStatus();
   public abstract long getStartTime();
   public abstract long getFinishTime();
   public abstract Counters getCounters();
-  
   public abstract List<TaskAttemptId> getRunningAttemptsList();
   public abstract TaskAttemptId getRunningAttempt(int index);
   public abstract int getRunningAttemptsCount();
@@ -42,6 +42,7 @@ public interface TaskReport {
   public abstract void setTaskId(TaskId taskId);
   public abstract void setTaskState(TaskState taskState);
   public abstract void setProgress(float progress);
+  public abstract void setStatus(String status);
   public abstract void setStartTime(long startTime);
   public abstract void setFinishTime(long finishTime);
   public abstract void setCounters(Counters counters);

+ 12 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/TaskReportPBImpl.java

@@ -49,6 +49,7 @@ public class TaskReportPBImpl extends ProtoBase<TaskReportProto> implements Task
   private List<TaskAttemptId> runningAttempts = null;
   private TaskAttemptId successfulAttemptId = null;
   private List<String> diagnostics = null;
+  private String status;
   
   
   public TaskReportPBImpl() {
@@ -171,11 +172,22 @@ public class TaskReportPBImpl extends ProtoBase<TaskReportProto> implements Task
     return (p.getProgress());
   }
 
+  @Override
+  public String getStatus() {
+    return status;
+  }
+
   @Override
   public void setProgress(float progress) {
     maybeInitBuilder();
     builder.setProgress((progress));
   }
+
+  @Override
+  public void setStatus(String status) {
+    this.status = status;
+  }
+
   @Override
   public TaskState getTaskState() {
     TaskReportProtoOrBuilder p = viaProto ? proto : builder;

+ 25 - 17
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.app.webapp.App;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
 import org.apache.hadoop.yarn.util.Times;
@@ -89,6 +90,7 @@ public class HsTaskPage extends HsView {
       headRow.
             th(".id", "Attempt").
             th(".state", "State").
+            th(".status", "Status").
             th(".node", "Node").
             th(".logs", "Logs").
             th(".tsh", "Start Time");
@@ -113,15 +115,16 @@ public class HsTaskPage extends HsView {
        // DataTables to display
        StringBuilder attemptsTableData = new StringBuilder("[\n");
 
-       for (TaskAttempt ta : getTaskAttempts()) {
-        String taid = MRApps.toString(ta.getID());
+       for (TaskAttempt attempt : getTaskAttempts()) {
+        final TaskAttemptInfo ta = new TaskAttemptInfo(attempt, false);
+        String taid = ta.getId();
 
-        String nodeHttpAddr = ta.getNodeHttpAddress();
-        String containerIdString = ta.getAssignedContainerID().toString();
-        String nodeIdString = ta.getAssignedContainerMgrAddress();
-        String nodeRackName = ta.getNodeRackName();
+        String nodeHttpAddr = ta.getNode();
+        String containerIdString = ta.getAssignedContainerIdStr();
+        String nodeIdString = attempt.getAssignedContainerMgrAddress();
+        String nodeRackName = ta.getRack();
 
-        long attemptStartTime = ta.getLaunchTime();
+        long attemptStartTime = ta.getStartTime();
         long shuffleFinishTime = -1;
         long sortFinishTime = -1;
         long attemptFinishTime = ta.getFinishTime();
@@ -129,8 +132,8 @@ public class HsTaskPage extends HsView {
         long elapsedSortTime = -1;
         long elapsedReduceTime = -1;
         if(type == TaskType.REDUCE) {
-          shuffleFinishTime = ta.getShuffleFinishTime();
-          sortFinishTime = ta.getSortFinishTime();
+          shuffleFinishTime = attempt.getShuffleFinishTime();
+          sortFinishTime = attempt.getSortFinishTime();
           elapsedShuffleTime =
               Times.elapsed(attemptStartTime, shuffleFinishTime, false);
           elapsedSortTime =
@@ -140,11 +143,13 @@ public class HsTaskPage extends HsView {
         }
         long attemptElapsed =
             Times.elapsed(attemptStartTime, attemptFinishTime, false);
-        int sortId = ta.getID().getId() + (ta.getID().getTaskId().getId() * 10000);
+        int sortId = attempt.getID().getId()
+                   + (attempt.getID().getTaskId().getId() * 10000);
 
         attemptsTableData.append("[\"")
         .append(sortId + " ").append(taid).append("\",\"")
-        .append(ta.getState().toString()).append("\",\"")
+        .append(ta.getState()).append("\",\"")
+        .append(ta.getStatus()).append("\",\"")
 
         .append("<a class='nodelink' href='" + MRWebAppUtil.getYARNWebappScheme() + nodeHttpAddr + "'>")
         .append(nodeRackName + "/" + nodeHttpAddr + "</a>\",\"")
@@ -167,8 +172,9 @@ public class HsTaskPage extends HsView {
           .append(elapsedReduceTime).append("\",\"");
         }
           attemptsTableData.append(attemptElapsed).append("\",\"")
-          .append(StringEscapeUtils.escapeJavaScript(StringEscapeUtils.escapeHtml(
-           Joiner.on('\n').join(ta.getDiagnostics())))).append("\"],\n");
+          .append(StringEscapeUtils.escapeJavaScript(
+              StringEscapeUtils.escapeHtml(ta.getNote())))
+          .append("\"],\n");
       }
        //Remove the last comma and close off the array of arrays
        if(attemptsTableData.charAt(attemptsTableData.length() - 2) == ',') {
@@ -184,6 +190,8 @@ public class HsTaskPage extends HsView {
               $name("attempt_name").$value("Attempt")._()._().
           th().input("search_init").$type(InputType.text).
               $name("attempt_state").$value("State")._()._().
+          th().input("search_init").$type(InputType.text).
+              $name("attempt_status").$value("Status")._()._().
           th().input("search_init").$type(InputType.text).
               $name("attempt_node").$value("Node")._()._().
           th().input("search_init").$type(InputType.text).
@@ -283,19 +291,19 @@ public class HsTaskPage extends HsView {
       .append("\n,aoColumnDefs:[\n")
 
       //logs column should not filterable (it includes container ID which may pollute searches)
-      .append("\n{'aTargets': [ 3 ]")
+      .append("\n{'aTargets': [ 4 ]")
       .append(", 'bSearchable': false }")
 
       .append("\n, {'sType':'numeric', 'aTargets': [ 0 ]")
       .append(", 'mRender': parseHadoopAttemptID }")
 
-      .append("\n, {'sType':'numeric', 'aTargets': [ 4, 5")
+      .append("\n, {'sType':'numeric', 'aTargets': [ 5, 6")
       //Column numbers are different for maps and reduces
-      .append(type == TaskType.REDUCE ? ", 6, 7" : "")
+      .append(type == TaskType.REDUCE ? ", 7, 8" : "")
       .append(" ], 'mRender': renderHadoopDate }")
 
       .append("\n, {'sType':'numeric', 'aTargets': [")
-      .append(type == TaskType.REDUCE ? "8, 9, 10, 11" : "6")
+      .append(type == TaskType.REDUCE ? "9, 10, 11, 12" : "7")
       .append(" ], 'mRender': renderHadoopElapsedTime }]")
 
       // Sort by id upon page load

+ 26 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestBlocks.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptReport;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
@@ -138,11 +139,31 @@ public class TestBlocks {
     when(attempt.getAssignedContainerMgrAddress()).thenReturn(
             "assignedContainerMgrAddress");
     when(attempt.getNodeRackName()).thenReturn("nodeRackName");
-    when(attempt.getLaunchTime()).thenReturn(100002L);
-    when(attempt.getFinishTime()).thenReturn(100012L);
-    when(attempt.getShuffleFinishTime()).thenReturn(100010L);
-    when(attempt.getSortFinishTime()).thenReturn(100011L);
-    when(attempt.getState()).thenReturn(TaskAttemptState.SUCCEEDED);
+
+    final long taStartTime = 100002L;
+    final long taFinishTime = 100012L;
+    final long taShuffleFinishTime = 100010L;
+    final long taSortFinishTime = 100011L;
+    final TaskAttemptState taState = TaskAttemptState.SUCCEEDED;
+
+    when(attempt.getLaunchTime()).thenReturn(taStartTime);
+    when(attempt.getFinishTime()).thenReturn(taFinishTime);
+    when(attempt.getShuffleFinishTime()).thenReturn(taShuffleFinishTime);
+    when(attempt.getSortFinishTime()).thenReturn(taSortFinishTime);
+    when(attempt.getState()).thenReturn(taState);
+
+    TaskAttemptReport taReport = mock(TaskAttemptReport.class);
+    when(taReport.getStartTime()).thenReturn(taStartTime);
+    when(taReport.getFinishTime()).thenReturn(taFinishTime);
+    when(taReport.getShuffleFinishTime()).thenReturn(taShuffleFinishTime);
+    when(taReport.getSortFinishTime()).thenReturn(taSortFinishTime);
+    when(taReport.getContainerId()).thenReturn(containerId);
+    when(taReport.getProgress()).thenReturn(1.0f);
+    when(taReport.getStateString()).thenReturn("Processed 128/128 records");
+    when(taReport.getTaskAttemptState()).thenReturn(taState);
+    when(taReport.getDiagnosticInfo()).thenReturn("");
+
+    when(attempt.getReport()).thenReturn(taReport);
 
     attempts.put(taId, attempt);
     when(task.getAttempts()).thenReturn(attempts);

+ 5 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesAttempts.java

@@ -444,9 +444,9 @@ public class TestHsWebServicesAttempts extends JerseyTest {
   public void verifyHsTaskAttempt(JSONObject info, TaskAttempt att,
       TaskType ttype) throws JSONException {
     if (ttype == TaskType.REDUCE) {
-      assertEquals("incorrect number of elements", 16, info.length());
+      assertEquals("incorrect number of elements", 17, info.length());
     } else {
-      assertEquals("incorrect number of elements", 11, info.length());
+      assertEquals("incorrect number of elements", 12, info.length());
     }
 
     verifyTaskAttemptGeneric(att, ttype, info.getString("id"),
@@ -551,11 +551,11 @@ public class TestHsWebServicesAttempts extends JerseyTest {
     assertEquals("mergeFinishTime wrong", ta.getSortFinishTime(),
         mergeFinishTime);
     assertEquals("elapsedShuffleTime wrong",
-        ta.getLaunchTime() - ta.getShuffleFinishTime(), elapsedShuffleTime);
+        ta.getShuffleFinishTime() - ta.getLaunchTime(), elapsedShuffleTime);
     assertEquals("elapsedMergeTime wrong",
-        ta.getShuffleFinishTime() - ta.getSortFinishTime(), elapsedMergeTime);
+        ta.getSortFinishTime() - ta.getShuffleFinishTime(), elapsedMergeTime);
     assertEquals("elapsedReduceTime wrong",
-        ta.getSortFinishTime() - ta.getFinishTime(), elapsedReduceTime);
+        ta.getFinishTime() - ta.getSortFinishTime(), elapsedReduceTime);
   }
 
   @Test

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesTasks.java

@@ -538,7 +538,7 @@ public class TestHsWebServicesTasks extends JerseyTest {
 
   public void verifyHsSingleTask(JSONObject info, Task task)
       throws JSONException {
-    assertEquals("incorrect number of elements", 8, info.length());
+    assertEquals("incorrect number of elements", 9, info.length());
 
     verifyTaskGeneric(task, info.getString("id"), info.getString("state"),
         info.getString("type"), info.getString("successfulAttempt"),

+ 36 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecutionWithMRApp.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.mapreduce.v2;
 
+import java.util.Collection;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Random;
@@ -106,17 +107,21 @@ public class TestSpeculativeExecutionWithMRApp {
 
     int maxTimeWait = 10;
     boolean successfullySpeculated = false;
+    TaskAttempt[] ta = null;
     while (maxTimeWait > 0 && !successfullySpeculated) {
       if (taskToBeSpeculated.getAttempts().size() != 2) {
         Thread.sleep(1000);
         clock.setTime(System.currentTimeMillis() + 20000);
       } else {
         successfullySpeculated = true;
+        // finish 1st TA, 2nd will be killed
+        ta = makeFirstAttemptWin(appEventHandler, taskToBeSpeculated);
       }
       maxTimeWait--;
     }
     Assert
       .assertTrue("Couldn't speculate successfully", successfullySpeculated);
+    verifySpeculationMessage(app, ta);
   }
 
   @Test(timeout = 60000)
@@ -197,16 +202,47 @@ public class TestSpeculativeExecutionWithMRApp {
 
     int maxTimeWait = 5;
     boolean successfullySpeculated = false;
+    TaskAttempt[] ta = null;
     while (maxTimeWait > 0 && !successfullySpeculated) {
       if (speculatedTask.getAttempts().size() != 2) {
         Thread.sleep(1000);
       } else {
         successfullySpeculated = true;
+        ta = makeFirstAttemptWin(appEventHandler, speculatedTask);
       }
       maxTimeWait--;
     }
     Assert
       .assertTrue("Couldn't speculate successfully", successfullySpeculated);
+    verifySpeculationMessage(app, ta);
+  }
+
+  private static TaskAttempt[] makeFirstAttemptWin(
+      EventHandler appEventHandler, Task speculatedTask) {
+
+    // finish 1st TA, 2nd will be killed
+    Collection<TaskAttempt> attempts = speculatedTask.getAttempts().values();
+    TaskAttempt[] ta = new TaskAttempt[attempts.size()];
+    attempts.toArray(ta);
+    appEventHandler.handle(
+        new TaskAttemptEvent(ta[0].getID(), TaskAttemptEventType.TA_DONE));
+    appEventHandler.handle(new TaskAttemptEvent(ta[0].getID(),
+        TaskAttemptEventType.TA_CONTAINER_CLEANED));
+    return ta;
+  }
+
+  private static void verifySpeculationMessage(MRApp app, TaskAttempt[] ta)
+      throws Exception {
+    app.waitForState(ta[0], TaskAttemptState.SUCCEEDED);
+    app.waitForState(ta[1], TaskAttemptState.KILLED);
+    boolean foundSpecMsg = false;
+    for (String msg : ta[1].getDiagnostics()) {
+      if (msg.contains("Speculation")) {
+        foundSpecMsg = true;
+        break;
+      }
+    }
+    Assert.assertTrue("No speculation diagnostics!", foundSpecMsg);
   }
 
   private TaskAttemptStatus createTaskAttemptStatus(TaskAttemptId id,

+ 16 - 1
hadoop-yarn-project/CHANGES.txt

@@ -1756,7 +1756,22 @@ Release 2.0.2-alpha - 2012-09-07
     YARN-138. Ensure default values for minimum/maximum container sizes is
     sane. (harsh & sseth via acmurthy)
 
-Release 0.23.10 - UNRELEASED
+Release 0.23.11 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+    YARN-1180. Update capacity scheduler docs to include types on the configs
+    (Chen He via jeagles)
+
+Release 0.23.10 - 2013-12-09
 
   INCOMPATIBLE CHANGES
 

+ 0 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java

@@ -86,9 +86,7 @@ public class TestRMFailover {
     setRpcAddressForRM(RM1_NODE_ID, RM1_PORT_BASE);
     setRpcAddressForRM(RM2_NODE_ID, RM2_PORT_BASE);
 
-    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_MAX_ATTEMPTS, 100);
     conf.setLong(YarnConfiguration.CLIENT_FAILOVER_SLEEPTIME_BASE_MS, 100L);
-    conf.setLong(YarnConfiguration.CLIENT_FAILOVER_SLEEPTIME_MAX_MS, 1000L);
     conf.setBoolean(YarnConfiguration.YARN_MINICLUSTER_FIXED_PORTS, true);
     conf.setBoolean(YarnConfiguration.YARN_MINICLUSTER_USE_RPC, true);
 

+ 5 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java

@@ -225,19 +225,17 @@ public class RMProxy<T> {
       int maxFailoverAttempts = conf.getInt(
           YarnConfiguration.CLIENT_FAILOVER_MAX_ATTEMPTS, -1);
 
-      RetryPolicy basePolicy = RetryPolicies.TRY_ONCE_THEN_FAIL;
       if (maxFailoverAttempts == -1) {
         if (waitForEver) {
-          basePolicy = RetryPolicies.FAILOVER_FOREVER;
+          maxFailoverAttempts = Integer.MAX_VALUE;
         } else {
-          basePolicy = new FailoverUptoMaximumTimePolicy(
-              System.currentTimeMillis() + rmConnectWaitMS);
+          maxFailoverAttempts = (int) (rmConnectWaitMS / failoverSleepBaseMs);
         }
-        maxFailoverAttempts = 0;
       }
 
-      return RetryPolicies.failoverOnNetworkException(basePolicy,
-          maxFailoverAttempts, failoverSleepBaseMs, failoverSleepMaxMs);
+      return RetryPolicies.failoverOnNetworkException(
+          RetryPolicies.TRY_ONCE_THEN_FAIL, maxFailoverAttempts,
+          failoverSleepBaseMs, failoverSleepMaxMs);
     }
 
     if (waitForEver) {

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm

@@ -220,7 +220,7 @@ Hadoop MapReduce Next Generation - Capacity Scheduler
 | | application, no single user can use more than 33% of the queue resources. |
 | | With 4 or more users, no user can use more than 25% of the queues |
 | | resources. A value of 100 implies no user limits are imposed. The default |
-| | is 100.|
+| | is 100. Value is specified as a integer.|
 *--------------------------------------+--------------------------------------+
 | <<<yarn.scheduler.capacity.<queue-path>.user-limit-factor>>> |   | 
 | | The multiple of the queue capacity which can be configured to allow a | 
@@ -249,6 +249,7 @@ Hadoop MapReduce Next Generation - Capacity Scheduler
 | | be rejected. Default is 10000. This can be set for all queues with |
 | | <<<yarn.scheduler.capacity.maximum-applications>>> and can also be overridden on a  |
 | | per queue basis by setting <<<yarn.scheduler.capacity.<queue-path>.maximum-applications>>>. |
+| | Integer value expected.|
 *--------------------------------------+--------------------------------------+
 | <<<yarn.scheduler.capacity.maximum-am-resource-percent>>> / |
 | <<<yarn.scheduler.capacity.<queue-path>.maximum-am-resource-percent>>> | |
@@ -276,7 +277,7 @@ Hadoop MapReduce Next Generation - Capacity Scheduler
 | | Thus, if the <root> queue is <<<STOPPED>>> no applications can be | 
 | | submitted to the entire cluster. |
 | | Existing applications continue to completion, thus the queue can be 
-| | <drained> gracefully. | 
+| | <drained> gracefully. Value is specified as Enumeration. |
 *--------------------------------------+--------------------------------------+
 | <<<yarn.scheduler.capacity.root.<queue-path>.acl_submit_applications>>> | |
 | | The <ACL> which controls who can <submit> applications to the given queue. |