Browse Source

Merge r1464808 through r1466652 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1466658 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 năm trước cách đây
mục cha
commit
3209d4a3c4
100 tập tin đã thay đổi với 2947 bổ sung572 xóa
  1. 7 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsFilter.java
  3. 18 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
  4. 14 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
  5. 63 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/filter/TestPatternFilter.java
  6. 32 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
  7. 28 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MockitoUtil.java
  8. 26 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  9. 16 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  10. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  11. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  12. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  13. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
  14. 15 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  15. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
  16. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  17. 14 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  18. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  19. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
  20. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java
  21. 259 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/AvailableSpaceVolumeChoosingPolicy.java
  22. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  23. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  24. 30 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  25. 54 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  26. 22 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  27. 22 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  28. 37 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  29. 6 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  30. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  31. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java
  32. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/GSet.java
  33. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/GSetByHashMap.java
  34. 12 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightGSet.java
  35. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  36. 28 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  37. 23 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
  38. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSFileSystemContract.java
  39. 35 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetTimes.java
  40. 26 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
  41. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
  42. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolManager.java
  43. 303 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/TestAvailableSpaceVolumeChoosingPolicy.java
  44. 15 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/TestRoundRobinVolumeChoosingPolicy.java
  45. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
  46. 1 14
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  47. 29 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  48. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestGSet.java
  49. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
  50. 15 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
  51. 9 0
      hadoop-mapreduce-project/CHANGES.txt
  52. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  53. 12 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
  54. 8 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalDistributedCacheManager.java
  55. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/avro/Events.avpr
  56. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  57. 35 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
  58. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
  59. 42 13
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestCompletedTask.java
  60. 44 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryEntities.java
  61. 335 201
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java
  62. 209 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryServer.java
  63. 6 1
      hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/Job20LineHistoryEventEmitter.java
  64. 24 3
      hadoop-yarn-project/CHANGES.txt
  65. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java
  66. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  67. 5 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/test/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/TestUnmanagedAMLauncher.java
  68. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  69. 3 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
  70. 56 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java
  71. 43 23
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
  72. 37 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/ResourceLocalizationSpec.java
  73. 118 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/impl/pb/ResourceLocalizationSpecPBImpl.java
  74. 5 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/LocalizerHeartbeatResponse.java
  75. 40 56
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/LocalizerHeartbeatResponsePBImpl.java
  76. 11 26
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java
  77. 0 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalCacheDirectoryManager.java
  78. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTracker.java
  79. 12 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
  80. 97 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
  81. 39 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerBuilderUtils.java
  82. 6 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_service_protos.proto
  83. 98 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
  84. 28 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java
  85. 14 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/MockLocalizerHeartbeatResponse.java
  86. 58 26
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestContainerLocalizer.java
  87. 73 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
  88. 11 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
  89. 19 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  90. 30 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
  91. 43 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  92. 6 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DefaultResourceCalculator.java
  93. 10 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DominantResourceCalculator.java
  94. 5 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceCalculator.java
  95. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java
  96. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  97. 42 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/InvalidResourceRequestException.java
  98. 45 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
  99. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
  100. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java

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

@@ -505,6 +505,10 @@ Trunk (Unreleased)
 
     HADOOP-9353. Activate native-win maven profile by default on Windows.
     (Arpit Agarwal via szetszwo)
+
+    HADOOP-9437. TestNativeIO#testRenameTo fails on Windows due to assumption
+    that POSIX errno is embedded in NativeIOException. (Chris Nauroth via
+    suresh)
     
 Release 2.0.5-beta - UNRELEASED
 
@@ -625,6 +629,9 @@ Release 2.0.4-alpha - UNRELEASED
 
   BUG FIXES
 
+    HADOOP-9467. Metrics2 record filter should check name as well as tags.
+    (Ganeshan Iyler via llu)
+
     HADOOP-9406. hadoop-client leaks dependency on JDK tools jar. (tucu)
 
     HADOOP-9301. hadoop client servlet/jsp/jetty/tomcat JARs creating

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsFilter.java

@@ -57,7 +57,7 @@ public abstract class MetricsFilter implements MetricsPlugin {
    * @return  true to accept; false otherwise.
    */
   public boolean accepts(MetricsRecord record) {
-    return accepts(record.tags());
+    return accepts(record.name()) && accepts(record.tags());
   }
 
 }

+ 18 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c

@@ -816,6 +816,7 @@ JNIEXPORT void JNICALL
 Java_org_apache_hadoop_io_nativeio_NativeIO_renameTo0(JNIEnv *env, 
 jclass clazz, jstring jsrc, jstring jdst)
 {
+#ifdef UNIX
   const char *src = NULL, *dst = NULL;
   
   src = (*env)->GetStringUTFChars(env, jsrc, NULL);
@@ -829,6 +830,23 @@ jclass clazz, jstring jsrc, jstring jdst)
 done:
   if (src) (*env)->ReleaseStringUTFChars(env, jsrc, src);
   if (dst) (*env)->ReleaseStringUTFChars(env, jdst, dst);
+#endif
+
+#ifdef WINDOWS
+  LPCWSTR src = NULL, dst = NULL;
+
+  src = (LPCWSTR) (*env)->GetStringChars(env, jsrc, NULL);
+  if (!src) goto done; // exception was thrown
+  dst = (LPCWSTR) (*env)->GetStringChars(env, jdst, NULL);
+  if (!dst) goto done; // exception was thrown
+  if (!MoveFile(src, dst)) {
+    throw_ioe(env, GetLastError());
+  }
+
+done:
+  if (src) (*env)->ReleaseStringChars(env, jsrc, src);
+  if (dst) (*env)->ReleaseStringChars(env, jdst, dst);
+#endif
 }
 
 /**

+ 14 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java

@@ -446,7 +446,13 @@ public class TestNativeIO {
       NativeIO.renameTo(nonExistentFile, targetFile);
       Assert.fail();
     } catch (NativeIOException e) {
-      Assert.assertEquals(e.getErrno(), Errno.ENOENT);
+      if (Path.WINDOWS) {
+        Assert.assertEquals(
+          String.format("The system cannot find the file specified.%n"),
+          e.getMessage());
+      } else {
+        Assert.assertEquals(Errno.ENOENT, e.getErrno());
+      }
     }
     
     // Test renaming a file to itself.  It should succeed and do nothing.
@@ -465,7 +471,13 @@ public class TestNativeIO {
       NativeIO.renameTo(sourceFile, badTarget);
       Assert.fail();
     } catch (NativeIOException e) {
-      Assert.assertEquals(e.getErrno(), Errno.ENOTDIR);
+      if (Path.WINDOWS) {
+        Assert.assertEquals(
+          String.format("The parameter is incorrect.%n"),
+          e.getMessage());
+      } else {
+        Assert.assertEquals(Errno.ENOTDIR, e.getErrno());
+      }
     }
 
     FileUtils.deleteQuietly(TEST_DIR);

+ 63 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/filter/TestPatternFilter.java

@@ -24,7 +24,9 @@ import java.util.List;
 import org.apache.commons.configuration.SubsetConfiguration;
 import org.junit.Test;
 import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
 
+import org.apache.hadoop.metrics2.MetricsRecord;
 import org.apache.hadoop.metrics2.MetricsTag;
 import org.apache.hadoop.metrics2.impl.ConfigBuilder;
 import static org.apache.hadoop.metrics2.lib.Interns.*;
@@ -38,6 +40,8 @@ public class TestPatternFilter {
     SubsetConfiguration empty = new ConfigBuilder().subset("");
     shouldAccept(empty, "anything");
     shouldAccept(empty, Arrays.asList(tag("key", "desc", "value")));
+    shouldAccept(empty, mockMetricsRecord("anything", Arrays.asList(
+      tag("key", "desc", "value"))));
   }
 
   /**
@@ -50,9 +54,15 @@ public class TestPatternFilter {
     shouldAccept(wl, "foo");
     shouldAccept(wl, Arrays.asList(tag("bar", "", ""),
                                    tag("foo", "", "f")));
+    shouldAccept(wl, mockMetricsRecord("foo", Arrays.asList(
+      tag("bar", "", ""), tag("foo", "", "f"))));
     shouldReject(wl, "bar");
     shouldReject(wl, Arrays.asList(tag("bar", "", "")));
     shouldReject(wl, Arrays.asList(tag("foo", "", "boo")));
+    shouldReject(wl, mockMetricsRecord("bar", Arrays.asList(
+      tag("foo", "", "f"))));
+    shouldReject(wl, mockMetricsRecord("foo", Arrays.asList(
+      tag("bar", "", ""))));
   }
 
   /**
@@ -64,9 +74,15 @@ public class TestPatternFilter {
         .add("p.exclude.tags", "foo:f").subset("p");
     shouldAccept(bl, "bar");
     shouldAccept(bl, Arrays.asList(tag("bar", "", "")));
+    shouldAccept(bl, mockMetricsRecord("bar", Arrays.asList(
+      tag("bar", "", ""))));
     shouldReject(bl, "foo");
     shouldReject(bl, Arrays.asList(tag("bar", "", ""),
                                    tag("foo", "", "f")));
+    shouldReject(bl, mockMetricsRecord("foo", Arrays.asList(
+      tag("bar", "", ""))));
+    shouldReject(bl, mockMetricsRecord("bar", Arrays.asList(
+      tag("bar", "", ""), tag("foo", "", "f"))));
   }
 
   /**
@@ -81,10 +97,18 @@ public class TestPatternFilter {
         .add("p.exclude.tags", "bar:b").subset("p");
     shouldAccept(c, "foo");
     shouldAccept(c, Arrays.asList(tag("foo", "", "f")));
+    shouldAccept(c, mockMetricsRecord("foo", Arrays.asList(
+      tag("foo", "", "f"))));
     shouldReject(c, "bar");
     shouldReject(c, Arrays.asList(tag("bar", "", "b")));
+    shouldReject(c, mockMetricsRecord("bar", Arrays.asList(
+      tag("foo", "", "f"))));
+    shouldReject(c, mockMetricsRecord("foo", Arrays.asList(
+      tag("bar", "", "b"))));
     shouldAccept(c, "foobar");
     shouldAccept(c, Arrays.asList(tag("foobar", "", "")));
+    shouldAccept(c, mockMetricsRecord("foobar", Arrays.asList(
+      tag("foobar", "", ""))));
   }
 
   /**
@@ -98,6 +122,8 @@ public class TestPatternFilter {
         .add("p.exclude.tags", "foo:f").subset("p");
     shouldAccept(c, "foo");
     shouldAccept(c, Arrays.asList(tag("foo", "", "f")));
+    shouldAccept(c, mockMetricsRecord("foo", Arrays.asList(
+      tag("foo", "", "f"))));
   }
 
   static void shouldAccept(SubsetConfiguration conf, String s) {
@@ -110,6 +136,17 @@ public class TestPatternFilter {
     assertTrue("accepts "+ tags, newRegexFilter(conf).accepts(tags));
   }
 
+  /**
+   * Asserts that filters with the given configuration accept the given record.
+   * 
+   * @param conf SubsetConfiguration containing filter configuration
+   * @param record MetricsRecord to check
+   */
+  static void shouldAccept(SubsetConfiguration conf, MetricsRecord record) {
+    assertTrue("accepts " + record, newGlobFilter(conf).accepts(record));
+    assertTrue("accepts " + record, newRegexFilter(conf).accepts(record));
+  }
+
   static void shouldReject(SubsetConfiguration conf, String s) {
     assertTrue("rejects "+ s, !newGlobFilter(conf).accepts(s));
     assertTrue("rejects "+ s, !newRegexFilter(conf).accepts(s));
@@ -120,6 +157,17 @@ public class TestPatternFilter {
     assertTrue("rejects "+ tags, !newRegexFilter(conf).accepts(tags));
   }
 
+  /**
+   * Asserts that filters with the given configuration reject the given record.
+   * 
+   * @param conf SubsetConfiguration containing filter configuration
+   * @param record MetricsRecord to check
+   */
+  static void shouldReject(SubsetConfiguration conf, MetricsRecord record) {
+    assertTrue("rejects " + record, !newGlobFilter(conf).accepts(record));
+    assertTrue("rejects " + record, !newRegexFilter(conf).accepts(record));
+  }
+
   /**
    * Create a new glob filter with a config object
    * @param conf  the config object
@@ -141,4 +189,19 @@ public class TestPatternFilter {
     f.init(conf);
     return f;
   }
+
+  /**
+   * Creates a mock MetricsRecord with the given name and tags.
+   * 
+   * @param name String name
+   * @param tags List<MetricsTag> tags
+   * @return MetricsRecord newly created mock
+   */
+  private static MetricsRecord mockMetricsRecord(String name,
+      List<MetricsTag> tags) {
+    MetricsRecord record = mock(MetricsRecord.class);
+    when(record.name()).thenReturn(name);
+    when(record.tags()).thenReturn(tags);
+    return record;
+  }
 }

+ 32 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java

@@ -20,6 +20,9 @@ package org.apache.hadoop.test;
 import java.io.File;
 import java.io.IOException;
 import java.io.StringWriter;
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
 import java.lang.reflect.InvocationTargetException;
 import java.util.Arrays;
 import java.util.Random;
@@ -330,4 +333,33 @@ public abstract class GenericTestUtils {
         " but got:\n" + output,
         Pattern.compile(pattern).matcher(output).find());
   }
+  
+  public static void assertValueNear(long expected, long actual, long allowedError) {
+    assertValueWithinRange(expected - allowedError, expected + allowedError, actual);
+  }
+  
+  public static void assertValueWithinRange(long expectedMin, long expectedMax,
+      long actual) {
+    Assert.assertTrue("Expected " + actual + " to be in range (" + expectedMin + ","
+        + expectedMax + ")", expectedMin <= actual && actual <= expectedMax);
+  }
+
+  /**
+   * Assert that there are no threads running whose name matches the
+   * given regular expression.
+   * @param regex the regex to match against
+   */
+  public static void assertNoThreadsMatching(String regex) {
+    Pattern pattern = Pattern.compile(regex);
+    ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
+    
+    ThreadInfo[] infos = threadBean.getThreadInfo(threadBean.getAllThreadIds(), 20);
+    for (ThreadInfo info : infos) {
+      if (info == null) continue;
+      if (pattern.matcher(info.getThreadName()).matches()) {
+        Assert.fail("Leaked thread: " + info + "\n" +
+            Joiner.on("\n").join(info.getStackTrace()));
+      }
+    }
+  }
 }

+ 28 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MockitoUtil.java

@@ -20,6 +20,9 @@ package org.apache.hadoop.test;
 import java.io.Closeable;
 
 import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.mockito.stubbing.Stubber;
 
 public abstract class MockitoUtil {
 
@@ -33,4 +36,29 @@ public abstract class MockitoUtil {
     return Mockito.mock(clazz,
         Mockito.withSettings().extraInterfaces(Closeable.class));
   }
+
+  /**
+   * Throw an exception from the mock/spy only in the case that the
+   * call stack at the time the method has a line which matches the given
+   * pattern.
+   *
+   * @param t the Throwable to throw
+   * @param pattern the pattern against which to match the call stack trace
+   * @return the stub in progress
+   */
+  public static Stubber doThrowWhenCallStackMatches(
+      final Throwable t, final String pattern) {
+    return Mockito.doAnswer(new Answer<Object>() {
+      @Override
+      public Object answer(InvocationOnMock invocation) throws Throwable {
+        t.setStackTrace(Thread.currentThread().getStackTrace());
+        for (StackTraceElement elem : t.getStackTrace()) {
+          if (elem.toString().matches(pattern)) {
+            throw t;
+          }
+        }
+        return invocation.callRealMethod();
+      }
+    });
+  }
 }

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

@@ -318,6 +318,9 @@ Trunk (Unreleased)
     HDFS-4586. TestDataDirs.testGetDataDirsFromURIs fails with all directories
     in dfs.datanode.data.dir are invalid. (Ivan Mitic via atm)
 
+    HDFS-4646. createNNProxyWithClientProtocol ignores configured timeout
+    value (Jagane Sundar via cos)
+
   BREAKDOWN OF HADOOP-8562 SUBTASKS AND RELATED JIRAS
 
     HDFS-4145. Merge hdfs cmd line scripts from branch-1-win. (David Lao,
@@ -349,12 +352,18 @@ Trunk (Unreleased)
     HDFS-4625. Make TestNNWithQJM#testNewNamenodeTakesOverWriter work on
     Windows. (Ivan Mitic via suresh)
 
+    HDFS-4674. TestBPOfferService fails on Windows due to failure parsing 
+    datanode data directory as URI. (Chris Nauroth via suresh)
+
 Release 2.0.5-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES
 
   NEW FEATURES
 
+    HDFS-1804. Add a new block-volume device choosing policy that looks at
+    free space. (atm)
+
   IMPROVEMENTS
 
     HDFS-4222. NN is unresponsive and loses heartbeats from DNs when 
@@ -384,6 +393,12 @@ Release 2.0.5-beta - UNRELEASED
 
     HDFS-4618. Default transaction interval for checkpoints is too low. (todd)
 
+    HDFS-4525. Provide an API for knowing that whether file is closed or not. 
+    (SreeHari via umamahesh)
+
+    HDFS-3940. Add Gset#clear method and clear the block map when namenode is
+    shutdown. (suresh)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -474,6 +489,17 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4658. Standby NN will log that it has received a block report "after
     becoming active" (atm)
 
+    HDFS-3981. Fix handling of FSN lock in getBlockLocations. (Xiaobo Peng
+    and todd via todd)
+
+    HDFS-4676. TestHDFSFileSystemContract should set MiniDFSCluster variable
+    to null to free up memory. (suresh)
+
+    HDFS-4669. TestBlockPoolManager fails using IBM java. (Tian Hong Wang via
+    suresh)
+
+    HDFS-4643. Fix flakiness in TestQuorumJournalManager. (todd)
+
 Release 2.0.4-alpha - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 16 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -1554,7 +1554,22 @@ public class DFSClient implements java.io.Closeable {
                                      UnresolvedPathException.class);
     }
   }
-
+  
+  /**
+   * Close status of a file
+   * @return true if file is already closed
+   */
+  public boolean isFileClosed(String src) throws IOException{
+    checkOpen();
+    try {
+      return namenode.isFileClosed(src);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     UnresolvedPathException.class);
+    }
+  }
+  
   /**
    * Get the file info for a specific file or directory. If src
    * refers to a symlink then the FileStatus of the link is returned.

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -369,6 +369,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_DATANODE_PLUGINS_KEY = "dfs.datanode.plugins";
   public static final String  DFS_DATANODE_FSDATASET_FACTORY_KEY = "dfs.datanode.fsdataset.factory";
   public static final String  DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_POLICY_KEY = "dfs.datanode.fsdataset.volume.choosing.policy";
+  public static final String  DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_KEY = "dfs.datanode.available-space-volume-choosing-policy.balanced-space-threshold";
+  public static final long    DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_DEFAULT = 1024L * 1024L * 1024L * 10L; // 10 GB
+  public static final String  DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_KEY = "dfs.datanode.available-space-volume-choosing-policy.balanced-space-preference-percent";
+  public static final float   DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_DEFAULT = 0.75f;
   public static final String  DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY = "dfs.datanode.socket.write.timeout";
   public static final String  DFS_DATANODE_STARTUP_KEY = "dfs.datanode.startup";
   public static final String  DFS_NAMENODE_PLUGINS_KEY = "dfs.namenode.plugins";

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

@@ -507,7 +507,7 @@ public class DFSUtil {
     
     // Look for configurations of the form <key>[.<nameserviceId>][.<namenodeId>]
     // across all of the configured nameservices and namenodes.
-    Map<String, Map<String, InetSocketAddress>> ret = Maps.newHashMap();
+    Map<String, Map<String, InetSocketAddress>> ret = Maps.newLinkedHashMap();
     for (String nsId : emptyAsSingletonNull(nameserviceIds)) {
       Map<String, InetSocketAddress> isas =
         getAddressesForNameserviceId(conf, nsId, defaultAddress, keys);

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -968,4 +968,17 @@ public class DistributedFileSystem extends FileSystem {
       String fromSnapshot, String toSnapshot) throws IOException {
     return dfs.getSnapshotDiffReport(snapshotDir, fromSnapshot, toSnapshot);
   }
+ 
+  /**
+   * Get the close status of a file
+   * @param src The path to the file
+   *
+   * @return return true if file is closed
+   * @throws FileNotFoundException if the file does not exist.
+   * @throws IOException If an I/O error occurred     
+   */
+  public boolean isFileClosed(Path src) throws IOException {
+    return dfs.isFileClosed(getPathName(src));
+  }
+  
 }

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java

@@ -260,7 +260,9 @@ public class NameNodeProxies {
     final long version = RPC.getProtocolVersion(ClientNamenodeProtocolPB.class);
     ClientNamenodeProtocolPB proxy = RPC.getProtocolProxy(
         ClientNamenodeProtocolPB.class, version, address, ugi, conf,
-        NetUtils.getDefaultSocketFactory(conf), 0, defaultPolicy).getProxy();
+        NetUtils.getDefaultSocketFactory(conf),
+        org.apache.hadoop.ipc.Client.getTimeout(conf), defaultPolicy)
+            .getProxy();
 
     if (withRetries) { // create the proxy with retries
 

+ 15 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -778,7 +778,21 @@ public interface ClientProtocol {
   @Idempotent
   public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
       FileNotFoundException, UnresolvedLinkException, IOException;
-
+  
+  /**
+   * Get the close status of a file
+   * @param src The string representation of the path to the file
+   *
+   * @return return true if file is closed
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException if the path contains a symlink.
+   * @throws IOException If an I/O error occurred     
+   */
+  @Idempotent
+  public boolean isFileClosed(String src) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException;
+  
   /**
    * Get the file info for a specific file or directory. If the path 
    * refers to a symlink then the FileStatus of the symlink is returned.

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java

@@ -97,7 +97,8 @@ public class LayoutVersion {
         "Serialize block lists with delta-encoded variable length ints, " +
         "add OP_UPDATE_BLOCKS"),
     RESERVED_REL1_2_0(-41, -32, "Reserved for release 1.2.0", true, CONCAT),
-    SNAPSHOT(-42, -40, "Support for snapshot feature", false);
+    ADD_INODE_ID(-42, -40, "Assign a unique inode id for each inode", false),
+    SNAPSHOT(-43, -42, "Support for snapshot feature", false);
     
     final int lv;
     final int ancestorLV;

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -90,6 +90,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSna
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
@@ -984,4 +986,17 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public IsFileClosedResponseProto isFileClosed(
+      RpcController controller, IsFileClosedRequestProto request) 
+      throws ServiceException {
+    try {
+      boolean result = server.isFileClosed(request.getSrc());
+      return IsFileClosedResponseProto.newBuilder().setResult(result).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+  
 }

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -86,6 +86,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSna
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
@@ -861,6 +862,19 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+  
+
+  @Override
+  public boolean isFileClosed(String src) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
+    IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      return rpcProxy.isFileClosed(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 
   @Override
   public Object getUnderlyingProxyObject() {

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -3186,4 +3186,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     OK
   }
 
+  public void shutdown() {
+    blocksMap.close();
+  }
 }

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

@@ -67,7 +67,7 @@ class BlocksMap {
 
 
   void close() {
-    // Empty blocks once GSet#clear is implemented (HDFS-3940)
+    blocks.clear();
   }
 
   BlockCollection getBlockCollection(Block b) {

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

@@ -160,8 +160,8 @@ class BlockPoolManager {
       Map<String, Map<String, InetSocketAddress>> addrMap) throws IOException {
     assert Thread.holdsLock(refreshNamenodesLock);
 
-    Set<String> toRefresh = Sets.newHashSet();
-    Set<String> toAdd = Sets.newHashSet();
+    Set<String> toRefresh = Sets.newLinkedHashSet();
+    Set<String> toAdd = Sets.newLinkedHashSet();
     Set<String> toRemove;
     
     synchronized (this) {
@@ -239,4 +239,4 @@ class BlockPoolManager {
   protected BPOfferService createBPOS(List<InetSocketAddress> nnAddrs) {
     return new BPOfferService(nnAddrs, dn);
   }
-}
+}

+ 259 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/AvailableSpaceVolumeChoosingPolicy.java

@@ -0,0 +1,259 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_KEY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
+
+/**
+ * A DN volume choosing policy which takes into account the amount of free
+ * space on each of the available volumes when considering where to assign a
+ * new replica allocation. By default this policy prefers assigning replicas to
+ * those volumes with more available free space, so as to over time balance the
+ * available space of all the volumes within a DN.
+ */
+public class AvailableSpaceVolumeChoosingPolicy<V extends FsVolumeSpi>
+    implements VolumeChoosingPolicy<V>, Configurable {
+  
+  private static final Log LOG = LogFactory.getLog(AvailableSpaceVolumeChoosingPolicy.class);
+  
+  private static final Random RAND = new Random();
+  
+  private long balancedSpaceThreshold = DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_DEFAULT;
+  private float balancedPreferencePercent = DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_DEFAULT;
+
+  @Override
+  public synchronized void setConf(Configuration conf) {
+    balancedSpaceThreshold = conf.getLong(
+        DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_KEY,
+        DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_DEFAULT);
+    balancedPreferencePercent = conf.getFloat(
+        DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_KEY,
+        DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_DEFAULT);
+    
+    LOG.info("Available space volume choosing policy initialized: " +
+        DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_KEY +
+        " = " + balancedSpaceThreshold + ", " +
+        DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_KEY +
+        " = " + balancedPreferencePercent);
+  }
+  
+  @Override
+  public synchronized Configuration getConf() {
+    // Nothing to do. Only added to fulfill the Configurable contract.
+    return null;
+  }
+  
+  private VolumeChoosingPolicy<V> roundRobinPolicyBalanced =
+      new RoundRobinVolumeChoosingPolicy<V>();
+  private VolumeChoosingPolicy<V> roundRobinPolicyHighAvailable =
+      new RoundRobinVolumeChoosingPolicy<V>();
+  private VolumeChoosingPolicy<V> roundRobinPolicyLowAvailable =
+      new RoundRobinVolumeChoosingPolicy<V>();
+
+  @Override
+  public synchronized V chooseVolume(List<V> volumes,
+      final long replicaSize) throws IOException {
+    if (volumes.size() < 1) {
+      throw new DiskOutOfSpaceException("No more available volumes");
+    }
+    
+    AvailableSpaceVolumeList volumesWithSpaces =
+        new AvailableSpaceVolumeList(volumes);
+    
+    if (volumesWithSpaces.areAllVolumesWithinFreeSpaceThreshold()) {
+      // If they're actually not too far out of whack, fall back on pure round
+      // robin.
+      V volume = roundRobinPolicyBalanced.chooseVolume(volumes, replicaSize);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("All volumes are within the configured free space balance " +
+            "threshold. Selecting " + volume + " for write of block size " +
+            replicaSize);
+      }
+      return volume;
+    } else {
+      V volume = null;
+      // If none of the volumes with low free space have enough space for the
+      // replica, always try to choose a volume with a lot of free space.
+      long mostAvailableAmongLowVolumes = volumesWithSpaces
+          .getMostAvailableSpaceAmongVolumesWithLowAvailableSpace();
+      
+      List<V> highAvailableVolumes = extractVolumesFromPairs(
+          volumesWithSpaces.getVolumesWithHighAvailableSpace());
+      List<V> lowAvailableVolumes = extractVolumesFromPairs(
+          volumesWithSpaces.getVolumesWithLowAvailableSpace());
+      
+      float preferencePercentScaler =
+          (highAvailableVolumes.size() * balancedPreferencePercent) +
+          (lowAvailableVolumes.size() * (1 - balancedPreferencePercent));
+      float scaledPreferencePercent =
+          (highAvailableVolumes.size() * balancedPreferencePercent) /
+          preferencePercentScaler;
+      if (mostAvailableAmongLowVolumes < replicaSize ||
+          RAND.nextFloat() < scaledPreferencePercent) {
+        volume = roundRobinPolicyHighAvailable.chooseVolume(
+            highAvailableVolumes,
+            replicaSize);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Volumes are imbalanced. Selecting " + volume +
+              " from high available space volumes for write of block size "
+              + replicaSize);
+        }
+      } else {
+        volume = roundRobinPolicyLowAvailable.chooseVolume(
+            lowAvailableVolumes,
+            replicaSize);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Volumes are imbalanced. Selecting " + volume +
+              " from low available space volumes for write of block size "
+              + replicaSize);
+        }
+      }
+      return volume;
+    }
+  }
+  
+  /**
+   * Used to keep track of the list of volumes we're choosing from.
+   */
+  private class AvailableSpaceVolumeList {
+    private final List<AvailableSpaceVolumePair> volumes;
+    
+    public AvailableSpaceVolumeList(List<V> volumes) throws IOException {
+      this.volumes = new ArrayList<AvailableSpaceVolumePair>();
+      for (V volume : volumes) {
+        this.volumes.add(new AvailableSpaceVolumePair(volume));
+      }
+    }
+    
+    /**
+     * Check if the available space on all the volumes is roughly equal.
+     * 
+     * @param volumes the volumes to check
+     * @return true if all volumes' free space is within the configured threshold,
+     *         false otherwise.
+     * @throws IOException
+     *           in the event of error checking amount of available space
+     */
+    public boolean areAllVolumesWithinFreeSpaceThreshold() {
+      long leastAvailable = Long.MAX_VALUE;
+      long mostAvailable = 0;
+      for (AvailableSpaceVolumePair volume : volumes) {
+        leastAvailable = Math.min(leastAvailable, volume.getAvailable());
+        mostAvailable = Math.max(mostAvailable, volume.getAvailable());
+      }
+      return (mostAvailable - leastAvailable) < balancedSpaceThreshold;
+    }
+    
+    /**
+     * @return the minimum amount of space available on a single volume,
+     *         across all volumes.
+     */
+    private long getLeastAvailableSpace() {
+      long leastAvailable = Long.MAX_VALUE;
+      for (AvailableSpaceVolumePair volume : volumes) {
+        leastAvailable = Math.min(leastAvailable, volume.getAvailable());
+      }
+      return leastAvailable;
+    }
+    
+    /**
+     * @return the maximum amount of space available across volumes with low space.
+     */
+    public long getMostAvailableSpaceAmongVolumesWithLowAvailableSpace() {
+      long mostAvailable = Long.MIN_VALUE;
+      for (AvailableSpaceVolumePair volume : getVolumesWithLowAvailableSpace()) {
+        mostAvailable = Math.max(mostAvailable, volume.getAvailable());
+      }
+      return mostAvailable;
+    }
+    
+    /**
+     * @return the list of volumes with relatively low available space.
+     */
+    public List<AvailableSpaceVolumePair> getVolumesWithLowAvailableSpace() {
+      long leastAvailable = getLeastAvailableSpace();
+      List<AvailableSpaceVolumePair> ret = new ArrayList<AvailableSpaceVolumePair>();
+      for (AvailableSpaceVolumePair volume : volumes) {
+        if (volume.getAvailable() <= leastAvailable + balancedSpaceThreshold) {
+          ret.add(volume);
+        }
+      }
+      return ret;
+    }
+    
+    /**
+     * @return the list of volumes with a lot of available space.
+     */
+    public List<AvailableSpaceVolumePair> getVolumesWithHighAvailableSpace() {
+      long leastAvailable = getLeastAvailableSpace();
+      List<AvailableSpaceVolumePair> ret = new ArrayList<AvailableSpaceVolumePair>();
+      for (AvailableSpaceVolumePair volume : volumes) {
+        if (volume.getAvailable() > leastAvailable + balancedSpaceThreshold) {
+          ret.add(volume);
+        }
+      }
+      return ret;
+    }
+    
+  }
+  
+  /**
+   * Used so that we only check the available space on a given volume once, at
+   * the beginning of {@link AvailableSpaceVolumeChoosingPolicy#chooseVolume(List, long)}.
+   */
+  private class AvailableSpaceVolumePair {
+    private final V volume;
+    private final long availableSpace;
+    
+    public AvailableSpaceVolumePair(V volume) throws IOException {
+      this.volume = volume;
+      this.availableSpace = volume.getAvailable();
+    }
+    
+    public long getAvailable() {
+      return availableSpace;
+    }
+    
+    public V getVolume() {
+      return volume;
+    }
+  }
+  
+  private List<V> extractVolumesFromPairs(List<AvailableSpaceVolumePair> volumes) {
+    List<V> ret = new ArrayList<V>();
+    for (AvailableSpaceVolumePair volume : volumes) {
+      ret.add(volume.getVolume());
+    }
+    return ret;
+  }
+
+}

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -2388,4 +2388,8 @@ public class FSDirectory implements Closeable {
       inode.setLocalName(name.getBytes());
     }
   }
+  
+  void shutdown() {
+    nameCache.reset();
+  }
 }

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -661,6 +661,7 @@ public class FSEditLog implements LogsPurgeable {
    */
   public void logOpenFile(String path, INodeFileUnderConstruction newNode) {
     AddOp op = AddOp.getInstance(cache.get())
+      .setInodeId(newNode.getId())
       .setPath(path)
       .setReplication(newNode.getFileReplication())
       .setModificationTime(newNode.getModificationTime())
@@ -702,6 +703,7 @@ public class FSEditLog implements LogsPurgeable {
    */
   public void logMkDir(String path, INode newNode) {
     MkdirOp op = MkdirOp.getInstance(cache.get())
+      .setInodeId(newNode.getId())
       .setPath(path)
       .setTimestamp(newNode.getModificationTime())
       .setPermissionStatus(newNode.getPermissionStatus());
@@ -819,6 +821,7 @@ public class FSEditLog implements LogsPurgeable {
   void logSymlink(String path, String value, long mtime, 
                   long atime, INodeSymlink node) {
     SymlinkOp op = SymlinkOp.getInstance(cache.get())
+      .setId(node.getId())
       .setPath(path)
       .setValue(value)
       .setModificationTime(mtime)

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

@@ -32,6 +32,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -174,7 +175,7 @@ public class FSEditLogLoader {
             }
           }
           try {
-            long inodeId = applyEditLogOp(op, fsDir, in.getVersion());
+            long inodeId = applyEditLogOp(op, fsDir, in.getVersion(), lastInodeId);
             if (lastInodeId < inodeId) {
               lastInodeId = inodeId;
             }
@@ -230,9 +231,30 @@ public class FSEditLogLoader {
     return numEdits;
   }
   
+  // allocate and update last allocated inode id
+  private long getAndUpdateLastInodeId(long inodeIdFromOp, int logVersion,
+      long lastInodeId) throws IOException {
+    long inodeId = inodeIdFromOp;
+
+    if (inodeId == INodeId.GRANDFATHER_INODE_ID) {
+      if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
+        throw new IOException("The layout version " + logVersion
+            + " supports inodeId but gave bogus inodeId");
+      }
+      inodeId = fsNamesys.allocateNewInodeId();
+    } else {
+      // need to reset lastInodeId. fsnamesys gets lastInodeId firstly from
+      // fsimage but editlog captures more recent inodeId allocations
+      if (inodeId > lastInodeId) {
+        fsNamesys.resetLastInodeId(inodeId);
+      }
+    }
+    return inodeId;
+  }
+
   @SuppressWarnings("deprecation")
   private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
-      int logVersion) throws IOException {
+      int logVersion, long lastInodeId) throws IOException {
     long inodeId = INodeId.GRANDFATHER_INODE_ID;
     if (LOG.isTraceEnabled()) {
       LOG.trace("replaying edit log: " + op);
@@ -265,7 +287,8 @@ public class FSEditLogLoader {
         assert addCloseOp.blocks.length == 0;
 
         // add to the file tree
-        inodeId = fsNamesys.allocateNewInodeId();
+        inodeId = getAndUpdateLastInodeId(addCloseOp.inodeId, logVersion,
+            lastInodeId);
         newFile = fsDir.unprotectedAddFile(inodeId,
             addCloseOp.path, addCloseOp.permissions, replication,
             addCloseOp.mtime, addCloseOp.atime, addCloseOp.blockSize, true,
@@ -373,7 +396,8 @@ public class FSEditLogLoader {
     }
     case OP_MKDIR: {
       MkdirOp mkdirOp = (MkdirOp)op;
-      inodeId = fsNamesys.allocateNewInodeId();
+      inodeId = getAndUpdateLastInodeId(mkdirOp.inodeId, logVersion,
+          lastInodeId);
       fsDir.unprotectedMkdir(inodeId, mkdirOp.path, mkdirOp.permissions,
                              mkdirOp.timestamp);
       break;
@@ -427,7 +451,8 @@ public class FSEditLogLoader {
     }
     case OP_SYMLINK: {
       SymlinkOp symlinkOp = (SymlinkOp)op;
-      inodeId = fsNamesys.allocateNewInodeId();
+      inodeId = getAndUpdateLastInodeId(symlinkOp.inodeId, logVersion,
+          lastInodeId);
       fsDir.unprotectedAddSymlink(inodeId, symlinkOp.path,
                                   symlinkOp.value, symlinkOp.mtime, 
                                   symlinkOp.atime, symlinkOp.permissionStatus);

+ 54 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -164,6 +164,7 @@ public abstract class FSEditLogOp {
   @SuppressWarnings("unchecked")
   static abstract class AddCloseOp extends FSEditLogOp implements BlockListUpdatingOp {
     int length;
+    long inodeId;
     String path;
     short replication;
     long mtime;
@@ -178,6 +179,11 @@ public abstract class FSEditLogOp {
       super(opCode);
       assert(opCode == OP_ADD || opCode == OP_CLOSE);
     }
+    
+    <T extends AddCloseOp> T setInodeId(long inodeId) {
+      this.inodeId = inodeId;
+      return (T)this;
+    }
 
     <T extends AddCloseOp> T setPath(String path) {
       this.path = path;
@@ -241,6 +247,7 @@ public abstract class FSEditLogOp {
     @Override
     public 
     void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeLong(inodeId, out);
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeShort(replication, out);
       FSImageSerialization.writeLong(mtime, out);
@@ -261,6 +268,12 @@ public abstract class FSEditLogOp {
       if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.length = in.readInt();
       }
+      if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
+        this.inodeId = in.readLong();
+      } else {
+        // The inodeId should be updated when this editLogOp is applied
+        this.inodeId = INodeId.GRANDFATHER_INODE_ID;
+      }
       if ((-17 < logVersion && length != 4) ||
           (logVersion <= -17 && length != 5 && !LayoutVersion.supports(
               Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) {
@@ -333,6 +346,8 @@ public abstract class FSEditLogOp {
       StringBuilder builder = new StringBuilder();
       builder.append("[length=");
       builder.append(length);
+      builder.append(", inodeId=");
+      builder.append(inodeId);
       builder.append(", path=");
       builder.append(path);
       builder.append(", replication=");
@@ -363,6 +378,8 @@ public abstract class FSEditLogOp {
     protected void toXml(ContentHandler contentHandler) throws SAXException {
       XMLUtils.addSaxString(contentHandler, "LENGTH",
           Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "INODEID",
+          Long.valueOf(inodeId).toString());
       XMLUtils.addSaxString(contentHandler, "PATH", path);
       XMLUtils.addSaxString(contentHandler, "REPLICATION",
           Short.valueOf(replication).toString());
@@ -382,6 +399,7 @@ public abstract class FSEditLogOp {
 
     @Override void fromXml(Stanza st) throws InvalidXmlException {
       this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.inodeId = Long.valueOf(st.getValue("INODEID"));
       this.path = st.getValue("PATH");
       this.replication = Short.valueOf(st.getValue("REPLICATION"));
       this.mtime = Long.valueOf(st.getValue("MTIME"));
@@ -913,6 +931,7 @@ public abstract class FSEditLogOp {
     
   static class MkdirOp extends FSEditLogOp {
     int length;
+    long inodeId;
     String path;
     long timestamp;
     PermissionStatus permissions;
@@ -925,6 +944,11 @@ public abstract class FSEditLogOp {
       return (MkdirOp)cache.get(OP_MKDIR);
     }
 
+    MkdirOp setInodeId(long inodeId) {
+      this.inodeId = inodeId;
+      return this;
+    }
+    
     MkdirOp setPath(String path) {
       this.path = path;
       return this;
@@ -943,6 +967,7 @@ public abstract class FSEditLogOp {
     @Override
     public 
     void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeLong(inodeId, out);
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeLong(timestamp, out); // mtime
       FSImageSerialization.writeLong(timestamp, out); // atime, unused at this
@@ -959,6 +984,12 @@ public abstract class FSEditLogOp {
           && !LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         throw new IOException("Incorrect data format. Mkdir operation.");
       }
+      if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
+        this.inodeId = FSImageSerialization.readLong(in);
+      } else {
+        // This id should be updated when this editLogOp is applied
+        this.inodeId = INodeId.GRANDFATHER_INODE_ID;
+      }
       this.path = FSImageSerialization.readString(in);
       if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.timestamp = FSImageSerialization.readLong(in);
@@ -985,6 +1016,8 @@ public abstract class FSEditLogOp {
       StringBuilder builder = new StringBuilder();
       builder.append("MkdirOp [length=");
       builder.append(length);
+      builder.append(", inodeId=");
+      builder.append(inodeId);
       builder.append(", path=");
       builder.append(path);
       builder.append(", timestamp=");
@@ -1003,6 +1036,8 @@ public abstract class FSEditLogOp {
     protected void toXml(ContentHandler contentHandler) throws SAXException {
       XMLUtils.addSaxString(contentHandler, "LENGTH",
           Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "INODEID",
+          Long.valueOf(inodeId).toString());
       XMLUtils.addSaxString(contentHandler, "PATH", path);
       XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
           Long.valueOf(timestamp).toString());
@@ -1011,6 +1046,7 @@ public abstract class FSEditLogOp {
     
     @Override void fromXml(Stanza st) throws InvalidXmlException {
       this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.inodeId = Long.valueOf(st.getValue("INODEID"));
       this.path = st.getValue("PATH");
       this.timestamp = Long.valueOf(st.getValue("TIMESTAMP"));
       this.permissions =
@@ -1489,6 +1525,7 @@ public abstract class FSEditLogOp {
 
   static class SymlinkOp extends FSEditLogOp {
     int length;
+    long inodeId;
     String path;
     String value;
     long mtime;
@@ -1503,6 +1540,11 @@ public abstract class FSEditLogOp {
       return (SymlinkOp)cache.get(OP_SYMLINK);
     }
 
+    SymlinkOp setId(long inodeId) {
+      this.inodeId = inodeId;
+      return this;
+    }
+    
     SymlinkOp setPath(String path) {
       this.path = path;
       return this;
@@ -1531,6 +1573,7 @@ public abstract class FSEditLogOp {
     @Override
     public 
     void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeLong(inodeId, out);      
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeString(value, out);
       FSImageSerialization.writeLong(mtime, out);
@@ -1548,6 +1591,12 @@ public abstract class FSEditLogOp {
               + "symlink operation.");
         }
       }
+      if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
+        this.inodeId = FSImageSerialization.readLong(in);
+      } else {
+        // This id should be updated when the editLogOp is applied
+        this.inodeId = INodeId.GRANDFATHER_INODE_ID;
+      }
       this.path = FSImageSerialization.readString(in);
       this.value = FSImageSerialization.readString(in);
 
@@ -1566,6 +1615,8 @@ public abstract class FSEditLogOp {
       StringBuilder builder = new StringBuilder();
       builder.append("SymlinkOp [length=");
       builder.append(length);
+      builder.append(", inodeId=");
+      builder.append(inodeId);
       builder.append(", path=");
       builder.append(path);
       builder.append(", value=");
@@ -1588,6 +1639,8 @@ public abstract class FSEditLogOp {
     protected void toXml(ContentHandler contentHandler) throws SAXException {
       XMLUtils.addSaxString(contentHandler, "LENGTH",
           Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "INODEID",
+          Long.valueOf(inodeId).toString());
       XMLUtils.addSaxString(contentHandler, "PATH", path);
       XMLUtils.addSaxString(contentHandler, "VALUE", value);
       XMLUtils.addSaxString(contentHandler, "MTIME",
@@ -1599,6 +1652,7 @@ public abstract class FSEditLogOp {
 
     @Override void fromXml(Stanza st) throws InvalidXmlException {
       this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.inodeId = Long.valueOf(st.getValue("INODEID"));
       this.path = st.getValue("PATH");
       this.value = st.getValue("VALUE");
       this.mtime = Long.valueOf(st.getValue("MTIME"));

+ 22 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -275,6 +275,20 @@ public class FSImageFormat {
           snapshotMap = namesystem.getSnapshotManager().read(in, this);
         }
 
+        // read the last allocated inode id in the fsimage
+        if (LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion)) {
+          long lastInodeId = in.readLong();
+          namesystem.resetLastInodeId(lastInodeId);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("load last allocated InodeId from fsimage:" + lastInodeId);
+          }
+        } else {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Old layout version doesn't have inode id."
+                + " Will assign new id for each inode.");
+          }
+        }
+        
         // read compression related info
         FSImageCompression compression;
         if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imgVersion)) {
@@ -285,8 +299,7 @@ public class FSImageFormat {
         in = compression.unwrapInputStream(fin);
 
         LOG.info("Loading image file " + curFile + " using " + compression);
-        // reset INodeId. TODO: remove this after inodeId is persisted in fsimage
-        namesystem.resetLastInodeIdWithoutChecking(INodeId.LAST_RESERVED_ID); 
+        
         // load all inodes
         LOG.info("Number of files = " + numFiles);
         if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
@@ -356,7 +369,7 @@ public class FSImageFormat {
    * @throws IOException
    */  
    private void loadLocalNameINodes(long numFiles, DataInput in) 
-   throws IOException {
+       throws IOException {
      assert LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
          getLayoutVersion());
      assert numFiles > 0;
@@ -546,7 +559,8 @@ public class FSImageFormat {
   INode loadINode(final byte[] localName, boolean isSnapshotINode,
       DataInput in) throws IOException {
     final int imgVersion = getLayoutVersion();
-    final long inodeId = namesystem.allocateNewInodeId();
+    long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? 
+           in.readLong() : namesystem.allocateNewInodeId();
     
     final short replication = namesystem.getBlockManager().adjustReplication(
         in.readShort());
@@ -662,8 +676,8 @@ public class FSImageFormat {
       LOG.info("Number of files under construction = " + size);
 
       for (int i = 0; i < size; i++) {
-        INodeFileUnderConstruction cons
-            = FSImageSerialization.readINodeUnderConstruction(in);
+        INodeFileUnderConstruction cons = FSImageSerialization
+            .readINodeUnderConstruction(in, namesystem, getLayoutVersion());
 
         // verify that file exists in namespace
         String path = cons.getLocalName();
@@ -804,6 +818,8 @@ public class FSImageFormat {
         out.writeLong(fsDir.rootDir.numItemsInTree());
         out.writeLong(sourceNamesystem.getGenerationStamp());
         out.writeLong(context.getTxId());
+        out.writeLong(sourceNamesystem.getLastInodeId());
+        
         sourceNamesystem.getSnapshotManager().write(out);
         
         // write compression info and set up compressed stream

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

@@ -30,6 +30,8 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@@ -103,8 +105,11 @@ public class FSImageSerialization {
   // from the input stream
   //
   static INodeFileUnderConstruction readINodeUnderConstruction(
-      DataInput in) throws IOException {
+      DataInput in, FSNamesystem fsNamesys, int imgVersion)
+      throws IOException {
     byte[] name = readBytes(in);
+    long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? in
+        .readLong() : fsNamesys.allocateNewInodeId();
     short blockReplication = in.readShort();
     long modificationTime = in.readLong();
     long preferredBlockSize = in.readLong();
@@ -132,10 +137,16 @@ public class FSImageSerialization {
     int numLocs = in.readInt();
     assert numLocs == 0 : "Unexpected block locations";
 
-    //TODO: get inodeId from fsimage after inodeId is persisted
-    return new INodeFileUnderConstruction(
-        INodeId.GRANDFATHER_INODE_ID, name, blockReplication, modificationTime,
-        preferredBlockSize, blocks, perm, clientName, clientMachine, null);
+    return new INodeFileUnderConstruction(inodeId,
+                                          name,
+                                          blockReplication, 
+                                          modificationTime,
+                                          preferredBlockSize,
+                                          blocks,
+                                          perm,
+                                          clientName,
+                                          clientMachine,
+                                          null);
   }
 
   // Helper function that writes an INodeUnderConstruction
@@ -146,6 +157,7 @@ public class FSImageSerialization {
                                            String path) 
                                            throws IOException {
     writeString(path, out);
+    out.writeLong(cons.getId());
     out.writeShort(cons.getFileReplication());
     out.writeLong(cons.getModificationTime());
     out.writeLong(cons.getPreferredBlockSize());
@@ -168,6 +180,7 @@ public class FSImageSerialization {
   public static void writeINodeFile(INodeFile file, DataOutput out,
       boolean writeUnderConstruction) throws IOException {
     writeLocalName(file, out);
+    out.writeLong(file.getId());
     out.writeShort(file.getFileReplication());
     out.writeLong(file.getModificationTime());
     out.writeLong(file.getAccessTime());
@@ -198,6 +211,7 @@ public class FSImageSerialization {
   public static void writeINodeDirectory(INodeDirectory node, DataOutput out)
       throws IOException {
     writeLocalName(node, out);
+    out.writeLong(node.getId());
     out.writeShort(0);  // replication
     out.writeLong(node.getModificationTime());
     out.writeLong(0);   // access time
@@ -224,6 +238,7 @@ public class FSImageSerialization {
   private static void writeINodeSymlink(INodeSymlink node, DataOutput out)
       throws IOException {
     writeLocalName(node, out);
+    out.writeLong(node.getId());
     out.writeShort(0);  // replication
     out.writeLong(0);   // modification time
     out.writeLong(0);   // access time
@@ -239,6 +254,7 @@ public class FSImageSerialization {
       boolean writeUnderConstruction, ReferenceMap referenceMap
       ) throws IOException {
     writeLocalName(ref, out);
+    out.writeLong(ref.getId());
     out.writeShort(0);  // replication
     out.writeLong(0);   // modification time
     out.writeLong(0);   // access time
@@ -403,4 +419,4 @@ public class FSImageSerialization {
     }
     return ret;
   }
-}
+}

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

@@ -1391,14 +1391,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         if (!iip.isSnapshot() //snapshots are readonly, so don't update atime.
             && doAccessTime && isAccessTimeSupported()) {
           final long now = now();
-          if (now <= inode.getAccessTime() + getAccessTimePrecision()) {
+          if (now > inode.getAccessTime() + getAccessTimePrecision()) {
             // if we have to set access time but we only have the readlock, then
             // restart this entire operation with the writeLock.
             if (isReadOp) {
               continue;
             }
+            dir.setTimes(src, inode, -1, now, false, iip.getLatestSnapshot());
           }
-          dir.setTimes(src, inode, -1, now, false, iip.getLatestSnapshot());
         }
         final long fileSize = iip.getPathSnapshot() != null?
             inode.computeFileSize(iip.getPathSnapshot())
@@ -2992,6 +2992,33 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     logAuditEvent(true, "getfileinfo", src);
     return stat;
   }
+  
+  /**
+   * Returns true if the file is closed
+   */
+  boolean isFileClosed(String src) 
+      throws AccessControlException, UnresolvedLinkException,
+      StandbyException, IOException {
+    FSPermissionChecker pc = getPermissionChecker();	
+    checkOperation(OperationCategory.READ);
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      if (isPermissionEnabled) {
+        checkTraverse(pc, src);
+      }
+      return !INodeFile.valueOf(dir.getINode(src), src).isUnderConstruction();
+    } catch (AccessControlException e) {
+      if (isAuditEnabled() && isExternalInvocation()) {
+        logAuditEvent(false, UserGroupInformation.getCurrentUser(),
+                      getRemoteIp(),
+                      "isFileClosed", src, null, null);
+      }
+      throw e;
+    } finally {
+      readUnlock();
+    }
+  }
 
   /**
    * Create all the necessary directories
@@ -4828,8 +4855,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * shutdown FSNamesystem
    */
   void shutdown() {
-    if (mbeanName != null)
+    if (mbeanName != null) {
       MBeans.unregister(mbeanName);
+    }
+    if (dir != null) {
+      dir.shutdown();
+    }
+    if (blockManager != null) {
+      blockManager.shutdown();
+    }
   }
   
 

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

@@ -692,7 +692,12 @@ class NameNodeRpcServer implements NamenodeProtocols {
     metrics.incrFileInfoOps();
     return namesystem.getFileInfo(src, true);
   }
-
+  
+  @Override // ClientProtocol
+  public boolean isFileClosed(String src) throws IOException{
+    return namesystem.isFileClosed(src);
+  }
+  
   @Override // ClientProtocol
   public HdfsFileStatus getFileLinkInfo(String src) throws IOException { 
     metrics.incrFileInfoOps();

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -171,6 +171,10 @@ class ImageLoaderCurrent implements ImageLoader {
         v.visit(ImageElement.NUM_SNAPSHOTTABLE_DIRS, in.readInt());
       }
 
+      if (LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) {
+        v.visit(ImageElement.LAST_INODE_ID, in.readLong());
+      }
+      
       if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imageVersion)) {
         boolean isCompressed = in.readBoolean();
         v.visit(ImageElement.IS_COMPRESSED, String.valueOf(isCompressed));
@@ -577,6 +581,9 @@ class ImageLoaderCurrent implements ImageLoader {
     }
 
     v.visit(ImageElement.INODE_PATH, pathName);
+    if (LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) {
+      v.visit(ImageElement.INODE_ID, in.readLong());
+    }
     v.visit(ImageElement.REPLICATION, in.readShort());
     v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
     if(LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imageVersion))

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java

@@ -81,6 +81,9 @@ abstract class ImageVisitor {
     DELEGATION_TOKEN_IDENTIFIER_EXPIRY_TIME,
     DELEGATION_TOKEN_IDENTIFIER_MASTER_KEY_ID,
     TRANSACTION_ID,
+    LAST_INODE_ID,
+    INODE_ID,
+
     SNAPSHOT_COUNTER,
     NUM_SNAPSHOTS_TOTAL,
     NUM_SNAPSHOTTABLE_DIRS,

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/GSet.java

@@ -81,4 +81,6 @@ public interface GSet<K, E extends K> extends Iterable<E> {
     * @throws NullPointerException if key == null.
   */
   E remove(K key);
+
+  void clear();
 }

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/GSetByHashMap.java

@@ -65,4 +65,9 @@ public class GSetByHashMap<K, E extends K> implements GSet<K, E> {
   public Iterator<E> iterator() {
     return m.values().iterator();
   }
+  
+  @Override
+  public void clear() {
+    m.clear();
+  }
 }

+ 12 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightGSet.java

@@ -85,7 +85,6 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
     if (LOG.isDebugEnabled()) {
       LOG.debug("recommended=" + recommended_length + ", actual=" + actual);
     }
-
     entries = new LinkedElement[actual];
     hash_mask = entries.length - 1;
   }
@@ -329,13 +328,18 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
     final int exponent = e2 < 0? 0: e2 > 30? 30: e2;
     final int c = 1 << exponent;
 
-    if (LightWeightGSet.LOG.isDebugEnabled()) {
-      LOG.debug("Computing capacity for map " + mapName);
-      LOG.debug("VM type       = " + vmBit + "-bit");
-      LOG.debug(percentage + "% max memory = "
-          + StringUtils.TraditionalBinaryPrefix.long2String(maxMemory, "B", 1));
-      LOG.debug("capacity      = 2^" + exponent + " = " + c + " entries");
-    }
+    LOG.info("Computing capacity for map " + mapName);
+    LOG.info("VM type       = " + vmBit + "-bit");
+    LOG.info(percentage + "% max memory = "
+        + StringUtils.TraditionalBinaryPrefix.long2String(maxMemory, "B", 1));
+    LOG.info("capacity      = 2^" + exponent + " = " + c + " entries");
     return c;
   }
+  
+  public void clear() {
+    for (int i = 0; i < entries.length; i++) {
+      entries[i] = null;
+    }
+    size = 0;
+  }
 }

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

@@ -347,6 +347,14 @@ message GetFileInfoResponseProto {
   optional HdfsFileStatusProto fs = 1;
 }
 
+message IsFileClosedRequestProto {
+  required string src = 1;
+}
+
+message IsFileClosedResponseProto {
+  required bool result = 1;
+}
+
 message GetFileLinkInfoRequestProto {
   required string src = 1;
 }
@@ -566,4 +574,6 @@ service ClientNamenodeProtocol {
       returns(DeleteSnapshotResponseProto);
   rpc getSnapshotDiffReport(GetSnapshotDiffReportRequestProto)
       returns(GetSnapshotDiffReportResponseProto);
+  rpc isFileClosed(IsFileClosedRequestProto)
+      returns(IsFileClosedResponseProto);
 }

+ 28 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -1231,4 +1231,32 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.datanode.fsdataset.volume.choosing.balanced-space-threshold</name>
+  <value>10737418240</value> <!-- 10 GB -->
+  <description>
+    Only used when the dfs.datanode.fsdataset.volume.choosing.policy is set to
+    org.apache.hadoop.hdfs.server.datanode.fsdataset.AvailableSpaceVolumeChoosingPolicy.
+    This setting controls how much DN volumes are allowed to differ in terms of
+    bytes of free disk space before they are considered imbalanced. If the free
+    space of all the volumes are within this range of each other, the volumes
+    will be considered balanced and block assignments will be done on a pure
+    round robin basis.
+  </description>
+</property>
+
+<property>
+  <name>dfs.datanode.fsdataset.volume.choosing.balanced-space-preference-percent</name>
+  <value>0.75f</value>
+  <description>
+    Only used when the dfs.datanode.fsdataset.volume.choosing.policy is set to
+    org.apache.hadoop.hdfs.server.datanode.fsdataset.AvailableSpaceVolumeChoosingPolicy.
+    This setting controls what percentage of new block allocations will be sent
+    to volumes with more available disk space than others. This setting should
+    be in the range 0.0 - 1.0, though in practice 0.5 - 1.0, since there should
+    be no reason to prefer that volumes with less available disk space receive
+    more block allocations.
+  </description>
+</property>
+
 </configuration>

+ 23 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -762,4 +762,27 @@ public class TestDistributedFileSystem {
       }
     }
   }
+
+  @Test(timeout=60000)
+  public void testFileCloseStatus() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    DistributedFileSystem fs = cluster.getFileSystem();
+    try {
+      // create a new file.
+      Path file = new Path("/simpleFlush.dat");
+      FSDataOutputStream output = fs.create(file);
+      // write to file
+      output.writeBytes("Some test data");
+      output.flush();
+      assertFalse("File status should be open", fs.isFileClosed(file));
+      output.close();
+      assertTrue("File status should be closed", fs.isFileClosed(file));
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+  
 }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSFileSystemContract.java

@@ -46,6 +46,7 @@ public class TestHDFSFileSystemContract extends FileSystemContractBaseTest {
   protected void tearDown() throws Exception {
     super.tearDown();
     cluster.shutdown();
+    cluster = null;
   }
 
   @Override

+ 35 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetTimes.java

@@ -27,6 +27,7 @@ import java.net.InetSocketAddress;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.Random;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -36,8 +37,11 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.test.MockitoUtil;
 import org.apache.hadoop.util.Time;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 /**
  * This class tests the access time on files.
@@ -273,6 +277,37 @@ public class TestSetTimes {
       cluster.shutdown();
     }
   }
+  
+  /**
+   * Test that when access time updates are not needed, the FSNamesystem
+   * write lock is not taken by getBlockLocations.
+   * Regression test for HDFS-3981.
+   */
+  @Test(timeout=60000)
+  public void testGetBlockLocationsOnlyUsesReadLock() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 100*1000);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .numDataNodes(0)
+      .build();
+    ReentrantReadWriteLock spyLock = NameNodeAdapter.spyOnFsLock(cluster.getNamesystem());
+    try {
+      // Create empty file in the FSN.
+      Path p = new Path("/empty-file");
+      DFSTestUtil.createFile(cluster.getFileSystem(), p, 0, (short)1, 0L);
+      
+      // getBlockLocations() should not need the write lock, since we just created
+      // the file (and thus its access time is already within the 100-second
+      // accesstime precision configured above). 
+      MockitoUtil.doThrowWhenCallStackMatches(
+          new AssertionError("Should not need write lock"),
+          ".*getBlockLocations.*")
+          .when(spyLock).writeLock();
+      cluster.getFileSystem().getFileBlockLocations(p, 0, 100);
+    } finally {
+      cluster.shutdown();
+    }
+  }
 
   public static void main(String[] args) throws Exception {
     new TestSetTimes().testTimes();

+ 26 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java

@@ -74,6 +74,8 @@ public class TestQuorumJournalManager {
   private Configuration conf;
   private QuorumJournalManager qjm;
   private List<AsyncLogger> spies;
+
+  private List<QuorumJournalManager> toClose = Lists.newLinkedList();
   
   static {
     ((Log4JLogger)ProtobufRpcEngine.LOG).getLogger().setLevel(Level.ALL);
@@ -98,11 +100,26 @@ public class TestQuorumJournalManager {
   
   @After
   public void shutdown() throws IOException {
+    IOUtils.cleanup(LOG, toClose.toArray(new Closeable[0]));
+    
+    // Should not leak clients between tests -- this can cause flaky tests.
+    // (See HDFS-4643)
+    GenericTestUtils.assertNoThreadsMatching(".*IPC Client.*");
+    
     if (cluster != null) {
       cluster.shutdown();
     }
   }
   
+  /**
+   * Enqueue a QJM for closing during shutdown. This makes the code a little
+   * easier to follow, with fewer try..finally clauses necessary.
+   */
+  private QuorumJournalManager closeLater(QuorumJournalManager qjm) {
+    toClose.add(qjm);
+    return qjm;
+  }
+  
   @Test
   public void testSingleWriter() throws Exception {
     writeSegment(cluster, qjm, 1, 3, true);
@@ -119,8 +136,8 @@ public class TestQuorumJournalManager {
   
   @Test
   public void testFormat() throws Exception {
-    QuorumJournalManager qjm = new QuorumJournalManager(
-        conf, cluster.getQuorumJournalURI("testFormat-jid"), FAKE_NSINFO);
+    QuorumJournalManager qjm = closeLater(new QuorumJournalManager(
+        conf, cluster.getQuorumJournalURI("testFormat-jid"), FAKE_NSINFO));
     assertFalse(qjm.hasSomeData());
     qjm.format(FAKE_NSINFO);
     assertTrue(qjm.hasSomeData());
@@ -128,8 +145,7 @@ public class TestQuorumJournalManager {
   
   @Test
   public void testReaderWhileAnotherWrites() throws Exception {
-    
-    QuorumJournalManager readerQjm = createSpyingQJM();
+    QuorumJournalManager readerQjm = closeLater(createSpyingQJM());
     List<EditLogInputStream> streams = Lists.newArrayList();
     readerQjm.selectInputStreams(streams, 0, false);
     assertEquals(0, streams.size());
@@ -251,8 +267,8 @@ public class TestQuorumJournalManager {
     
     
     // Make a new QJM
-    qjm = new QuorumJournalManager(
-        conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO);
+    qjm = closeLater(new QuorumJournalManager(
+        conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO));
     qjm.recoverUnfinalizedSegments();
     checkRecovery(cluster, 1, 3);
 
@@ -364,8 +380,8 @@ public class TestQuorumJournalManager {
         NNStorage.getInProgressEditsFileName(1));
 
     // Make a new QJM
-    qjm = new QuorumJournalManager(
-        conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO);
+    qjm = closeLater(new QuorumJournalManager(
+        conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO));
     qjm.recoverUnfinalizedSegments();
     checkRecovery(cluster, 1, 3);
   }
@@ -902,8 +918,8 @@ public class TestQuorumJournalManager {
         return Mockito.spy(logger);
       }
     };
-    return new QuorumJournalManager(
-        conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO, spyFactory);
+    return closeLater(new QuorumJournalManager(
+        conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO, spyFactory));
   }
 
   private static void waitForAllPendingCalls(AsyncLoggerSet als)

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java

@@ -17,10 +17,12 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 
+import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.Map;
@@ -68,6 +70,8 @@ public class TestBPOfferService {
       TestBPOfferService.class);
   private static final ExtendedBlock FAKE_BLOCK =
     new ExtendedBlock(FAKE_BPID, 12345L);
+  private static final String TEST_BUILD_DATA = System.getProperty(
+    "test.build.data", "build/test/data");
 
   static {
     ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);
@@ -90,6 +94,8 @@ public class TestBPOfferService {
     mockDn = Mockito.mock(DataNode.class);
     Mockito.doReturn(true).when(mockDn).shouldRun();
     Configuration conf = new Configuration();
+    File dnDataDir = new File(new File(TEST_BUILD_DATA, "dfs"), "data");
+    conf.set(DFS_DATANODE_DATA_DIR_KEY, dnDataDir.toURI().toString());
     Mockito.doReturn(conf).when(mockDn).getConf();
     Mockito.doReturn(new DNConf(conf)).when(mockDn).getDnConf();
     Mockito.doReturn(DataNodeMetrics.create(conf, "fake dn"))

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolManager.java

@@ -113,7 +113,7 @@ public class TestBlockPoolManager {
 
     // Remove the first NS
     conf.set(DFSConfigKeys.DFS_NAMESERVICES,
-        "ns1");
+        "ns2");
     bpm.refreshNamenodes(conf);
     assertEquals(
         "stop #1\n" +

+ 303 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/TestAvailableSpaceVolumeChoosingPolicy.java

@@ -0,0 +1,303 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.fsdataset;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_KEY;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestAvailableSpaceVolumeChoosingPolicy {
+  
+  private static final int RANDOMIZED_ITERATIONS = 10000;
+  private static final float RANDOMIZED_ERROR_PERCENT = 0.05f;
+  private static final long RANDOMIZED_ALLOWED_ERROR = (long) (RANDOMIZED_ERROR_PERCENT * RANDOMIZED_ITERATIONS);
+  
+  private static void initPolicy(VolumeChoosingPolicy<FsVolumeSpi> policy,
+      float preferencePercent) {
+    Configuration conf = new Configuration();
+    // Set the threshold to consider volumes imbalanced to 1MB
+    conf.setLong(
+        DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_KEY,
+        1024 * 1024); // 1MB
+    conf.setFloat(
+        DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_KEY,
+        preferencePercent);
+    ((Configurable) policy).setConf(conf);
+  }
+  
+  // Test the Round-Robin block-volume fallback path when all volumes are within
+  // the threshold.
+  @Test(timeout=60000)
+  public void testRR() throws Exception {
+    @SuppressWarnings("unchecked")
+    final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy = 
+        ReflectionUtils.newInstance(AvailableSpaceVolumeChoosingPolicy.class, null);
+    initPolicy(policy, 1.0f);
+    TestRoundRobinVolumeChoosingPolicy.testRR(policy);
+  }
+  
+  // ChooseVolume should throw DiskOutOfSpaceException
+  // with volume and block sizes in exception message.
+  @Test(timeout=60000)
+  public void testRRPolicyExceptionMessage() throws Exception {
+    final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy
+        = new AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi>();
+    initPolicy(policy, 1.0f);
+    TestRoundRobinVolumeChoosingPolicy.testRRPolicyExceptionMessage(policy);
+  }
+  
+  @Test(timeout=60000)
+  public void testTwoUnbalancedVolumes() throws Exception {
+    @SuppressWarnings("unchecked")
+    final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy = 
+        ReflectionUtils.newInstance(AvailableSpaceVolumeChoosingPolicy.class, null);
+    initPolicy(policy, 1.0f);
+    
+    List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
+    
+    // First volume with 1MB free space
+    volumes.add(Mockito.mock(FsVolumeSpi.class));
+    Mockito.when(volumes.get(0).getAvailable()).thenReturn(1024L * 1024L);
+    
+    // Second volume with 3MB free space, which is a difference of 2MB, more
+    // than the threshold of 1MB.
+    volumes.add(Mockito.mock(FsVolumeSpi.class));
+    Mockito.when(volumes.get(1).getAvailable()).thenReturn(1024L * 1024L * 3);
+    
+    Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
+    Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
+    Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
+  }
+  
+  @Test(timeout=60000)
+  public void testThreeUnbalancedVolumes() throws Exception {
+    @SuppressWarnings("unchecked")
+    final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy = 
+        ReflectionUtils.newInstance(AvailableSpaceVolumeChoosingPolicy.class, null);
+    
+    List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
+    
+    // First volume with 1MB free space
+    volumes.add(Mockito.mock(FsVolumeSpi.class));
+    Mockito.when(volumes.get(0).getAvailable()).thenReturn(1024L * 1024L);
+    
+    // Second volume with 3MB free space, which is a difference of 2MB, more
+    // than the threshold of 1MB.
+    volumes.add(Mockito.mock(FsVolumeSpi.class));
+    Mockito.when(volumes.get(1).getAvailable()).thenReturn(1024L * 1024L * 3);
+    
+    // Third volume, again with 3MB free space.
+    volumes.add(Mockito.mock(FsVolumeSpi.class));
+    Mockito.when(volumes.get(2).getAvailable()).thenReturn(1024L * 1024L * 3);
+    
+    // We should alternate assigning between the two volumes with a lot of free
+    // space.
+    initPolicy(policy, 1.0f);
+    Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
+    Assert.assertEquals(volumes.get(2), policy.chooseVolume(volumes, 100));
+    Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
+    Assert.assertEquals(volumes.get(2), policy.chooseVolume(volumes, 100));
+
+    // All writes should be assigned to the volume with the least free space.
+    initPolicy(policy, 0.0f);
+    Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 100));
+    Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 100));
+    Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 100));
+    Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 100));
+  }
+  
+  @Test(timeout=60000)
+  public void testFourUnbalancedVolumes() throws Exception {
+    @SuppressWarnings("unchecked")
+    final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy = 
+        ReflectionUtils.newInstance(AvailableSpaceVolumeChoosingPolicy.class, null);
+    
+    List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
+    
+    // First volume with 1MB free space
+    volumes.add(Mockito.mock(FsVolumeSpi.class));
+    Mockito.when(volumes.get(0).getAvailable()).thenReturn(1024L * 1024L);
+    
+    // Second volume with 1MB + 1 byte free space
+    volumes.add(Mockito.mock(FsVolumeSpi.class));
+    Mockito.when(volumes.get(1).getAvailable()).thenReturn(1024L * 1024L + 1);
+    
+    // Third volume with 3MB free space, which is a difference of 2MB, more
+    // than the threshold of 1MB.
+    volumes.add(Mockito.mock(FsVolumeSpi.class));
+    Mockito.when(volumes.get(2).getAvailable()).thenReturn(1024L * 1024L * 3);
+    
+    // Fourth volume, again with 3MB free space.
+    volumes.add(Mockito.mock(FsVolumeSpi.class));
+    Mockito.when(volumes.get(3).getAvailable()).thenReturn(1024L * 1024L * 3);
+    
+    // We should alternate assigning between the two volumes with a lot of free
+    // space.
+    initPolicy(policy, 1.0f);
+    Assert.assertEquals(volumes.get(2), policy.chooseVolume(volumes, 100));
+    Assert.assertEquals(volumes.get(3), policy.chooseVolume(volumes, 100));
+    Assert.assertEquals(volumes.get(2), policy.chooseVolume(volumes, 100));
+    Assert.assertEquals(volumes.get(3), policy.chooseVolume(volumes, 100));
+
+    // We should alternate assigning between the two volumes with less free
+    // space.
+    initPolicy(policy, 0.0f);
+    Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 100));
+    Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
+    Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 100));
+    Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
+  }
+  
+  @Test(timeout=60000)
+  public void testNotEnoughSpaceOnSelectedVolume() throws Exception {
+    @SuppressWarnings("unchecked")
+    final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy = 
+        ReflectionUtils.newInstance(AvailableSpaceVolumeChoosingPolicy.class, null);
+    
+    List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
+    
+    // First volume with 1MB free space
+    volumes.add(Mockito.mock(FsVolumeSpi.class));
+    Mockito.when(volumes.get(0).getAvailable()).thenReturn(1024L * 1024L);
+    
+    // Second volume with 3MB free space, which is a difference of 2MB, more
+    // than the threshold of 1MB.
+    volumes.add(Mockito.mock(FsVolumeSpi.class));
+    Mockito.when(volumes.get(1).getAvailable()).thenReturn(1024L * 1024L * 3);
+    
+    // All writes should be assigned to the volume with the least free space.
+    // However, if the volume with the least free space doesn't have enough
+    // space to accept the replica size, and another volume does have enough
+    // free space, that should be chosen instead.
+    initPolicy(policy, 0.0f);
+    Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 1024L * 1024L * 2));
+  }
+  
+  @Test(timeout=60000)
+  public void testAvailableSpaceChanges() throws Exception {
+    @SuppressWarnings("unchecked")
+    final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy = 
+        ReflectionUtils.newInstance(AvailableSpaceVolumeChoosingPolicy.class, null);
+    initPolicy(policy, 1.0f);
+    
+    List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
+    
+    // First volume with 1MB free space
+    volumes.add(Mockito.mock(FsVolumeSpi.class));
+    Mockito.when(volumes.get(0).getAvailable()).thenReturn(1024L * 1024L);
+    
+    // Second volume with 3MB free space, which is a difference of 2MB, more
+    // than the threshold of 1MB.
+    volumes.add(Mockito.mock(FsVolumeSpi.class));
+    Mockito.when(volumes.get(1).getAvailable())
+        .thenReturn(1024L * 1024L * 3)
+        .thenReturn(1024L * 1024L * 3)
+        .thenReturn(1024L * 1024L * 3)
+        .thenReturn(1024L * 1024L * 1); // After the third check, return 1MB.
+    
+    // Should still be able to get a volume for the replica even though the
+    // available space on the second volume changed.
+    Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
+  }
+  
+  @Test(timeout=60000)
+  public void randomizedTest1() throws Exception {
+    doRandomizedTest(0.75f, 1, 1);
+  }
+  
+  @Test(timeout=60000)
+  public void randomizedTest2() throws Exception {
+    doRandomizedTest(0.75f, 5, 1);
+  }
+  
+  @Test(timeout=60000)
+  public void randomizedTest3() throws Exception {
+    doRandomizedTest(0.75f, 1, 5);
+  }
+  
+  @Test(timeout=60000)
+  public void randomizedTest4() throws Exception {
+    doRandomizedTest(0.90f, 5, 1);
+  }
+  
+  /*
+   * Ensure that we randomly select the lesser-used volumes with appropriate
+   * frequency.
+   */
+  public void doRandomizedTest(float preferencePercent, int lowSpaceVolumes,
+      int highSpaceVolumes) throws Exception {
+    @SuppressWarnings("unchecked")
+    final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy = 
+        ReflectionUtils.newInstance(AvailableSpaceVolumeChoosingPolicy.class, null);
+    
+    List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
+    
+    // Volumes with 1MB free space
+    for (int i = 0; i < lowSpaceVolumes; i++) {
+      FsVolumeSpi volume = Mockito.mock(FsVolumeSpi.class);
+      Mockito.when(volume.getAvailable()).thenReturn(1024L * 1024L);
+      volumes.add(volume);
+    }
+    
+    // Volumes with 3MB free space
+    for (int i = 0; i < highSpaceVolumes; i++) {
+      FsVolumeSpi volume = Mockito.mock(FsVolumeSpi.class);
+      Mockito.when(volume.getAvailable()).thenReturn(1024L * 1024L * 3);
+      volumes.add(volume);
+    }
+    
+    initPolicy(policy, preferencePercent);
+    long lowAvailableSpaceVolumeSelected = 0;
+    long highAvailableSpaceVolumeSelected = 0;
+    for (int i = 0; i < RANDOMIZED_ITERATIONS; i++) {
+      FsVolumeSpi volume = policy.chooseVolume(volumes, 100);
+      for (int j = 0; j < volumes.size(); j++) {
+        // Note how many times the first low available volume was selected
+        if (volume == volumes.get(j) && j == 0) {
+          lowAvailableSpaceVolumeSelected++;
+        }
+        // Note how many times the first high available volume was selected
+        if (volume == volumes.get(j) && j == lowSpaceVolumes) {
+          highAvailableSpaceVolumeSelected++;
+          break;
+        }
+      }
+    }
+    
+    // Calculate the expected ratio of how often low available space volumes
+    // were selected vs. high available space volumes.
+    float expectedSelectionRatio = preferencePercent / (1 - preferencePercent);
+    
+    GenericTestUtils.assertValueNear(
+        (long)(lowAvailableSpaceVolumeSelected * expectedSelectionRatio),
+        highAvailableSpaceVolumeSelected,
+        RANDOMIZED_ALLOWED_ERROR);
+  }
+  
+}

+ 15 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/TestRoundRobinVolumeChoosingPolicy.java

@@ -32,6 +32,14 @@ public class TestRoundRobinVolumeChoosingPolicy {
   // Test the Round-Robin block-volume choosing algorithm.
   @Test
   public void testRR() throws Exception {
+    @SuppressWarnings("unchecked")
+    final RoundRobinVolumeChoosingPolicy<FsVolumeSpi> policy = 
+        ReflectionUtils.newInstance(RoundRobinVolumeChoosingPolicy.class, null);
+    testRR(policy);
+  }
+  
+  public static void testRR(VolumeChoosingPolicy<FsVolumeSpi> policy)
+      throws Exception {
     final List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
 
     // First volume, with 100 bytes of space.
@@ -41,10 +49,6 @@ public class TestRoundRobinVolumeChoosingPolicy {
     // Second volume, with 200 bytes of space.
     volumes.add(Mockito.mock(FsVolumeSpi.class));
     Mockito.when(volumes.get(1).getAvailable()).thenReturn(200L);
-
-    @SuppressWarnings("unchecked")
-    final RoundRobinVolumeChoosingPolicy<FsVolumeSpi> policy = 
-        ReflectionUtils.newInstance(RoundRobinVolumeChoosingPolicy.class, null);
     
     // Test two rounds of round-robin choosing
     Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 0));
@@ -69,6 +73,13 @@ public class TestRoundRobinVolumeChoosingPolicy {
   // with volume and block sizes in exception message.
   @Test
   public void testRRPolicyExceptionMessage() throws Exception {
+    final RoundRobinVolumeChoosingPolicy<FsVolumeSpi> policy
+        = new RoundRobinVolumeChoosingPolicy<FsVolumeSpi>();
+    testRRPolicyExceptionMessage(policy);
+  }
+  
+  public static void testRRPolicyExceptionMessage(
+      VolumeChoosingPolicy<FsVolumeSpi> policy) throws Exception {
     final List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
 
     // First volume, with 500 bytes of space.
@@ -79,8 +90,6 @@ public class TestRoundRobinVolumeChoosingPolicy {
     volumes.add(Mockito.mock(FsVolumeSpi.class));
     Mockito.when(volumes.get(1).getAvailable()).thenReturn(600L);
 
-    final RoundRobinVolumeChoosingPolicy<FsVolumeSpi> policy
-        = new RoundRobinVolumeChoosingPolicy<FsVolumeSpi>();
     int blockSize = 700;
     try {
       policy.chooseVolume(volumes, blockSize);

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

@@ -218,7 +218,7 @@ public abstract class FSImageTestUtil {
         FsPermission.createImmutable((short)0755));
     for (int i = 1; i <= numDirs; i++) {
       String dirName = "dir" + i;
-      INodeDirectory dir = new INodeDirectory(newInodeId + i -1,
+      INodeDirectory dir = new INodeDirectory(newInodeId + i - 1,
           DFSUtil.string2Bytes(dirName), perms, 0L);
       editLog.logMkDir("/" + dirName, dir);
     }

+ 1 - 14
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -30,8 +30,6 @@ import static org.junit.Assert.fail;
 import java.io.File;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
-import java.lang.management.ThreadInfo;
-import java.lang.management.ThreadMXBean;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.util.ArrayList;
@@ -122,18 +120,7 @@ public class TestCheckpoint {
   
   @After
   public void checkForSNNThreads() {
-    ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
-    
-    ThreadInfo[] infos = threadBean.getThreadInfo(threadBean.getAllThreadIds(), 20);
-    for (ThreadInfo info : infos) {
-      if (info == null) continue;
-      LOG.info("Check thread: " + info.getThreadName());
-      if (info.getThreadName().contains("SecondaryNameNode")) {
-        fail("Leaked thread: " + info + "\n" +
-            Joiner.on("\n").join(info.getStackTrace()));
-      }
-    }
-    LOG.info("--------");
+    GenericTestUtils.assertNoThreadsMatching(".*SecondaryNameNode.*");
   }
   
   static void checkFile(FileSystem fileSys, Path name, int repl)

+ 29 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -24,10 +24,8 @@ import static org.junit.Assert.fail;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.EnumSet;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Options;
@@ -40,10 +38,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
-import org.apache.hadoop.io.EnumSetWritable;
 import org.junit.Test;
 
 public class TestINodeFile {
@@ -366,9 +361,7 @@ public class TestINodeFile {
       cluster.waitActive();
 
       FSNamesystem fsn = cluster.getNamesystem();
-      long lastId = fsn.getLastInodeId();
-
-      assertTrue(lastId == 1001);
+      assertTrue(fsn.getLastInodeId() == 1001);
 
       // Create one directory and the last inode id should increase to 1002
       FileSystem fs = cluster.getFileSystem();
@@ -376,14 +369,10 @@ public class TestINodeFile {
       assertTrue(fs.mkdirs(path));
       assertTrue(fsn.getLastInodeId() == 1002);
 
-      // Use namenode rpc to create a file
-      NamenodeProtocols nnrpc = cluster.getNameNodeRpc();
-      HdfsFileStatus fileStatus = nnrpc.create("/test1/file", new FsPermission(
-          (short) 0755), "client",
-          new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE)), true,
-          (short) 1, 128 * 1024 * 1024L);
+      int fileLen = 1024;
+      Path filePath = new Path("/test1/file");
+      DFSTestUtil.createFile(fs, filePath, fileLen, (short) 1, 0);
       assertTrue(fsn.getLastInodeId() == 1003);
-      assertTrue(fileStatus.getFileId() == 1003);
 
       // Rename doesn't increase inode id
       Path renamedPath = new Path("/test2");
@@ -395,7 +384,32 @@ public class TestINodeFile {
       // Make sure empty editlog can be handled
       cluster.restartNameNode();
       cluster.waitActive();
+      fsn = cluster.getNamesystem();
       assertTrue(fsn.getLastInodeId() == 1003);
+
+      DFSTestUtil.createFile(fs, new Path("/test2/file2"), fileLen, (short) 1,
+          0);
+      long id = fsn.getLastInodeId();
+      assertTrue(id == 1004);
+      fs.delete(new Path("/test2"), true);
+      // create a file under construction
+      FSDataOutputStream outStream = fs.create(new Path("/test3/file"));
+      assertTrue(outStream != null);
+      assertTrue(fsn.getLastInodeId() == 1006);
+
+      // Apply editlogs to fsimage, test fsimage with inodeUnderConstruction can
+      // be handled
+      fsn.enterSafeMode(false);
+      fsn.saveNamespace();
+      fsn.leaveSafeMode();
+
+      outStream.close();
+
+      // The lastInodeId in fsimage should remain 1006 after reboot
+      cluster.restartNameNode();
+      cluster.waitActive();
+      fsn = cluster.getNamesystem();
+      assertTrue(fsn.getLastInodeId() == 1006);
     } finally {
       if (cluster != null) {
         cluster.shutdown();

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestGSet.java

@@ -388,6 +388,11 @@ public class TestGSet {
       return String.format(" iterate=%5d, contain=%5d, time elapsed=%5d.%03ds",
           iterate_count, contain_count, t/1000, t%1000);
     }
+
+    @Override
+    public void clear() {
+      gset.clear();
+    }
   }
 
   /** Test data set */

BIN
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored


+ 15 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml

@@ -1,6 +1,6 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <EDITS>
-  <EDITS_VERSION>-41</EDITS_VERSION>
+  <EDITS_VERSION>-42</EDITS_VERSION>
   <RECORD>
     <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <DATA>
@@ -41,6 +41,7 @@
     <DATA>
       <TXID>5</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1002</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709370</MTIME>
@@ -60,6 +61,7 @@
     <DATA>
       <TXID>6</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709398</MTIME>
@@ -98,6 +100,7 @@
     <DATA>
       <TXID>9</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1003</INODEID>
       <PATH>/directory_mkdir</PATH>
       <TIMESTAMP>1357889709598</TIMESTAMP>
       <PERMISSION_STATUS>
@@ -165,6 +168,7 @@
     <DATA>
       <TXID>17</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1004</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709615</MTIME>
@@ -184,6 +188,7 @@
     <DATA>
       <TXID>18</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709616</MTIME>
@@ -264,6 +269,7 @@
     <DATA>
       <TXID>26</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1005</INODEID>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709631</MTIME>
@@ -355,6 +361,7 @@
     <DATA>
       <TXID>33</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709689</MTIME>
@@ -396,6 +403,7 @@
     <DATA>
       <TXID>35</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1006</INODEID>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709692</MTIME>
@@ -487,6 +495,7 @@
     <DATA>
       <TXID>42</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709710</MTIME>
@@ -528,6 +537,7 @@
     <DATA>
       <TXID>44</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1007</INODEID>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709713</MTIME>
@@ -619,6 +629,7 @@
     <DATA>
       <TXID>51</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709744</MTIME>
@@ -666,6 +677,7 @@
     <DATA>
       <TXID>53</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1008</INODEID>
       <PATH>/file_symlink</PATH>
       <VALUE>/file_concat_target</VALUE>
       <MTIME>1357889709751</MTIME>
@@ -739,6 +751,7 @@
     <DATA>
       <TXID>58</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>1009</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709825</MTIME>
@@ -805,6 +818,7 @@
     <DATA>
       <TXID>64</TXID>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889712257</MTIME>

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

@@ -166,6 +166,9 @@ Release 2.0.5-alpha - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-5129. Allow tags to JobHistory for deeper analytics. (billie via
+    acmurthy)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -222,6 +225,9 @@ Release 2.0.4-beta - UNRELEASED
     MAPREDUCE-5088. MR Client gets an renewer token exception while Oozie is
     submitting a job (Daryn Sharp via cos)
 
+    MAPREDUCE-5138. Fix LocalDistributedCacheManager after YARN-112. (Omkar Vinit
+    Joshi via vinodkv)
+
 Release 2.0.3-alpha - 2013-02-06 
 
   INCOMPATIBLE CHANGES
@@ -773,6 +779,9 @@ Release 0.23.7 - UNRELEASED
 
     MAPREDUCE-4991. coverage for gridmix (Aleksey Gorshkov via tgraves)
 
+    MAPREDUCE-5007. fix coverage org.apache.hadoop.mapreduce.v2.hs (Aleksey 
+    Gorshkov via tgraves)
+
 Release 0.23.6 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

@@ -1305,7 +1305,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
             job.conf.get(MRJobConfig.WORKFLOW_ID, ""),
             job.conf.get(MRJobConfig.WORKFLOW_NAME, ""),
             job.conf.get(MRJobConfig.WORKFLOW_NODE_NAME, ""),
-            getWorkflowAdjacencies(job.conf));
+            getWorkflowAdjacencies(job.conf),
+            job.conf.get(MRJobConfig.WORKFLOW_TAGS, ""));
         job.eventHandler.handle(new JobHistoryEvent(job.jobId, jse));
         //TODO JH Verify jobACLs, UserName via UGI?
 

+ 12 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java

@@ -114,6 +114,7 @@ public class TestJobImpl {
     conf.set(MRJobConfig.WORKFLOW_NODE_NAME, "testNodeName");
     conf.set(MRJobConfig.WORKFLOW_ADJACENCY_PREFIX_STRING + "key1", "value1");
     conf.set(MRJobConfig.WORKFLOW_ADJACENCY_PREFIX_STRING + "key2", "value2");
+    conf.set(MRJobConfig.WORKFLOW_TAGS, "tag1,tag2");
     
  
     AsyncDispatcher dispatcher = new AsyncDispatcher();
@@ -126,7 +127,8 @@ public class TestJobImpl {
     commitHandler.start();
 
     JobSubmittedEventHandler jseHandler = new JobSubmittedEventHandler("testId",
-        "testName", "testNodeName", "\"key2\"=\"value2\" \"key1\"=\"value1\" ");
+        "testName", "testNodeName", "\"key2\"=\"value2\" \"key1\"=\"value1\" ",
+        "tag1,tag2");
     dispatcher.register(EventType.class, jseHandler);
     JobImpl job = createStubbedJob(conf, dispatcher, 0);
     job.handle(new JobEvent(job.getID(), JobEventType.JOB_INIT));
@@ -706,14 +708,18 @@ public class TestJobImpl {
     
     private String workflowAdjacencies;
     
+    private String workflowTags;
+    
     private Boolean assertBoolean;
 
     public JobSubmittedEventHandler(String workflowId, String workflowName,
-        String workflowNodeName, String workflowAdjacencies) {
+        String workflowNodeName, String workflowAdjacencies,
+        String workflowTags) {
       this.workflowId = workflowId;
       this.workflowName = workflowName;
       this.workflowNodeName = workflowNodeName;
       this.workflowAdjacencies = workflowAdjacencies;
+      this.workflowTags = workflowTags;
       assertBoolean = null;
     }
 
@@ -739,6 +745,10 @@ public class TestJobImpl {
         setAssertValue(false);
         return;
       }
+      if (!workflowTags.equals(jsEvent.getWorkflowTags())) {
+        setAssertValue(false);
+        return;
+      }
       setAssertValue(true);
     }
     

+ 8 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalDistributedCacheManager.java

@@ -32,13 +32,13 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Random;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -91,6 +91,9 @@ class LocalDistributedCacheManager {
     Map<String, LocalResource> localResources = 
       new LinkedHashMap<String, LocalResource>();
     MRApps.setupDistributedCache(conf, localResources);
+    // Generating unique numbers for FSDownload.
+    AtomicLong uniqueNumberGenerator =
+        new AtomicLong(System.currentTimeMillis());
     
     // Find which resources are to be put on the local classpath
     Map<String, Path> classpaths = new HashMap<String, Path>();
@@ -128,8 +131,10 @@ class LocalDistributedCacheManager {
       Path destPath = localDirAllocator.getLocalPathForWrite(".", conf);
       Map<LocalResource, Future<Path>> resourcesToPaths = Maps.newHashMap();
       for (LocalResource resource : localResources.values()) {
-        Callable<Path> download = new FSDownload(localFSFileContext, ugi, conf,
-            destPath, resource, new Random());
+        Callable<Path> download =
+            new FSDownload(localFSFileContext, ugi, conf, new Path(destPath,
+                Long.toString(uniqueNumberGenerator.incrementAndGet())),
+                resource);
         Future<Path> future = exec.submit(download);
         resourcesToPaths.put(resource, future);
       }

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/avro/Events.avpr

@@ -95,7 +95,8 @@
           {"name": "workflowId", "type": "string"},
           {"name": "workflowName", "type": "string"},
           {"name": "workflowNodeName", "type": "string"},
-          {"name": "workflowAdjacencies", "type": "string"}
+          {"name": "workflowAdjacencies", "type": "string"},
+          {"name": "workflowTags", "type": "string"}
       ]
      },
 

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

@@ -664,6 +664,8 @@ public interface MRJobConfig {
   public static final String WORKFLOW_ADJACENCY_PREFIX_PATTERN =
       "^mapreduce\\.workflow\\.adjacency\\..+";
 
+  public static final String WORKFLOW_TAGS = "mapreduce.workflow.tags";
+
   /**
    * The maximum number of application attempts.
    * It is a application-specific setting.

+ 35 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java

@@ -75,6 +75,31 @@ public class JobSubmittedEvent implements HistoryEvent {
       Map<JobACL, AccessControlList> jobACLs, String jobQueueName,
       String workflowId, String workflowName, String workflowNodeName,
       String workflowAdjacencies) {
+    this(id, jobName, userName, submitTime, jobConfPath, jobACLs,
+        jobQueueName, workflowId, workflowName, workflowNodeName,
+        workflowAdjacencies, "");
+  }
+
+    /**
+     * Create an event to record job submission
+     * @param id The job Id of the job
+     * @param jobName Name of the job
+     * @param userName Name of the user who submitted the job
+     * @param submitTime Time of submission
+     * @param jobConfPath Path of the Job Configuration file
+     * @param jobACLs The configured acls for the job.
+     * @param jobQueueName The job-queue to which this job was submitted to
+     * @param workflowId The Id of the workflow
+     * @param workflowName The name of the workflow
+     * @param workflowNodeName The node name of the workflow
+     * @param workflowAdjacencies The adjacencies of the workflow
+     * @param workflowTags Comma-separated tags for the workflow
+     */
+    public JobSubmittedEvent(JobID id, String jobName, String userName,
+        long submitTime, String jobConfPath,
+        Map<JobACL, AccessControlList> jobACLs, String jobQueueName,
+        String workflowId, String workflowName, String workflowNodeName,
+        String workflowAdjacencies, String workflowTags) {
     datum.jobid = new Utf8(id.toString());
     datum.jobName = new Utf8(jobName);
     datum.userName = new Utf8(userName);
@@ -101,6 +126,9 @@ public class JobSubmittedEvent implements HistoryEvent {
     if (workflowAdjacencies != null) {
       datum.workflowAdjacencies = new Utf8(workflowAdjacencies);
     }
+    if (workflowTags != null) {
+      datum.workflowTags = new Utf8(workflowTags);
+    }
   }
 
   JobSubmittedEvent() {}
@@ -168,6 +196,13 @@ public class JobSubmittedEvent implements HistoryEvent {
     }
     return null;
   }
+  /** Get the workflow tags */
+  public String getWorkflowTags() {
+    if (datum.workflowTags != null) {
+      return datum.workflowTags.toString();
+    }
+    return null;
+  }
   /** Get the event type */
   public EventType getEventType() { return EventType.JOB_SUBMITTED; }
 

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

@@ -869,4 +869,9 @@ public class HistoryFileManager extends AbstractService {
       }
     }
   }
+  // for test
+  @VisibleForTesting
+  void setMaxHistoryAge(long newValue){
+    maxHistoryAge=newValue;
+  } 
 }

+ 42 - 13
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestCompletedTask.java

@@ -21,46 +21,75 @@ package org.apache.hadoop.mapreduce.v2.hs;
 import java.util.Map;
 import java.util.TreeMap;
 
+import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
+import org.apache.hadoop.mapreduce.v2.api.records.Phase;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
 import org.apache.hadoop.mapreduce.v2.hs.CompletedTask;
-import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Mockito;
+import static org.mockito.Mockito.*;
+import static org.junit.Assert.*;
 
 public class TestCompletedTask{
 
-  @Test
+  @Test (timeout=5000)
   public void testTaskStartTimes() {
     
-    TaskId taskId = Mockito.mock(TaskId.class); 
-    TaskInfo taskInfo = Mockito.mock(TaskInfo.class);
+    TaskId taskId = mock(TaskId.class); 
+    TaskInfo taskInfo = mock(TaskInfo.class);
     Map<TaskAttemptID, TaskAttemptInfo> taskAttempts
       = new TreeMap<TaskAttemptID, TaskAttemptInfo>();
     
     TaskAttemptID id = new TaskAttemptID("0", 0, TaskType.MAP, 0, 0);
-    TaskAttemptInfo info = Mockito.mock(TaskAttemptInfo.class);
-    Mockito.when(info.getAttemptId()).thenReturn(id);
-    Mockito.when(info.getStartTime()).thenReturn(10l);
+    TaskAttemptInfo info = mock(TaskAttemptInfo.class);
+    when(info.getAttemptId()).thenReturn(id);
+    when(info.getStartTime()).thenReturn(10l);
     taskAttempts.put(id, info);
     
     id = new TaskAttemptID("1", 0, TaskType.MAP, 1, 1);
-    info = Mockito.mock(TaskAttemptInfo.class);
-    Mockito.when(info.getAttemptId()).thenReturn(id);
-    Mockito.when(info.getStartTime()).thenReturn(20l);
+    info = mock(TaskAttemptInfo.class);
+    when(info.getAttemptId()).thenReturn(id);
+    when(info.getStartTime()).thenReturn(20l);
     taskAttempts.put(id, info);
     
     
-    Mockito.when(taskInfo.getAllTaskAttempts()).thenReturn(taskAttempts);
+    when(taskInfo.getAllTaskAttempts()).thenReturn(taskAttempts);
     CompletedTask task = new CompletedTask(taskId, taskInfo);
     TaskReport report = task.getReport();
 
     // Make sure the startTime returned by report is the lesser of the 
     // attempy launch times
-    Assert.assertTrue(report.getStartTime() == 10);
+    assertTrue(report.getStartTime() == 10);
+  }
+  /**
+   * test some methods of CompletedTaskAttempt
+   */
+  @Test (timeout=5000)
+  public void testCompletedTaskAttempt(){
+    
+    TaskAttemptInfo attemptInfo= mock(TaskAttemptInfo.class);
+    when(attemptInfo.getRackname()).thenReturn("Rackname");
+    when(attemptInfo.getShuffleFinishTime()).thenReturn(11L);
+    when(attemptInfo.getSortFinishTime()).thenReturn(12L);
+    when(attemptInfo.getShufflePort()).thenReturn(10);
+    
+    JobID jobId= new JobID("12345",0);
+    TaskID taskId =new TaskID(jobId,TaskType.REDUCE, 0);
+    TaskAttemptID taskAttemptId= new TaskAttemptID(taskId, 0);
+    when(attemptInfo.getAttemptId()).thenReturn(taskAttemptId);
+    
+    
+    CompletedTaskAttempt taskAttemt= new CompletedTaskAttempt(null,attemptInfo);
+    assertEquals( "Rackname",   taskAttemt.getNodeRackName());
+    assertEquals( Phase.CLEANUP,   taskAttemt.getPhase());
+    assertTrue(  taskAttemt.isFinished());
+    assertEquals( 11L,   taskAttemt.getShuffleFinishTime());
+    assertEquals( 12L,   taskAttemt.getSortFinishTime());
+    assertEquals( 10,   taskAttemt.getShufflePort());
   }
 }

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

@@ -45,7 +45,9 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
 
+import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
 @RunWith(value = Parameterized.class)
@@ -79,7 +81,7 @@ public class TestJobHistoryEntities {
   }
 
   /* Verify some expected values based on the history file */
-  @Test (timeout=10000)
+  @Test (timeout=100000)
   public void testCompletedJob() throws Exception {
     HistoryFileInfo info = mock(HistoryFileInfo.class);
     when(info.getConfFile()).thenReturn(fullConfPath);
@@ -168,4 +170,45 @@ public class TestJobHistoryEntities {
     assertEquals(45454, rta1Report.getNodeManagerPort());
     assertEquals(9999, rta1Report.getNodeManagerHttpPort());
   }
+  /**
+   * Simple test of some methods of CompletedJob
+   * @throws Exception
+   */
+  @Test (timeout=30000)
+  public void testGetTaskAttemptCompletionEvent() throws Exception{
+    HistoryFileInfo info = mock(HistoryFileInfo.class);
+    when(info.getConfFile()).thenReturn(fullConfPath);
+    completedJob =
+      new CompletedJob(conf, jobId, fulleHistoryPath, loadTasks, "user",
+          info, jobAclsManager);
+    TaskCompletionEvent[] events= completedJob.getMapAttemptCompletionEvents(0,1000);
+    assertEquals(10, completedJob.getMapAttemptCompletionEvents(0,10).length);
+    int currentEventId=0;
+    for (TaskCompletionEvent taskAttemptCompletionEvent : events) {
+      int eventId= taskAttemptCompletionEvent.getEventId();
+      assertTrue(eventId>=currentEventId);
+      currentEventId=eventId;
+    }
+    assertNull(completedJob.loadConfFile() );
+    // job name
+    assertEquals("Sleep job",completedJob.getName());
+    // queue name
+    assertEquals("default",completedJob.getQueueName());
+    // progress
+    assertEquals(1.0, completedJob.getProgress(),0.001);
+    // 11 rows in answer
+    assertEquals(11,completedJob.getTaskAttemptCompletionEvents(0,1000).length);
+    // select first 10 rows
+    assertEquals(10,completedJob.getTaskAttemptCompletionEvents(0,10).length);
+    // select 5-10 rows include 5th
+    assertEquals(6,completedJob.getTaskAttemptCompletionEvents(5,10).length);
+
+    // without errors
+    assertEquals(1,completedJob.getDiagnostics().size());
+    assertEquals("",completedJob.getDiagnostics().get(0));
+
+    assertEquals(0, completedJob.getJobACLs().size());
+
+  }
+
 }

+ 335 - 201
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java

@@ -19,6 +19,9 @@
 package org.apache.hadoop.mapreduce.v2.hs;
 
 import java.io.ByteArrayOutputStream;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.Arrays;
@@ -54,6 +57,9 @@ 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.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.mapreduce.v2.api.records.impl.pb.JobIdPBImpl;
+import org.apache.hadoop.mapreduce.v2.api.records.impl.pb.TaskIdPBImpl;
 import org.apache.hadoop.mapreduce.v2.app.MRApp;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
@@ -65,7 +71,9 @@ import org.apache.hadoop.mapreduce.v2.hs.TestJobHistoryEvents.MRAppWithHistory;
 import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
+import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobsInfo;
 import org.apache.hadoop.net.DNSToSwitchMapping;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.service.Service;
 import org.apache.hadoop.yarn.util.BuilderUtils;
@@ -80,12 +88,12 @@ public class TestJobHistoryParsing {
 
   private static final String RACK_NAME = "/MyRackName";
 
-  private  ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+  private ByteArrayOutputStream outContent = new ByteArrayOutputStream();
 
   public static class MyResolver implements DNSToSwitchMapping {
     @Override
     public List<String> resolve(List<String> names) {
-      return Arrays.asList(new String[]{RACK_NAME});
+      return Arrays.asList(new String[] { RACK_NAME });
     }
 
     @Override
@@ -93,14 +101,14 @@ public class TestJobHistoryParsing {
     }
   }
 
-  @Test (timeout=50000)
+  @Test(timeout = 50000)
   public void testJobInfo() throws Exception {
     JobInfo info = new JobInfo();
     Assert.assertEquals("NORMAL", info.getPriority());
     info.printAll();
   }
 
-  @Test (timeout=50000)
+  @Test(timeout = 300000)
   public void testHistoryParsing() throws Exception {
     LOG.info("STARTING testHistoryParsing()");
     try {
@@ -109,8 +117,8 @@ public class TestJobHistoryParsing {
       LOG.info("FINISHED testHistoryParsing()");
     }
   }
-  
-  @Test (timeout=50000)
+
+  @Test(timeout = 50000)
   public void testHistoryParsingWithParseErrors() throws Exception {
     LOG.info("STARTING testHistoryParsingWithParseErrors()");
     try {
@@ -119,18 +127,18 @@ public class TestJobHistoryParsing {
       LOG.info("FINISHED testHistoryParsingWithParseErrors()");
     }
   }
-  
-  private static String getJobSummary(FileContext fc, Path path) throws IOException {
+
+  private static String getJobSummary(FileContext fc, Path path)
+      throws IOException {
     Path qPath = fc.makeQualified(path);
     FSDataInputStream in = fc.open(qPath);
     String jobSummaryString = in.readUTF();
     in.close();
     return jobSummaryString;
   }
-  
+
   private void checkHistoryParsing(final int numMaps, final int numReduces,
-      final int numSuccessfulMaps) 
-  throws Exception {
+      final int numSuccessfulMaps) throws Exception {
     Configuration conf = new Configuration();
     conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name"));
     long amStartTimeEst = System.currentTimeMillis();
@@ -138,9 +146,8 @@ public class TestJobHistoryParsing {
         CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
         MyResolver.class, DNSToSwitchMapping.class);
     RackResolver.init(conf);
-    MRApp app = 
-        new MRAppWithHistory(numMaps, numReduces, true, 
-            this.getClass().getName(), true);
+    MRApp app = new MRAppWithHistory(numMaps, numReduces, true, this.getClass()
+        .getName(), true);
     app.submit(conf);
     Job job = app.getContext().getAllJobs().values().iterator().next();
     JobId jobId = job.getID();
@@ -152,7 +159,7 @@ public class TestJobHistoryParsing {
 
     String jobhistoryDir = JobHistoryUtils
         .getHistoryIntermediateDoneDirForUser(conf);
-    
+
     FileContext fc = null;
     try {
       fc = FileContext.getFileContext(conf);
@@ -160,7 +167,7 @@ public class TestJobHistoryParsing {
       LOG.info("Can not get FileContext", ioe);
       throw (new Exception("Can not get File Context"));
     }
-    
+
     if (numMaps == numSuccessfulMaps) {
       String summaryFileName = JobHistoryUtils
           .getIntermediateSummaryFileName(jobId);
@@ -185,20 +192,22 @@ public class TestJobHistoryParsing {
           Long.parseLong(jobSummaryElements.get("submitTime")) != 0);
       Assert.assertTrue("launchTime should not be 0",
           Long.parseLong(jobSummaryElements.get("launchTime")) != 0);
-      Assert.assertTrue("firstMapTaskLaunchTime should not be 0",
-          Long.parseLong(jobSummaryElements.get("firstMapTaskLaunchTime")) != 0);
       Assert
-      .assertTrue(
-          "firstReduceTaskLaunchTime should not be 0",
-          Long.parseLong(jobSummaryElements.get("firstReduceTaskLaunchTime")) != 0);
+          .assertTrue(
+              "firstMapTaskLaunchTime should not be 0",
+              Long.parseLong(jobSummaryElements.get("firstMapTaskLaunchTime")) != 0);
+      Assert
+          .assertTrue("firstReduceTaskLaunchTime should not be 0",
+              Long.parseLong(jobSummaryElements
+                  .get("firstReduceTaskLaunchTime")) != 0);
       Assert.assertTrue("finishTime should not be 0",
           Long.parseLong(jobSummaryElements.get("finishTime")) != 0);
       Assert.assertEquals("Mismatch in num map slots", numSuccessfulMaps,
           Integer.parseInt(jobSummaryElements.get("numMaps")));
       Assert.assertEquals("Mismatch in num reduce slots", numReduces,
           Integer.parseInt(jobSummaryElements.get("numReduces")));
-      Assert.assertEquals("User does not match", System.getProperty("user.name"),
-          jobSummaryElements.get("user"));
+      Assert.assertEquals("User does not match",
+          System.getProperty("user.name"), jobSummaryElements.get("user"));
       Assert.assertEquals("Queue does not match", "default",
           jobSummaryElements.get("queue"));
       Assert.assertEquals("Status does not match", "SUCCEEDED",
@@ -210,8 +219,8 @@ public class TestJobHistoryParsing {
     HistoryFileInfo fileInfo = jobHistory.getJobFileInfo(jobId);
     JobInfo jobInfo;
     long numFinishedMaps;
-    
-    synchronized(fileInfo) {
+
+    synchronized (fileInfo) {
       Path historyFilePath = fileInfo.getHistoryFile();
       FSDataInputStream in = null;
       LOG.info("JobHistoryFile is: " + historyFilePath);
@@ -228,11 +237,11 @@ public class TestJobHistoryParsing {
       if (numMaps == numSuccessfulMaps) {
         reader = realReader;
       } else {
-        final AtomicInteger numFinishedEvents = new AtomicInteger(0);  // Hack!
+        final AtomicInteger numFinishedEvents = new AtomicInteger(0); // Hack!
         Mockito.when(reader.getNextEvent()).thenAnswer(
             new Answer<HistoryEvent>() {
-              public HistoryEvent answer(InvocationOnMock invocation) 
-              throws IOException {
+              public HistoryEvent answer(InvocationOnMock invocation)
+                  throws IOException {
                 HistoryEvent event = realReader.getNextEvent();
                 if (event instanceof TaskFinishedEvent) {
                   numFinishedEvents.incrementAndGet();
@@ -244,22 +253,20 @@ public class TestJobHistoryParsing {
                   throw new IOException("test");
                 }
               }
-            }
-        );
+            });
       }
 
       jobInfo = parser.parse(reader);
 
-      numFinishedMaps = 
-        computeFinishedMaps(jobInfo, numMaps, numSuccessfulMaps);
+      numFinishedMaps = computeFinishedMaps(jobInfo, numMaps, numSuccessfulMaps);
 
       if (numFinishedMaps != numMaps) {
         Exception parseException = parser.getParseException();
-        Assert.assertNotNull("Didn't get expected parse exception", 
+        Assert.assertNotNull("Didn't get expected parse exception",
             parseException);
       }
     }
-    
+
     Assert.assertEquals("Incorrect username ", System.getProperty("user.name"),
         jobInfo.getUsername());
     Assert.assertEquals("Incorrect jobName ", "test", jobInfo.getJobname());
@@ -267,7 +274,7 @@ public class TestJobHistoryParsing {
         jobInfo.getJobQueueName());
     Assert
         .assertEquals("incorrect conf path", "test", jobInfo.getJobConfPath());
-    Assert.assertEquals("incorrect finishedMap ", numSuccessfulMaps, 
+    Assert.assertEquals("incorrect finishedMap ", numSuccessfulMaps,
         numFinishedMaps);
     Assert.assertEquals("incorrect finishedReduces ", numReduces,
         jobInfo.getFinishedReduces());
@@ -275,8 +282,8 @@ public class TestJobHistoryParsing {
         jobInfo.getUberized());
     Map<TaskID, TaskInfo> allTasks = jobInfo.getAllTasks();
     int totalTasks = allTasks.size();
-    Assert.assertEquals("total number of tasks is incorrect  ", 
-        (numMaps+numReduces), totalTasks);
+    Assert.assertEquals("total number of tasks is incorrect  ",
+        (numMaps + numReduces), totalTasks);
 
     // Verify aminfo
     Assert.assertEquals(1, jobInfo.getAMInfos().size());
@@ -306,8 +313,7 @@ public class TestJobHistoryParsing {
 
     // Deep compare Job and JobInfo
     for (Task task : job.getTasks().values()) {
-      TaskInfo taskInfo = allTasks.get(
-          TypeConverter.fromYarn(task.getID()));
+      TaskInfo taskInfo = allTasks.get(TypeConverter.fromYarn(task.getID()));
       Assert.assertNotNull("TaskInfo not found", taskInfo);
       for (TaskAttempt taskAttempt : task.getAttempts().values()) {
         TaskAttemptInfo taskAttemptInfo = taskInfo.getAllTaskAttempts().get(
@@ -318,27 +324,32 @@ public class TestJobHistoryParsing {
         if (numMaps == numSuccessfulMaps) {
           Assert.assertEquals(MRApp.NM_HOST, taskAttemptInfo.getHostname());
           Assert.assertEquals(MRApp.NM_PORT, taskAttemptInfo.getPort());
-          
+
           // Verify rack-name
-          Assert.assertEquals("rack-name is incorrect", taskAttemptInfo
-              .getRackname(), RACK_NAME);
+          Assert.assertEquals("rack-name is incorrect",
+              taskAttemptInfo.getRackname(), RACK_NAME);
         }
       }
     }
-    
+
     // test output for HistoryViewer
-    PrintStream stdps=System.out;
+    PrintStream stdps = System.out;
     try {
       System.setOut(new PrintStream(outContent));
       HistoryViewer viewer = new HistoryViewer(fc.makeQualified(
           fileInfo.getHistoryFile()).toString(), conf, true);
       viewer.print();
-      
-      for (TaskInfo taskInfo : allTasks.values()) { 
-        
-        String test=  (taskInfo.getTaskStatus()==null?"":taskInfo.getTaskStatus())+" "+taskInfo.getTaskType()+" task list for "+taskInfo.getTaskId().getJobID();
-        Assert.assertTrue(outContent.toString().indexOf(test)>0);
-        Assert.assertTrue(outContent.toString().indexOf(taskInfo.getTaskId().toString())>0);
+
+      for (TaskInfo taskInfo : allTasks.values()) {
+
+        String test = (taskInfo.getTaskStatus() == null ? "" : taskInfo
+            .getTaskStatus())
+            + " "
+            + taskInfo.getTaskType()
+            + " task list for " + taskInfo.getTaskId().getJobID();
+        Assert.assertTrue(outContent.toString().indexOf(test) > 0);
+        Assert.assertTrue(outContent.toString().indexOf(
+            taskInfo.getTaskId().toString()) > 0);
       }
     } finally {
       System.setOut(stdps);
@@ -363,186 +374,180 @@ public class TestJobHistoryParsing {
     }
     return numFinishedMaps;
   }
-  
-  @Test (timeout=50000)
+
+  @Test(timeout = 30000)
   public void testHistoryParsingForFailedAttempts() throws Exception {
     LOG.info("STARTING testHistoryParsingForFailedAttempts");
     try {
-    Configuration conf = new Configuration();
-    conf
-        .setClass(
-            CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
-            MyResolver.class, DNSToSwitchMapping.class);
-    RackResolver.init(conf);
-    MRApp app = new MRAppWithHistoryWithFailedAttempt(2, 1, true, this.getClass().getName(),
-        true);
-    app.submit(conf);
-    Job job = app.getContext().getAllJobs().values().iterator().next();
-    JobId jobId = job.getID();
-    app.waitForState(job, JobState.SUCCEEDED);
-    
-    // make sure all events are flushed
-    app.waitForState(Service.STATE.STOPPED);
-
-    String jobhistoryDir = JobHistoryUtils
-        .getHistoryIntermediateDoneDirForUser(conf);
-    JobHistory jobHistory = new JobHistory();
-    jobHistory.init(conf);
-
-    JobIndexInfo jobIndexInfo = jobHistory.getJobFileInfo(jobId)
-        .getJobIndexInfo();
-    String jobhistoryFileName = FileNameIndexUtils
-        .getDoneFileName(jobIndexInfo);
-
-    Path historyFilePath = new Path(jobhistoryDir, jobhistoryFileName);
-    FSDataInputStream in = null;
-    FileContext fc = null;
-    try {
-      fc = FileContext.getFileContext(conf);
-      in = fc.open(fc.makeQualified(historyFilePath));
-    } catch (IOException ioe) {
-      LOG.info("Can not open history file: " + historyFilePath, ioe);
-      throw (new Exception("Can not open History File"));
-    }
+      Configuration conf = new Configuration();
+      conf.setClass(
+          CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+          MyResolver.class, DNSToSwitchMapping.class);
+      RackResolver.init(conf);
+      MRApp app = new MRAppWithHistoryWithFailedAttempt(2, 1, true, this
+          .getClass().getName(), true);
+      app.submit(conf);
+      Job job = app.getContext().getAllJobs().values().iterator().next();
+      JobId jobId = job.getID();
+      app.waitForState(job, JobState.SUCCEEDED);
+
+      // make sure all events are flushed
+      app.waitForState(Service.STATE.STOPPED);
+
+      String jobhistoryDir = JobHistoryUtils
+          .getHistoryIntermediateDoneDirForUser(conf);
+      JobHistory jobHistory = new JobHistory();
+      jobHistory.init(conf);
+
+      JobIndexInfo jobIndexInfo = jobHistory.getJobFileInfo(jobId)
+          .getJobIndexInfo();
+      String jobhistoryFileName = FileNameIndexUtils
+          .getDoneFileName(jobIndexInfo);
+
+      Path historyFilePath = new Path(jobhistoryDir, jobhistoryFileName);
+      FSDataInputStream in = null;
+      FileContext fc = null;
+      try {
+        fc = FileContext.getFileContext(conf);
+        in = fc.open(fc.makeQualified(historyFilePath));
+      } catch (IOException ioe) {
+        LOG.info("Can not open history file: " + historyFilePath, ioe);
+        throw (new Exception("Can not open History File"));
+      }
 
-    JobHistoryParser parser = new JobHistoryParser(in);
-    JobInfo jobInfo = parser.parse();
-    Exception parseException = parser.getParseException();
-    Assert.assertNull("Caught an expected exception " + parseException, 
-        parseException);
-    int noOffailedAttempts = 0;
-    Map<TaskID, TaskInfo> allTasks = jobInfo.getAllTasks();
-    for (Task task : job.getTasks().values()) {
-      TaskInfo taskInfo = allTasks.get(TypeConverter.fromYarn(task.getID()));
-      for (TaskAttempt taskAttempt : task.getAttempts().values()) {
-        TaskAttemptInfo taskAttemptInfo = taskInfo.getAllTaskAttempts().get(
-            TypeConverter.fromYarn((taskAttempt.getID())));
-        // Verify rack-name for all task attempts
-        Assert.assertEquals("rack-name is incorrect", taskAttemptInfo
-            .getRackname(), RACK_NAME);
-        if (taskAttemptInfo.getTaskStatus().equals("FAILED")) {
-          noOffailedAttempts++;
+      JobHistoryParser parser = new JobHistoryParser(in);
+      JobInfo jobInfo = parser.parse();
+      Exception parseException = parser.getParseException();
+      Assert.assertNull("Caught an expected exception " + parseException,
+          parseException);
+      int noOffailedAttempts = 0;
+      Map<TaskID, TaskInfo> allTasks = jobInfo.getAllTasks();
+      for (Task task : job.getTasks().values()) {
+        TaskInfo taskInfo = allTasks.get(TypeConverter.fromYarn(task.getID()));
+        for (TaskAttempt taskAttempt : task.getAttempts().values()) {
+          TaskAttemptInfo taskAttemptInfo = taskInfo.getAllTaskAttempts().get(
+              TypeConverter.fromYarn((taskAttempt.getID())));
+          // Verify rack-name for all task attempts
+          Assert.assertEquals("rack-name is incorrect",
+              taskAttemptInfo.getRackname(), RACK_NAME);
+          if (taskAttemptInfo.getTaskStatus().equals("FAILED")) {
+            noOffailedAttempts++;
+          }
         }
       }
-    }
-    Assert.assertEquals("No of Failed tasks doesn't match.", 2, noOffailedAttempts);
+      Assert.assertEquals("No of Failed tasks doesn't match.", 2,
+          noOffailedAttempts);
     } finally {
       LOG.info("FINISHED testHistoryParsingForFailedAttempts");
     }
   }
-  
-  @Test (timeout=5000)
+
+  @Test(timeout = 60000)
   public void testCountersForFailedTask() throws Exception {
     LOG.info("STARTING testCountersForFailedTask");
     try {
-    Configuration conf = new Configuration();
-    conf
-        .setClass(
-            CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
-            MyResolver.class, DNSToSwitchMapping.class);
-    RackResolver.init(conf);
-    MRApp app = new MRAppWithHistoryWithFailedTask(2, 1, true,
-        this.getClass().getName(), true);
-    app.submit(conf);
-    Job job = app.getContext().getAllJobs().values().iterator().next();
-    JobId jobId = job.getID();
-    app.waitForState(job, JobState.FAILED);
-
-    // make sure all events are flushed
-    app.waitForState(Service.STATE.STOPPED);
-
-    String jobhistoryDir = JobHistoryUtils
-        .getHistoryIntermediateDoneDirForUser(conf);
-    JobHistory jobHistory = new JobHistory();
-    jobHistory.init(conf);
-
-    JobIndexInfo jobIndexInfo = jobHistory.getJobFileInfo(jobId)
-        .getJobIndexInfo();
-    String jobhistoryFileName = FileNameIndexUtils
-        .getDoneFileName(jobIndexInfo);
-
-    Path historyFilePath = new Path(jobhistoryDir, jobhistoryFileName);
-    FSDataInputStream in = null;
-    FileContext fc = null;
-    try {
-      fc = FileContext.getFileContext(conf);
-      in = fc.open(fc.makeQualified(historyFilePath));
-    } catch (IOException ioe) {
-      LOG.info("Can not open history file: " + historyFilePath, ioe);
-      throw (new Exception("Can not open History File"));
-    }
+      Configuration conf = new Configuration();
+      conf.setClass(
+          CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+          MyResolver.class, DNSToSwitchMapping.class);
+      RackResolver.init(conf);
+      MRApp app = new MRAppWithHistoryWithFailedTask(2, 1, true, this
+          .getClass().getName(), true);
+      app.submit(conf);
+      Job job = app.getContext().getAllJobs().values().iterator().next();
+      JobId jobId = job.getID();
+      app.waitForState(job, JobState.FAILED);
+
+      // make sure all events are flushed
+      app.waitForState(Service.STATE.STOPPED);
+
+      String jobhistoryDir = JobHistoryUtils
+          .getHistoryIntermediateDoneDirForUser(conf);
+      JobHistory jobHistory = new JobHistory();
+      jobHistory.init(conf);
+
+      JobIndexInfo jobIndexInfo = jobHistory.getJobFileInfo(jobId)
+          .getJobIndexInfo();
+      String jobhistoryFileName = FileNameIndexUtils
+          .getDoneFileName(jobIndexInfo);
+
+      Path historyFilePath = new Path(jobhistoryDir, jobhistoryFileName);
+      FSDataInputStream in = null;
+      FileContext fc = null;
+      try {
+        fc = FileContext.getFileContext(conf);
+        in = fc.open(fc.makeQualified(historyFilePath));
+      } catch (IOException ioe) {
+        LOG.info("Can not open history file: " + historyFilePath, ioe);
+        throw (new Exception("Can not open History File"));
+      }
 
-    JobHistoryParser parser = new JobHistoryParser(in);
-    JobInfo jobInfo = parser.parse();
-    Exception parseException = parser.getParseException();
-    Assert.assertNull("Caught an expected exception " + parseException,
-        parseException);
-    for (Map.Entry<TaskID,TaskInfo> entry : jobInfo.getAllTasks().entrySet()) {
-      TaskId yarnTaskID = TypeConverter.toYarn(entry.getKey());
-      CompletedTask ct = new CompletedTask(yarnTaskID, entry.getValue());
-      Assert.assertNotNull("completed task report has null counters",
-          ct.getReport().getCounters());
-      //Make sure all the completedTask has counters, and the counters are not empty
-      Assert.assertTrue(ct.getReport().getCounters()
-          .getAllCounterGroups().size() > 0);
-    }
+      JobHistoryParser parser = new JobHistoryParser(in);
+      JobInfo jobInfo = parser.parse();
+      Exception parseException = parser.getParseException();
+      Assert.assertNull("Caught an expected exception " + parseException,
+          parseException);
+      for (Map.Entry<TaskID, TaskInfo> entry : jobInfo.getAllTasks().entrySet()) {
+        TaskId yarnTaskID = TypeConverter.toYarn(entry.getKey());
+        CompletedTask ct = new CompletedTask(yarnTaskID, entry.getValue());
+        Assert.assertNotNull("completed task report has null counters", ct
+            .getReport().getCounters());
+      }
     } finally {
       LOG.info("FINISHED testCountersForFailedTask");
     }
   }
 
-  @Test (timeout=50000)
+  @Test(timeout = 50000)
   public void testScanningOldDirs() throws Exception {
     LOG.info("STARTING testScanningOldDirs");
     try {
-    Configuration conf = new Configuration();
-    conf
-        .setClass(
-            CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
-            MyResolver.class, DNSToSwitchMapping.class);
-    RackResolver.init(conf);
-    MRApp app =
-        new MRAppWithHistory(1, 1, true,
-            this.getClass().getName(), true);
-    app.submit(conf);
-    Job job = app.getContext().getAllJobs().values().iterator().next();
-    JobId jobId = job.getID();
-    LOG.info("JOBID is " + TypeConverter.fromYarn(jobId).toString());
-    app.waitForState(job, JobState.SUCCEEDED);
-
-    // make sure all events are flushed
-    app.waitForState(Service.STATE.STOPPED);
-
-    HistoryFileManagerForTest hfm = new HistoryFileManagerForTest();
-    hfm.init(conf);
-    HistoryFileInfo fileInfo = hfm.getFileInfo(jobId);
-    Assert.assertNotNull("Unable to locate job history", fileInfo);
-
-    // force the manager to "forget" the job
-    hfm.deleteJobFromJobListCache(fileInfo);
-    final int msecPerSleep = 10;
-    int msecToSleep = 10 * 1000;
-    while (fileInfo.isMovePending() && msecToSleep > 0) {
-      Assert.assertTrue(!fileInfo.didMoveFail());
-      msecToSleep -= msecPerSleep;
-      Thread.sleep(msecPerSleep);
-    }
-    Assert.assertTrue("Timeout waiting for history move", msecToSleep > 0);
+      Configuration conf = new Configuration();
+      conf.setClass(
+          CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+          MyResolver.class, DNSToSwitchMapping.class);
+      RackResolver.init(conf);
+      MRApp app = new MRAppWithHistory(1, 1, true, this.getClass().getName(),
+          true);
+      app.submit(conf);
+      Job job = app.getContext().getAllJobs().values().iterator().next();
+      JobId jobId = job.getID();
+      LOG.info("JOBID is " + TypeConverter.fromYarn(jobId).toString());
+      app.waitForState(job, JobState.SUCCEEDED);
+
+      // make sure all events are flushed
+      app.waitForState(Service.STATE.STOPPED);
+
+      HistoryFileManagerForTest hfm = new HistoryFileManagerForTest();
+      hfm.init(conf);
+      HistoryFileInfo fileInfo = hfm.getFileInfo(jobId);
+      Assert.assertNotNull("Unable to locate job history", fileInfo);
+
+      // force the manager to "forget" the job
+      hfm.deleteJobFromJobListCache(fileInfo);
+      final int msecPerSleep = 10;
+      int msecToSleep = 10 * 1000;
+      while (fileInfo.isMovePending() && msecToSleep > 0) {
+        Assert.assertTrue(!fileInfo.didMoveFail());
+        msecToSleep -= msecPerSleep;
+        Thread.sleep(msecPerSleep);
+      }
+      Assert.assertTrue("Timeout waiting for history move", msecToSleep > 0);
 
-    fileInfo = hfm.getFileInfo(jobId);
-    Assert.assertNotNull("Unable to locate old job history", fileInfo);
-   } finally {
+      fileInfo = hfm.getFileInfo(jobId);
+      Assert.assertNotNull("Unable to locate old job history", fileInfo);
+    } finally {
       LOG.info("FINISHED testScanningOldDirs");
     }
   }
 
   static class MRAppWithHistoryWithFailedAttempt extends MRAppWithHistory {
 
-    public MRAppWithHistoryWithFailedAttempt(int maps, int reduces, boolean autoComplete,
-        String testName, boolean cleanOnStart) {
+    public MRAppWithHistoryWithFailedAttempt(int maps, int reduces,
+        boolean autoComplete, String testName, boolean cleanOnStart) {
       super(maps, reduces, autoComplete, testName, cleanOnStart);
     }
-    
+
     @SuppressWarnings("unchecked")
     @Override
     protected void attemptLaunched(TaskAttemptId attemptID) {
@@ -558,8 +563,8 @@ public class TestJobHistoryParsing {
 
   static class MRAppWithHistoryWithFailedTask extends MRAppWithHistory {
 
-    public MRAppWithHistoryWithFailedTask(int maps, int reduces, boolean autoComplete,
-        String testName, boolean cleanOnStart) {
+    public MRAppWithHistoryWithFailedTask(int maps, int reduces,
+        boolean autoComplete, String testName, boolean cleanOnStart) {
       super(maps, reduces, autoComplete, testName, cleanOnStart);
     }
 
@@ -587,4 +592,133 @@ public class TestJobHistoryParsing {
     t.testHistoryParsing();
     t.testHistoryParsingForFailedAttempts();
   }
+
+  /**
+   * Test clean old history files. Files should be deleted after 1 week by
+   * default.
+   */
+  @Test(timeout = 15000)
+  public void testDeleteFileInfo() throws Exception {
+    LOG.info("STARTING testDeleteFileInfo");
+    try {
+      Configuration conf = new Configuration();
+
+      conf.setClass(
+          CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+          MyResolver.class, DNSToSwitchMapping.class);
+
+      RackResolver.init(conf);
+      MRApp app = new MRAppWithHistory(1, 1, true, this.getClass().getName(),
+          true);
+      app.submit(conf);
+      Job job = app.getContext().getAllJobs().values().iterator().next();
+      JobId jobId = job.getID();
+
+      app.waitForState(job, JobState.SUCCEEDED);
+
+      // make sure all events are flushed
+      app.waitForState(Service.STATE.STOPPED);
+
+      HistoryFileManager hfm = new HistoryFileManager();
+      hfm.init(conf);
+      HistoryFileInfo fileInfo = hfm.getFileInfo(jobId);
+      hfm.initExisting();
+      // wait for move files form the done_intermediate directory to the gone
+      // directory
+      while (fileInfo.isMovePending()) {
+        Thread.sleep(300);
+      }
+
+      Assert.assertNotNull(hfm.jobListCache.values());
+
+      // try to remove fileInfo
+      hfm.clean();
+      // check that fileInfo does not deleted
+      Assert.assertFalse(fileInfo.isDeleted());
+      // correct live time
+      hfm.setMaxHistoryAge(-1);
+      hfm.clean();
+      // should be deleted !
+      Assert.assertTrue("file should be deleted ", fileInfo.isDeleted());
+
+    } finally {
+      LOG.info("FINISHED testDeleteFileInfo");
+    }
+  }
+
+  /**
+   * Simple test some methods of JobHistory
+   */
+  @Test(timeout = 20000)
+  public void testJobHistoryMethods() throws Exception {
+    LOG.info("STARTING testJobHistoryMethods");
+    try {
+      Configuration configuration = new Configuration();
+      configuration
+          .setClass(
+              CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+              MyResolver.class, DNSToSwitchMapping.class);
+
+      RackResolver.init(configuration);
+      MRApp app = new MRAppWithHistory(1, 1, true, this.getClass().getName(),
+          true);
+      app.submit(configuration);
+      Job job = app.getContext().getAllJobs().values().iterator().next();
+      JobId jobId = job.getID();
+      LOG.info("JOBID is " + TypeConverter.fromYarn(jobId).toString());
+      app.waitForState(job, JobState.SUCCEEDED);
+
+      JobHistory jobHistory = new JobHistory();
+      jobHistory.init(configuration);
+      // Method getAllJobs
+      Assert.assertEquals(1, jobHistory.getAllJobs().size());
+      // and with ApplicationId
+      Assert.assertEquals(1, jobHistory.getAllJobs(app.getAppID()).size());
+
+      JobsInfo jobsinfo = jobHistory.getPartialJobs(0L, 10L, null, "default",
+          0L, System.currentTimeMillis() + 1, 0L,
+          System.currentTimeMillis() + 1, JobState.SUCCEEDED);
+
+      Assert.assertEquals(1, jobsinfo.getJobs().size());
+      Assert.assertNotNull(jobHistory.getApplicationAttemptId());
+      // test Application Id
+      Assert.assertEquals("application_0_0000", jobHistory.getApplicationID()
+          .toString());
+      Assert
+          .assertEquals("Job History Server", jobHistory.getApplicationName());
+      // method does not work
+      Assert.assertNull(jobHistory.getEventHandler());
+      // method does not work
+      Assert.assertNull(jobHistory.getClock());
+      // method does not work
+      Assert.assertNull(jobHistory.getClusterInfo());
+
+    } finally {
+      LOG.info("FINISHED testJobHistoryMethods");
+    }
+  }
+
+  /**
+   * Simple test PartialJob
+   */
+  @Test(timeout = 1000)
+  public void testPartialJob() throws Exception {
+    JobId jobId = new JobIdPBImpl();
+    jobId.setId(0);
+    JobIndexInfo jii = new JobIndexInfo(0L, System.currentTimeMillis(), "user",
+        "jobName", jobId, 3, 2, "JobStatus");
+    PartialJob test = new PartialJob(jii, jobId);
+    
+    Assert.assertEquals(1.0f, test.getProgress(), 0.001f);
+    assertNull(test.getAllCounters());
+    assertNull(test.getTasks());
+    assertNull(test.getTasks(TaskType.MAP));
+    assertNull(test.getTask(new TaskIdPBImpl()));
+
+    assertNull(test.getTaskAttemptCompletionEvents(0, 100));
+    assertNull(test.getMapAttemptCompletionEvents(0, 100));
+    assertTrue(test.checkAccess(UserGroupInformation.getCurrentUser(), null));
+    assertNull(test.getAMInfos());
+
+  }
 }

+ 209 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryServer.java

@@ -0,0 +1,209 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce.v2.hs;
+
+
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.mapreduce.TaskCounter;
+import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDiagnosticsRequest;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDiagnosticsResponse;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskAttemptCompletionEventsRequest;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskAttemptCompletionEventsResponse;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskAttemptReportRequest;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskAttemptReportResponse;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskReportRequest;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskReportResponse;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+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.TaskId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
+import org.apache.hadoop.mapreduce.v2.app.MRApp;
+import org.apache.hadoop.mapreduce.v2.app.job.Job;
+import org.apache.hadoop.mapreduce.v2.app.job.Task;
+import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
+import org.apache.hadoop.mapreduce.v2.hs.TestJobHistoryEvents.MRAppWithHistory;
+import org.apache.hadoop.mapreduce.v2.hs.TestJobHistoryParsing.MyResolver;
+import org.apache.hadoop.net.DNSToSwitchMapping;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.service.Service;
+import org.apache.hadoop.yarn.service.Service.STATE;
+import org.apache.hadoop.yarn.util.RackResolver;
+import org.junit.After;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+/*
+test JobHistoryServer protocols....
+ */
+public class TestJobHistoryServer {
+  private static RecordFactory recordFactory = RecordFactoryProvider
+          .getRecordFactory(null);
+
+
+  
+  JobHistoryServer historyServer=null;
+  // simple test init/start/stop   JobHistoryServer. Status should change.
+  
+  @Test (timeout= 50000 )
+  public void testStartStopServer() throws Exception {
+
+    historyServer = new JobHistoryServer();
+    Configuration config = new Configuration();
+    historyServer.init(config);
+    assertEquals(STATE.INITED, historyServer.getServiceState());
+    assertEquals(3, historyServer.getServices().size());
+    historyServer.start();
+    assertEquals(STATE.STARTED, historyServer.getServiceState());
+    historyServer.stop();
+    assertEquals(STATE.STOPPED, historyServer.getServiceState());
+    assertNotNull(historyServer.getClientService());
+    HistoryClientService historyService = historyServer.getClientService();
+    assertNotNull(historyService.getClientHandler().getConnectAddress());
+
+    
+    
+  }
+
+
+
+  //Test reports of  JobHistoryServer. History server should get log files from  MRApp and read them
+  
+  @Test (timeout= 50000 )
+  public void testReports() throws Exception {
+    Configuration config = new Configuration();
+    config
+            .setClass(
+                    CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+                    MyResolver.class, DNSToSwitchMapping.class);
+
+    RackResolver.init(config);
+    MRApp app = new MRAppWithHistory(1, 1, true, this.getClass().getName(),
+            true);
+    app.submit(config);
+    Job job = app.getContext().getAllJobs().values().iterator().next();
+    app.waitForState(job, JobState.SUCCEEDED);
+
+    historyServer = new JobHistoryServer();
+
+    historyServer.init(config);
+    historyServer.start();
+    
+    // search JobHistory  service
+    JobHistory jobHistory= null;
+    for (Service service : historyServer.getServices() ) {
+      if (service instanceof JobHistory) {
+        jobHistory = (JobHistory) service;
+      }
+    };
+    
+    Map<JobId, Job> jobs= jobHistory.getAllJobs();
+    
+    assertEquals(1, jobs.size());
+    assertEquals("job_0_0000",jobs.keySet().iterator().next().toString());
+    
+    
+    Task task = job.getTasks().values().iterator().next();
+    TaskAttempt attempt = task.getAttempts().values().iterator().next();
+
+    HistoryClientService historyService = historyServer.getClientService();
+    MRClientProtocol protocol = historyService.getClientHandler();
+
+    GetTaskAttemptReportRequest gtarRequest = recordFactory
+            .newRecordInstance(GetTaskAttemptReportRequest.class);
+    // test getTaskAttemptReport
+    TaskAttemptId taId = attempt.getID();
+    taId.setTaskId(task.getID());
+    taId.getTaskId().setJobId(job.getID());
+    gtarRequest.setTaskAttemptId(taId);
+    GetTaskAttemptReportResponse response = protocol
+            .getTaskAttemptReport(gtarRequest);
+    assertEquals("container_0_0000_01_000000", response.getTaskAttemptReport()
+            .getContainerId().toString());
+    assertTrue(response.getTaskAttemptReport().getDiagnosticInfo().isEmpty());
+    // counters
+    assertNotNull(response.getTaskAttemptReport().getCounters()
+            .getCounter(TaskCounter.PHYSICAL_MEMORY_BYTES));
+    assertEquals(taId.toString(), response.getTaskAttemptReport()
+            .getTaskAttemptId().toString());
+    // test getTaskReport
+    GetTaskReportRequest request = recordFactory
+            .newRecordInstance(GetTaskReportRequest.class);
+    TaskId taskId = task.getID();
+    taskId.setJobId(job.getID());
+    request.setTaskId(taskId);
+    GetTaskReportResponse reportResponse = protocol.getTaskReport(request);
+    assertEquals("", reportResponse.getTaskReport().getDiagnosticsList()
+            .iterator().next());
+    // progress
+    assertEquals(1.0f, reportResponse.getTaskReport().getProgress(), 0.01);
+    // report has corrected taskId
+    assertEquals(taskId.toString(), reportResponse.getTaskReport().getTaskId()
+            .toString());
+    // Task state should be SUCCEEDED
+    assertEquals(TaskState.SUCCEEDED, reportResponse.getTaskReport()
+            .getTaskState());
+    // test getTaskAttemptCompletionEvents
+    GetTaskAttemptCompletionEventsRequest taskAttemptRequest = recordFactory
+            .newRecordInstance(GetTaskAttemptCompletionEventsRequest.class);
+    taskAttemptRequest.setJobId(job.getID());
+    GetTaskAttemptCompletionEventsResponse taskAttemptCompletionEventsResponse = protocol
+            .getTaskAttemptCompletionEvents(taskAttemptRequest);
+    assertEquals(0, taskAttemptCompletionEventsResponse.getCompletionEventCount());
+    
+    // test getDiagnostics
+    GetDiagnosticsRequest diagnosticRequest = recordFactory
+            .newRecordInstance(GetDiagnosticsRequest.class);
+    diagnosticRequest.setTaskAttemptId(taId);
+    GetDiagnosticsResponse diagnosticResponse = protocol
+            .getDiagnostics(diagnosticRequest);
+    // it is strange : why one empty string ?
+    assertEquals(1, diagnosticResponse.getDiagnosticsCount());
+    assertEquals("", diagnosticResponse.getDiagnostics(0));
+
+  }
+ // test main method
+  @Test (timeout =60000)
+  public void testMainMethod() throws Exception {
+
+    ExitUtil.disableSystemExit();
+    try {
+      JobHistoryServer.main(new String[0]);
+
+    } catch (ExitUtil.ExitException e) {
+      assertEquals(0,e.status);
+      ExitUtil.resetFirstExitException();
+      fail();
+    }
+  }
+  
+  @After
+  public void stop(){
+    if(historyServer !=null && !STATE.STOPPED.equals(historyServer.getServiceState())){
+      historyServer.stop();
+    }
+  }
+}

+ 6 - 1
hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/Job20LineHistoryEventEmitter.java

@@ -92,6 +92,10 @@ public class Job20LineHistoryEventEmitter extends HistoryEventEmitter {
       if (workflowAdjacencies == null) {
         workflowAdjacencies = "";
       }
+      String workflowTags = line.get("WORKFLOW_TAGS");
+      if (workflowTags == null) {
+        workflowTags = "";
+      }
       
 
       if (submitTime != null) {
@@ -104,7 +108,8 @@ public class Job20LineHistoryEventEmitter extends HistoryEventEmitter {
           new HashMap<JobACL, AccessControlList>();
         return new JobSubmittedEvent(jobID, jobName, user,
             that.originalSubmitTime, jobConf, jobACLs, jobQueueName,
-            workflowId, workflowName, workflowNodeName, workflowAdjacencies);
+            workflowId, workflowName, workflowNodeName, workflowAdjacencies,
+            workflowTags);
       }
 
       return null;

+ 24 - 3
hadoop-yarn-project/CHANGES.txt

@@ -25,6 +25,12 @@ Trunk - Unreleased
 
     YARN-491. TestContainerLogsPage fails on Windows. (Chris Nauroth via hitesh)
 
+    YARN-557. Fix TestUnmanagedAMLauncher failure on Windows. (Chris Nauroth via
+    vinodkv)
+
+    YARN-524 TestYarnVersionInfo failing if generated properties doesn't
+    include an SVN URL. (stevel)
+
   BREAKDOWN OF HADOOP-8562 SUBTASKS
 
     YARN-158. Yarn creating package-info.java must not depend on sh.
@@ -55,9 +61,6 @@ Trunk - Unreleased
     YARN-359. Fixing commands for container signalling in Windows. (Chris Nauroth
     via vinodkv)
     
-    YARN-524 TestYarnVersionInfo failing if generated properties doesn't
-    include an SVN URL(stevel)
-
 Release 2.0.5-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -123,6 +126,12 @@ Release 2.0.5-beta - UNRELEASED
     YARN-458. YARN daemon addresses must be placed in many different configs. 
     (sandyr via tucu)
 
+    YARN-193. Scheduler.normalizeRequest does not account for allocation
+    requests that exceed maximumAllocation limits (Zhijie Shen via bikas)
+
+    YARN-479. NM retry behavior for connection to RM should be similar for
+    lost heartbeats (Jian He via bikas)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -195,6 +204,16 @@ Release 2.0.5-beta - UNRELEASED
     to implement closeable so that they can be stopped when needed via
     RPC.stopProxy(). (Siddharth Seth via vinodkv)
 
+    YARN-99. Modify private distributed cache to localize files such that no
+    local directory hits unix file count limits and thus prevent job failures.
+    (Omkar Vinit Joshi via vinodkv)
+
+    YARN-112. Fixed a race condition during localization that fails containers.
+    (Omkar Vinit Joshi via vinodkv)
+
+    YARN-534. Change RM restart recovery to also account for AM max-attempts 
+    configuration after the restart. (Jian He via vinodkv)
+
 Release 2.0.4-alpha - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -509,6 +528,8 @@ Release 0.23.7 - UNRELEASED
     YARN-200. yarn log does not output all needed information, and is in a
     binary format (Ravi Prakash via jlowe)
 
+    YARN-525. make CS node-locality-delay refreshable (Thomas Graves via jlowe)
+
   OPTIMIZATIONS
 
     YARN-357. App submission should not be synchronized (daryn)

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java

@@ -103,7 +103,7 @@ public interface ApplicationConstants {
      * $USER
      * Final, non-modifiable.
      */
-    USER("USER"),
+    USER(Shell.WINDOWS ? "USERNAME": "USER"),
     
     /**
      * $LOGNAME

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

@@ -683,7 +683,7 @@ public class ApplicationMaster {
       ctx.setResource(container.getResource());
 
       String jobUserName = System.getenv(ApplicationConstants.Environment.USER
-          .name());
+          .key());
       ctx.setUser(jobUserName);
       LOG.info("Setting user in ContainerLaunchContext to: " + jobUserName);
 

+ 5 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/test/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/TestUnmanagedAMLauncher.java

@@ -30,6 +30,7 @@ import junit.framework.Assert;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.junit.AfterClass;
@@ -50,7 +51,7 @@ public class TestUnmanagedAMLauncher {
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 128);
     if (yarnCluster == null) {
       yarnCluster = new MiniYARNCluster(
-          TestUnmanagedAMLauncher.class.getName(), 1, 1, 1);
+          TestUnmanagedAMLauncher.class.getSimpleName(), 1, 1, 1);
       yarnCluster.init(conf);
       yarnCluster.start();
       URL url = Thread.currentThread().getContextClassLoader()
@@ -93,7 +94,7 @@ public class TestUnmanagedAMLauncher {
     return envClassPath;
   }
 
-  @Test(timeout=10000)
+  @Test(timeout=30000)
   public void testDSShell() throws Exception {
     String classpath = getTestRuntimeClasspath();
     String javaHome = System.getenv("JAVA_HOME");
@@ -112,7 +113,8 @@ public class TestUnmanagedAMLauncher {
         javaHome
             + "/bin/java -Xmx512m "
             + "org.apache.hadoop.yarn.applications.distributedshell.ApplicationMaster "
-            + "--container_memory 128 --num_containers 1 --priority 0 --shell_command ls" };
+            + "--container_memory 128 --num_containers 1 --priority 0 "
+            + "--shell_command " + (Shell.WINDOWS ? "dir" : "ls") };
 
     LOG.info("Initializing Launcher");
     UnmanagedAMLauncher launcher = new UnmanagedAMLauncher(new Configuration(

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -122,9 +122,9 @@ public class YarnConfiguration extends Configuration {
   public static final String RM_SCHEDULER_MAXIMUM_ALLOCATION_MB =
     YARN_PREFIX + "scheduler.maximum-allocation-mb";
   public static final int DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB = 8192;
-  public static final String RM_SCHEDULER_MAXIMUM_ALLOCATION_CORES =
+  public static final String RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES =
       YARN_PREFIX + "scheduler.maximum-allocation-vcores";
-  public static final int DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_CORES = 32;
+  public static final int DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES = 4;
 
   /** Number of threads to handle scheduler interface.*/
   public static final String RM_SCHEDULER_CLIENT_THREAD_COUNT =

+ 3 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java

@@ -23,7 +23,6 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
-import java.util.Random;
 import java.util.concurrent.Callable;
 import java.util.regex.Pattern;
 
@@ -36,13 +35,12 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.RunJar;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 
 /**
  * Download a single URL to the local disk.
@@ -51,8 +49,7 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 public class FSDownload implements Callable<Path> {
 
   private static final Log LOG = LogFactory.getLog(FSDownload.class);
-  
-  private Random rand;
+
   private FileContext files;
   private final UserGroupInformation userUgi;
   private Configuration conf;
@@ -71,13 +68,12 @@ public class FSDownload implements Callable<Path> {
 
 
   public FSDownload(FileContext files, UserGroupInformation ugi, Configuration conf,
-      Path destDirPath, LocalResource resource, Random rand) {
+      Path destDirPath, LocalResource resource) {
     this.conf = conf;
     this.destDirPath = destDirPath;
     this.files = files;
     this.userUgi = ugi;
     this.resource = resource;
-    this.rand = rand;
   }
 
   LocalResource getResource() {
@@ -270,11 +266,6 @@ public class FSDownload implements Callable<Path> {
     } catch (URISyntaxException e) {
       throw new IOException("Invalid resource", e);
     }
-    Path tmp;
-    do {
-      tmp = new Path(destDirPath, String.valueOf(rand.nextLong()));
-    } while (files.util().exists(tmp));
-    destDirPath = tmp;
     createDir(destDirPath, cachePerms);
     final Path dst_work = new Path(destDirPath + "_tmp");
     createDir(dst_work, cachePerms);
@@ -305,8 +296,6 @@ public class FSDownload implements Callable<Path> {
         files.delete(dst_work, true);
       } catch (FileNotFoundException ignore) {
       }
-      // clear ref to internal var
-      rand = null;
       conf = null;
       resource = null;
     }

+ 56 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestFSDownload.java

@@ -35,6 +35,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.jar.JarOutputStream;
 import java.util.jar.Manifest;
 
@@ -66,6 +67,8 @@ import org.junit.Test;
 public class TestFSDownload {
 
   private static final Log LOG = LogFactory.getLog(TestFSDownload.class);
+  private static AtomicLong uniqueNumberGenerator =
+    new AtomicLong(System.currentTimeMillis());
   
   @AfterClass
   public static void deleteTestDir() throws IOException {
@@ -267,9 +270,11 @@ public class TestFSDownload {
     rsrcVis.put(rsrc, vis);
     Path destPath = dirs.getLocalPathForWrite(
         basedir.toString(), size, conf);
+    destPath = new Path (destPath,
+      Long.toString(uniqueNumberGenerator.incrementAndGet()));
     FSDownload fsd =
       new FSDownload(files, UserGroupInformation.getCurrentUser(), conf,
-          destPath, rsrc, new Random(sharedSeed));
+          destPath, rsrc);
     pending.put(rsrc, exec.submit(fsd));
 
     try {
@@ -320,9 +325,11 @@ public class TestFSDownload {
       rsrcVis.put(rsrc, vis);
       Path destPath = dirs.getLocalPathForWrite(
           basedir.toString(), sizes[i], conf);
+      destPath = new Path (destPath,
+          Long.toString(uniqueNumberGenerator.incrementAndGet()));
       FSDownload fsd =
           new FSDownload(files, UserGroupInformation.getCurrentUser(), conf,
-              destPath, rsrc, new Random(sharedSeed));
+              destPath, rsrc);
       pending.put(rsrc, exec.submit(fsd));
     }
 
@@ -380,9 +387,10 @@ public class TestFSDownload {
     Path p = new Path(basedir, "" + 1);
     LocalResource rsrc = createTarFile(files, p, size, rand, vis);
     Path destPath = dirs.getLocalPathForWrite(basedir.toString(), size, conf);
+    destPath = new Path (destPath,
+        Long.toString(uniqueNumberGenerator.incrementAndGet()));
     FSDownload fsd = new FSDownload(files,
-        UserGroupInformation.getCurrentUser(), conf, destPath, rsrc,
-        new Random(sharedSeed));
+        UserGroupInformation.getCurrentUser(), conf, destPath, rsrc);
     pending.put(rsrc, exec.submit(fsd));
     
     try {
@@ -437,9 +445,10 @@ public class TestFSDownload {
     LocalResource rsrcjar = createJarFile(files, p, size, rand, vis);
     rsrcjar.setType(LocalResourceType.PATTERN);
     Path destPathjar = dirs.getLocalPathForWrite(basedir.toString(), size, conf);
+    destPathjar = new Path (destPathjar,
+        Long.toString(uniqueNumberGenerator.incrementAndGet()));
     FSDownload fsdjar = new FSDownload(files,
-        UserGroupInformation.getCurrentUser(), conf, destPathjar, rsrcjar,
-        new Random(sharedSeed));
+        UserGroupInformation.getCurrentUser(), conf, destPathjar, rsrcjar);
     pending.put(rsrcjar, exec.submit(fsdjar));
 
     try {
@@ -493,9 +502,10 @@ public class TestFSDownload {
     Path p = new Path(basedir, "" + 1);
     LocalResource rsrczip = createZipFile(files, p, size, rand, vis);
     Path destPathjar = dirs.getLocalPathForWrite(basedir.toString(), size, conf);
+    destPathjar = new Path (destPathjar,
+        Long.toString(uniqueNumberGenerator.incrementAndGet()));
     FSDownload fsdzip = new FSDownload(files,
-        UserGroupInformation.getCurrentUser(), conf, destPathjar, rsrczip,
-        new Random(sharedSeed));
+        UserGroupInformation.getCurrentUser(), conf, destPathjar, rsrczip);
     pending.put(rsrczip, exec.submit(fsdzip));
 
     try {
@@ -586,9 +596,11 @@ public class TestFSDownload {
       rsrcVis.put(rsrc, vis);
       Path destPath = dirs.getLocalPathForWrite(
           basedir.toString(), conf);
+      destPath = new Path (destPath,
+          Long.toString(uniqueNumberGenerator.incrementAndGet()));
       FSDownload fsd =
           new FSDownload(files, UserGroupInformation.getCurrentUser(), conf,
-              destPath, rsrc, new Random(sharedSeed));
+              destPath, rsrc);
       pending.put(rsrc, exec.submit(fsd));
     }
     
@@ -614,4 +626,38 @@ public class TestFSDownload {
     
 
   }
-}
+
+  @Test(timeout = 1000)
+  public void testUniqueDestinationPath() throws Exception {
+    Configuration conf = new Configuration();
+    FileContext files = FileContext.getLocalFSFileContext(conf);
+    final Path basedir = files.makeQualified(new Path("target",
+        TestFSDownload.class.getSimpleName()));
+    files.mkdir(basedir, null, true);
+    conf.setStrings(TestFSDownload.class.getName(), basedir.toString());
+
+    ExecutorService singleThreadedExec = Executors.newSingleThreadExecutor();
+
+    LocalDirAllocator dirs =
+        new LocalDirAllocator(TestFSDownload.class.getName());
+    Path destPath = dirs.getLocalPathForWrite(basedir.toString(), conf);
+    destPath =
+        new Path(destPath, Long.toString(uniqueNumberGenerator
+            .incrementAndGet()));
+    try {
+      Path p = new Path(basedir, "dir" + 0 + ".jar");
+      LocalResourceVisibility vis = LocalResourceVisibility.PRIVATE;
+      LocalResource rsrc = createJar(files, p, vis);
+      FSDownload fsd =
+          new FSDownload(files, UserGroupInformation.getCurrentUser(), conf,
+              destPath, rsrc);
+      Future<Path> rPath = singleThreadedExec.submit(fsd);
+      // Now FSDownload will not create a random directory to localize the
+      // resource. Therefore the final localizedPath for the resource should be
+      // destination directory (passed as an argument) + file name.
+      Assert.assertEquals(destPath, rPath.get().getParent());
+    } finally {
+      singleThreadedExec.shutdown();
+    }
+  }
+}

+ 43 - 23
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java

@@ -87,10 +87,9 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
   private final NodeHealthCheckerService healthChecker;
   private final NodeManagerMetrics metrics;
-
-  private boolean previousHeartBeatSucceeded;
-  private List<ContainerStatus> previousContainersStatuses =
-      new ArrayList<ContainerStatus>();
+  private long rmConnectWaitMS;
+  private long rmConnectionRetryIntervalMS;
+  private boolean waitForEver;
 
   public NodeStatusUpdaterImpl(Context context, Dispatcher dispatcher,
       NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
@@ -99,7 +98,6 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     this.context = context;
     this.dispatcher = dispatcher;
     this.metrics = metrics;
-    this.previousHeartBeatSucceeded = true;
   }
 
   @Override
@@ -137,8 +135,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
             YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS);
     
     LOG.info("Initialized nodemanager for " + nodeId + ":" +
-    		" physical-memory=" + memoryMb + " virtual-memory=" + virtualMemoryMb +
-    		" physical-cores=" + cpuCores + " virtual-cores=" + virtualCores);
+        " physical-memory=" + memoryMb + " virtual-memory=" + virtualMemoryMb +
+        " physical-cores=" + cpuCores + " virtual-cores=" + virtualCores);
     
     super.init(conf);
   }
@@ -192,12 +190,12 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
   private void registerWithRM() throws YarnRemoteException {
     Configuration conf = getConfig();
-    long rmConnectWaitMS =
+    rmConnectWaitMS =
         conf.getInt(
             YarnConfiguration.RESOURCEMANAGER_CONNECT_WAIT_SECS,
             YarnConfiguration.DEFAULT_RESOURCEMANAGER_CONNECT_WAIT_SECS)
         * 1000;
-    long rmConnectionRetryIntervalMS =
+    rmConnectionRetryIntervalMS =
         conf.getLong(
             YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_SECS,
             YarnConfiguration
@@ -210,7 +208,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
           " should not be negative.");
     }
 
-    boolean waitForEver = (rmConnectWaitMS == -1000);
+    waitForEver = (rmConnectWaitMS == -1000);
 
     if(! waitForEver) {
       if(rmConnectWaitMS < 0) {
@@ -319,14 +317,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     NodeStatus nodeStatus = recordFactory.newRecordInstance(NodeStatus.class);
     nodeStatus.setNodeId(this.nodeId);
 
-    List<ContainerStatus> containersStatuses = new ArrayList<ContainerStatus>();
-    if(previousHeartBeatSucceeded) {
-      previousContainersStatuses.clear();
-    } else {
-      containersStatuses.addAll(previousContainersStatuses);
-    }
-
     int numActiveContainers = 0;
+    List<ContainerStatus> containersStatuses = new ArrayList<ContainerStatus>();
     for (Iterator<Entry<ContainerId, Container>> i =
         this.context.getContainers().entrySet().iterator(); i.hasNext();) {
       Entry<ContainerId, Container> e = i.next();
@@ -341,7 +333,6 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
       LOG.info("Sending out status for container: " + containerStatus);
 
       if (containerStatus.getState() == ContainerState.COMPLETE) {
-        previousContainersStatuses.add(containerStatus);
         // Remove
         i.remove();
 
@@ -404,6 +395,9 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         while (!isStopped) {
           // Send heartbeat
           try {
+            NodeHeartbeatResponse response = null;
+            int rmRetryCount = 0;
+            long waitStartTime = System.currentTimeMillis();
             NodeStatus nodeStatus = getNodeStatus();
             nodeStatus.setResponseId(lastHeartBeatID);
             
@@ -414,9 +408,31 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
               request.setLastKnownMasterKey(NodeStatusUpdaterImpl.this.context
                 .getContainerTokenSecretManager().getCurrentKey());
             }
-            NodeHeartbeatResponse response =
-              resourceTracker.nodeHeartbeat(request);
-            previousHeartBeatSucceeded = true;
+            while (!isStopped) {
+              try {
+                rmRetryCount++;
+                response = resourceTracker.nodeHeartbeat(request);
+                break;
+              } catch (Throwable e) {
+                LOG.warn("Trying to heartbeat to ResourceManager, "
+                    + "current no. of failed attempts is " + rmRetryCount);
+                if(System.currentTimeMillis() - waitStartTime < rmConnectWaitMS
+                    || waitForEver) {
+                  try {
+                    LOG.info("Sleeping for " + rmConnectionRetryIntervalMS/1000
+                        + " seconds before next heartbeat to RM");
+                    Thread.sleep(rmConnectionRetryIntervalMS);
+                  } catch(InterruptedException ex) {
+                    //done nothing
+                  }
+                } else {
+                  String errorMessage = "Failed to heartbeat to RM, " +
+                      "no. of failed attempts is "+rmRetryCount;
+                  LOG.error(errorMessage,e);
+                  throw new YarnException(errorMessage,e);
+                }
+              }
+            }
             //get next heartbeat interval from response
             nextHeartBeatInterval = response.getNextHeartBeatInterval();
             // See if the master-key has rolled over
@@ -432,7 +448,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
             if (response.getNodeAction() == NodeAction.SHUTDOWN) {
               LOG
                   .info("Recieved SHUTDOWN signal from Resourcemanager as part of heartbeat," +
-                  		" hence shutting down.");
+                      " hence shutting down.");
               dispatcher.getEventHandler().handle(
                   new NodeManagerEvent(NodeManagerEventType.SHUTDOWN));
               break;
@@ -461,8 +477,12 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
               dispatcher.getEventHandler().handle(
                   new CMgrCompletedAppsEvent(appsToCleanup));
             }
+          } catch (YarnException e) {
+            //catch and throw the exception if tried MAX wait time to connect RM
+            dispatcher.getEventHandler().handle(
+                new NodeManagerEvent(NodeManagerEventType.SHUTDOWN));
+            throw e;
           } catch (Throwable e) {
-            previousHeartBeatSucceeded = false;
             // TODO Better error handling. Thread can die with the rest of the
             // NM still running.
             LOG.error("Caught exception in status-updater", e);

+ 37 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/ResourceLocalizationSpec.java

@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.nodemanager.api;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.URL;
+
+import com.google.common.annotations.VisibleForTesting;
+
+@Private
+@VisibleForTesting
+public interface ResourceLocalizationSpec {
+
+  public void setResource(LocalResource rsrc);
+
+  public LocalResource getResource();
+
+  public void setDestinationDirectory(URL destinationDirectory);
+
+  public URL getDestinationDirectory();
+}

+ 118 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/impl/pb/ResourceLocalizationSpecPBImpl.java

@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.nodemanager.api.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.ProtoBase;
+import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.URLPBImpl;
+import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.ResourceLocalizationSpecProto;
+import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.ResourceLocalizationSpecProtoOrBuilder;
+import org.apache.hadoop.yarn.server.nodemanager.api.ResourceLocalizationSpec;
+
+public class ResourceLocalizationSpecPBImpl extends
+    ProtoBase<ResourceLocalizationSpecProto> implements
+    ResourceLocalizationSpec {
+
+  private ResourceLocalizationSpecProto proto = ResourceLocalizationSpecProto
+    .getDefaultInstance();
+  private ResourceLocalizationSpecProto.Builder builder = null;
+  private boolean viaProto;
+  private LocalResource resource = null;
+  private URL destinationDirectory = null;
+
+  public ResourceLocalizationSpecPBImpl() {
+    builder = ResourceLocalizationSpecProto.newBuilder();
+  }
+
+  public ResourceLocalizationSpecPBImpl(ResourceLocalizationSpecProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  @Override
+  public LocalResource getResource() {
+    ResourceLocalizationSpecProtoOrBuilder p = viaProto ? proto : builder;
+    if (resource != null) {
+      return resource;
+    }
+    if (!p.hasResource()) {
+      return null;
+    }
+    resource = new LocalResourcePBImpl(p.getResource());
+    return resource;
+  }
+
+  @Override
+  public void setResource(LocalResource rsrc) {
+    maybeInitBuilder();
+    resource = rsrc;
+  }
+
+  @Override
+  public URL getDestinationDirectory() {
+    ResourceLocalizationSpecProtoOrBuilder p = viaProto ? proto : builder;
+    if (destinationDirectory != null) {
+      return destinationDirectory;
+    }
+    if (!p.hasDestinationDirectory()) {
+      return null;
+    }
+    destinationDirectory = new URLPBImpl(p.getDestinationDirectory());
+    return destinationDirectory;
+  }
+
+  @Override
+  public void setDestinationDirectory(URL destinationDirectory) {
+    maybeInitBuilder();
+    this.destinationDirectory = destinationDirectory;
+  }
+
+  @Override
+  public ResourceLocalizationSpecProto getProto() {
+    mergeLocalToBuilder();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private synchronized void maybeInitBuilder() {
+    if (builder == null || viaProto) {
+      builder = ResourceLocalizationSpecProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    ResourceLocalizationSpecProtoOrBuilder l = viaProto ? proto : builder;
+    if (this.resource != null
+        && !(l.getResource()
+          .equals(((LocalResourcePBImpl) resource).getProto()))) {
+      maybeInitBuilder();
+      builder.setResource(((LocalResourcePBImpl) resource).getProto());
+    }
+    if (this.destinationDirectory != null
+        && !(l.getDestinationDirectory()
+          .equals(((URLPBImpl) destinationDirectory).getProto()))) {
+      maybeInitBuilder();
+      builder.setDestinationDirectory(((URLPBImpl) destinationDirectory)
+        .getProto());
+    }
+  }
+}

+ 5 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/LocalizerHeartbeatResponse.java

@@ -18,18 +18,13 @@
 package org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords;
 
 import java.util.List;
-
-import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.server.nodemanager.api.*;
 
 public interface LocalizerHeartbeatResponse {
-  public LocalizerAction getLocalizerAction();
-  public List<LocalResource> getAllResources();
-  public LocalResource getLocalResource(int i);
 
+  public LocalizerAction getLocalizerAction();
   public void setLocalizerAction(LocalizerAction action);
 
-  public void addAllResources(List<LocalResource> resources);
-  public void addResource(LocalResource resource);
-  public void removeResource(int index);
-  public void clearResources();
-}
+  public List<ResourceLocalizationSpec> getResourceSpecs();
+  public void setResourceSpecs(List<ResourceLocalizationSpec> rsrcs);
+}

+ 40 - 56
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/LocalizerHeartbeatResponsePBImpl.java

@@ -21,13 +21,14 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.hadoop.yarn.api.records.LocalResource;
+
 import org.apache.hadoop.yarn.api.records.ProtoBase;
-import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerActionProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerHeartbeatResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerHeartbeatResponseProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.ResourceLocalizationSpecProto;
+import org.apache.hadoop.yarn.server.nodemanager.api.ResourceLocalizationSpec;
+import org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.ResourceLocalizationSpecPBImpl;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAction;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
 
@@ -40,13 +41,14 @@ public class LocalizerHeartbeatResponsePBImpl
   LocalizerHeartbeatResponseProto.Builder builder = null;
   boolean viaProto = false;
 
-  private List<LocalResource> resources;
+  private List<ResourceLocalizationSpec> resourceSpecs;
 
   public LocalizerHeartbeatResponsePBImpl() {
     builder = LocalizerHeartbeatResponseProto.newBuilder();
   }
 
-  public LocalizerHeartbeatResponsePBImpl(LocalizerHeartbeatResponseProto proto) {
+  public LocalizerHeartbeatResponsePBImpl(
+      LocalizerHeartbeatResponseProto proto) {
     this.proto = proto;
     viaProto = true;
   }
@@ -59,7 +61,7 @@ public class LocalizerHeartbeatResponsePBImpl
   }
 
   private void mergeLocalToBuilder() {
-    if (resources != null) {
+    if (resourceSpecs != null) {
       addResourcesToProto();
     }
   }
@@ -79,6 +81,7 @@ public class LocalizerHeartbeatResponsePBImpl
     viaProto = false;
   }
 
+  @Override
   public LocalizerAction getLocalizerAction() {
     LocalizerHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
     if (!p.hasAction()) {
@@ -87,14 +90,10 @@ public class LocalizerHeartbeatResponsePBImpl
     return convertFromProtoFormat(p.getAction());
   }
 
-  public List<LocalResource> getAllResources() {
-    initResources();
-    return this.resources;
-  }
-
-  public LocalResource getLocalResource(int i) {
+  @Override
+  public List<ResourceLocalizationSpec> getResourceSpecs() {
     initResources();
-    return this.resources.get(i);
+    return this.resourceSpecs;
   }
 
   public void setLocalizerAction(LocalizerAction action) {
@@ -106,31 +105,39 @@ public class LocalizerHeartbeatResponsePBImpl
     builder.setAction(convertToProtoFormat(action));
   }
 
+  public void setResourceSpecs(List<ResourceLocalizationSpec> rsrcs) {
+    maybeInitBuilder();
+    if (rsrcs == null) {
+      builder.clearResources();
+      return;
+    }
+    this.resourceSpecs = rsrcs;
+  }
+
   private void initResources() {
-    if (this.resources != null) {
+    if (this.resourceSpecs != null) {
       return;
     }
     LocalizerHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
-    List<LocalResourceProto> list = p.getResourcesList();
-    this.resources = new ArrayList<LocalResource>();
-
-    for (LocalResourceProto c : list) {
-      this.resources.add(convertFromProtoFormat(c));
+    List<ResourceLocalizationSpecProto> list = p.getResourcesList();
+    this.resourceSpecs = new ArrayList<ResourceLocalizationSpec>();
+    for (ResourceLocalizationSpecProto c : list) {
+      this.resourceSpecs.add(convertFromProtoFormat(c));
     }
   }
 
   private void addResourcesToProto() {
     maybeInitBuilder();
     builder.clearResources();
-    if (this.resources == null) 
+    if (this.resourceSpecs == null) 
       return;
-    Iterable<LocalResourceProto> iterable =
-        new Iterable<LocalResourceProto>() {
+    Iterable<ResourceLocalizationSpecProto> iterable =
+        new Iterable<ResourceLocalizationSpecProto>() {
       @Override
-      public Iterator<LocalResourceProto> iterator() {
-        return new Iterator<LocalResourceProto>() {
+      public Iterator<ResourceLocalizationSpecProto> iterator() {
+        return new Iterator<ResourceLocalizationSpecProto>() {
 
-          Iterator<LocalResource> iter = resources.iterator();
+          Iterator<ResourceLocalizationSpec> iter = resourceSpecs.iterator();
 
           @Override
           public boolean hasNext() {
@@ -138,8 +145,10 @@ public class LocalizerHeartbeatResponsePBImpl
           }
 
           @Override
-          public LocalResourceProto next() {
-            return convertToProtoFormat(iter.next());
+          public ResourceLocalizationSpecProto next() {
+            ResourceLocalizationSpec resource = iter.next();
+            
+            return ((ResourceLocalizationSpecPBImpl)resource).getProto();
           }
 
           @Override
@@ -154,34 +163,10 @@ public class LocalizerHeartbeatResponsePBImpl
     builder.addAllResources(iterable);
   }
 
-  public void addAllResources(List<LocalResource> resources) {
-    if (resources == null)
-      return;
-    initResources();
-    this.resources.addAll(resources);
-  }
 
-  public void addResource(LocalResource resource) {
-    initResources();
-    this.resources.add(resource);
-  }
-
-  public void removeResource(int index) {
-    initResources();
-    this.resources.remove(index);
-  }
-
-  public void clearResources() {
-    initResources();
-    this.resources.clear();
-  }
-
-  private LocalResource convertFromProtoFormat(LocalResourceProto p) {
-    return new LocalResourcePBImpl(p);
-  }
-
-  private LocalResourceProto convertToProtoFormat(LocalResource s) {
-    return ((LocalResourcePBImpl)s).getProto();
+  private ResourceLocalizationSpec convertFromProtoFormat(
+      ResourceLocalizationSpecProto p) {
+    return new ResourceLocalizationSpecPBImpl(p);
   }
 
   private LocalizerActionProto convertToProtoFormat(LocalizerAction a) {
@@ -191,5 +176,4 @@ public class LocalizerHeartbeatResponsePBImpl
   private LocalizerAction convertFromProtoFormat(LocalizerActionProto a) {
     return LocalizerAction.valueOf(a.name());
   }
-
-}
+}

+ 11 - 26
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java

@@ -51,6 +51,7 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
@@ -59,6 +60,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocol;
+import org.apache.hadoop.yarn.server.nodemanager.api.ResourceLocalizationSpec;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalResourceStatus;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus;
@@ -89,8 +91,6 @@ public class ContainerLocalizer {
   private final String localizerId;
   private final FileContext lfs;
   private final Configuration conf;
-  private final LocalDirAllocator appDirs;
-  private final LocalDirAllocator userDirs;
   private final RecordFactory recordFactory;
   private final Map<LocalResource,Future<Path>> pendingResources;
   private final String appCacheDirContextName;
@@ -112,8 +112,6 @@ public class ContainerLocalizer {
     this.recordFactory = recordFactory;
     this.conf = new Configuration();
     this.appCacheDirContextName = String.format(APPCACHE_CTXT_FMT, appId);
-    this.appDirs = new LocalDirAllocator(appCacheDirContextName);
-    this.userDirs = new LocalDirAllocator(String.format(USERCACHE_CTXT_FMT, user));
     this.pendingResources = new HashMap<LocalResource,Future<Path>>();
   }
 
@@ -197,10 +195,10 @@ public class ContainerLocalizer {
     return new ExecutorCompletionService<Path>(exec);
   }
 
-  Callable<Path> download(LocalDirAllocator lda, LocalResource rsrc,
+  Callable<Path> download(Path path, LocalResource rsrc,
       UserGroupInformation ugi) throws IOException {
-    Path destPath = lda.getLocalPathForWrite(".", getEstimatedSize(rsrc), conf);
-    return new FSDownload(lfs, ugi, conf, destPath, rsrc, new Random());
+    DiskChecker.checkDir(new File(path.toUri().getRawPath()));
+    return new FSDownload(lfs, ugi, conf, path, rsrc);
   }
 
   static long getEstimatedSize(LocalResource rsrc) {
@@ -238,25 +236,12 @@ public class ContainerLocalizer {
         LocalizerHeartbeatResponse response = nodemanager.heartbeat(status);
         switch (response.getLocalizerAction()) {
         case LIVE:
-          List<LocalResource> newResources = response.getAllResources();
-          for (LocalResource r : newResources) {
-            if (!pendingResources.containsKey(r)) {
-              final LocalDirAllocator lda;
-              switch (r.getVisibility()) {
-              default:
-                LOG.warn("Unknown visibility: " + r.getVisibility()
-                        + ", Using userDirs");
-                //Falling back to userDirs for unknown visibility.
-              case PUBLIC:
-              case PRIVATE:
-                lda = userDirs;
-                break;
-              case APPLICATION:
-                lda = appDirs;
-                break;
-              }
-              // TODO: Synchronization??
-              pendingResources.put(r, cs.submit(download(lda, r, ugi)));
+          List<ResourceLocalizationSpec> newRsrcs = response.getResourceSpecs();
+          for (ResourceLocalizationSpec newRsrc : newRsrcs) {
+            if (!pendingResources.containsKey(newRsrc.getResource())) {
+              pendingResources.put(newRsrc.getResource(), cs.submit(download(
+                new Path(newRsrc.getDestinationDirectory().getFile()),
+                newRsrc.getResource(), ugi)));
             }
           }
           break;

+ 0 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalCacheDirectoryManager.java

@@ -22,8 +22,6 @@ import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.Queue;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;

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

@@ -43,4 +43,5 @@ interface LocalResourcesTracker
   // TODO: Remove this in favour of EventHandler.handle
   void localizationCompleted(LocalResourceRequest req, boolean success);
 
+  long nextUniqueNumber();
 }

+ 12 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java

@@ -21,6 +21,7 @@ import java.io.File;
 import java.util.Iterator;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -66,6 +67,12 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
    */
   private ConcurrentHashMap<LocalResourceRequest, Path>
     inProgressLocalResourcesMap;
+  /*
+   * starting with 10 to accommodate 0-9 directories created as a part of
+   * LocalCacheDirectoryManager. So there will be one unique number generator
+   * per APPLICATION, USER and PUBLIC cache.
+   */
+  private AtomicLong uniqueNumberGenerator = new AtomicLong(9);
 
   public LocalResourcesTrackerImpl(String user, Dispatcher dispatcher,
       boolean useLocalCacheDirectoryManager, Configuration conf) {
@@ -283,4 +290,9 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
       }
     }
   }
+
+  @Override
+  public long nextUniqueNumber() {
+    return uniqueNumberGenerator.incrementAndGet();
+  }
 }

+ 97 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java

@@ -34,7 +34,6 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Random;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ConcurrentHashMap;
@@ -80,10 +79,12 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
 import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocol;
+import org.apache.hadoop.yarn.server.nodemanager.api.ResourceLocalizationSpec;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalResourceStatus;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAction;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus;
+import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.ResourceStatusType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
@@ -105,6 +106,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.even
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceRequestEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security.LocalizerTokenSecretManager;
 import org.apache.hadoop.yarn.server.nodemanager.security.authorize.NMPolicyProvider;
+import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerBuilderUtils;
 import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.service.CompositeService;
 import org.apache.hadoop.yarn.util.ConverterUtils;
@@ -326,7 +328,7 @@ public class ResourceLocalizationService extends CompositeService
     // 0) Create application tracking structs
     String userName = app.getUser();
     privateRsrc.putIfAbsent(userName, new LocalResourcesTrackerImpl(userName,
-      dispatcher, false, super.getConfig()));
+      dispatcher, true, super.getConfig()));
     if (null != appRsrc.putIfAbsent(
       ConverterUtils.toString(app.getAppId()),
       new LocalResourcesTrackerImpl(app.getUser(), dispatcher, false, super
@@ -476,6 +478,21 @@ public class ResourceLocalizationService extends CompositeService
     }
   }
 
+  private String getUserFileCachePath(String user) {
+    String path =
+        "." + Path.SEPARATOR + ContainerLocalizer.USERCACHE + Path.SEPARATOR
+            + user + Path.SEPARATOR + ContainerLocalizer.FILECACHE;
+    return path;
+  }
+
+  private String getUserAppCachePath(String user, String appId) {
+    String path =
+        "." + Path.SEPARATOR + ContainerLocalizer.USERCACHE + Path.SEPARATOR
+            + user + Path.SEPARATOR + ContainerLocalizer.APPCACHE
+            + Path.SEPARATOR + appId;
+    return path;
+  }
+
   /**
    * Sub-component handling the spawning of {@link ContainerLocalizer}s
    */
@@ -648,8 +665,11 @@ public class ResourceLocalizationService extends CompositeService
               DiskChecker.checkDir(
                 new File(publicDirDestPath.toUri().getPath()));
             }
+            publicDirDestPath =
+                new Path(publicDirDestPath, Long.toString(publicRsrc
+                  .nextUniqueNumber()));
             pending.put(queue.submit(new FSDownload(
-                lfs, null, conf, publicDirDestPath, resource, new Random())),
+                lfs, null, conf, publicDirDestPath, resource)),
                 request);
             attempts.put(key, new LinkedList<LocalizerResourceRequestEvent>());
           } catch (IOException e) {
@@ -803,7 +823,20 @@ public class ResourceLocalizationService extends CompositeService
         LocalResource next = findNextResource();
         if (next != null) {
           response.setLocalizerAction(LocalizerAction.LIVE);
-          response.addResource(next);
+          try {
+            ArrayList<ResourceLocalizationSpec> rsrcs =
+                new ArrayList<ResourceLocalizationSpec>();
+            ResourceLocalizationSpec rsrc =
+                NodeManagerBuilderUtils.newResourceLocalizationSpec(next,
+                  getPathForLocalization(next));
+            rsrcs.add(rsrc);
+            response.setResourceSpecs(rsrcs);
+          } catch (IOException e) {
+            LOG.error("local path for PRIVATE localization could not be found."
+                + "Disks might have failed.", e);
+          } catch (URISyntaxException e) {
+            // TODO fail? Already translated several times...
+          }
         } else if (pending.isEmpty()) {
           // TODO: Synchronization
           response.setLocalizerAction(LocalizerAction.DIE);
@@ -812,7 +845,8 @@ public class ResourceLocalizationService extends CompositeService
         }
         return response;
       }
-
+      ArrayList<ResourceLocalizationSpec> rsrcs =
+          new ArrayList<ResourceLocalizationSpec>();
       for (LocalResourceStatus stat : remoteResourceStatuses) {
         LocalResource rsrc = stat.getResource();
         LocalResourceRequest req = null;
@@ -835,6 +869,7 @@ public class ResourceLocalizationService extends CompositeService
                   new ResourceLocalizedEvent(req,
                     ConverterUtils.getPathFromYarnURL(stat.getLocalPath()),
                     stat.getLocalSize()));
+              localizationCompleted(stat);
             } catch (URISyntaxException e) { }
             if (pending.isEmpty()) {
               // TODO: Synchronization
@@ -844,7 +879,17 @@ public class ResourceLocalizationService extends CompositeService
             response.setLocalizerAction(LocalizerAction.LIVE);
             LocalResource next = findNextResource();
             if (next != null) {
-              response.addResource(next);
+              try {
+                ResourceLocalizationSpec resource =
+                    NodeManagerBuilderUtils.newResourceLocalizationSpec(next,
+                      getPathForLocalization(next));
+                rsrcs.add(resource);
+              } catch (IOException e) {
+                LOG.error("local path for PRIVATE localization could not be " +
+                  "found. Disks might have failed.", e);
+              } catch (URISyntaxException e) {
+                  //TODO fail? Already translated several times...
+              }
             }
             break;
           case FETCH_PENDING:
@@ -854,6 +899,7 @@ public class ResourceLocalizationService extends CompositeService
             LOG.info("DEBUG: FAILED " + req, stat.getException());
             assoc.getResource().unlock();
             response.setLocalizerAction(LocalizerAction.DIE);
+            localizationCompleted(stat);
             // TODO: Why is this event going directly to the container. Why not
             // the resource itself? What happens to the resource? Is it removed?
             dispatcher.getEventHandler().handle(
@@ -869,9 +915,53 @@ public class ResourceLocalizationService extends CompositeService
             break;
         }
       }
+      response.setResourceSpecs(rsrcs);
       return response;
     }
 
+    private void localizationCompleted(LocalResourceStatus stat) {
+      try {
+        LocalResource rsrc = stat.getResource();
+        LocalResourceRequest key = new LocalResourceRequest(rsrc);
+        String user = context.getUser();
+        ApplicationId appId =
+            context.getContainerId().getApplicationAttemptId()
+              .getApplicationId();
+        LocalResourceVisibility vis = rsrc.getVisibility();
+        LocalResourcesTracker tracker =
+            getLocalResourcesTracker(vis, user, appId);
+        if (stat.getStatus() == ResourceStatusType.FETCH_SUCCESS) {
+          tracker.localizationCompleted(key, true);
+        } else {
+          tracker.localizationCompleted(key, false);
+        }
+      } catch (URISyntaxException e) {
+        LOG.error("Invalid resource URL specified", e);
+      }
+    }
+
+    private Path getPathForLocalization(LocalResource rsrc) throws IOException,
+        URISyntaxException {
+      String user = context.getUser();
+      ApplicationId appId =
+          context.getContainerId().getApplicationAttemptId().getApplicationId();
+      LocalResourceVisibility vis = rsrc.getVisibility();
+      LocalResourcesTracker tracker =
+          getLocalResourcesTracker(vis, user, appId);
+      String cacheDirectory = null;
+      if (vis == LocalResourceVisibility.PRIVATE) {// PRIVATE Only
+        cacheDirectory = getUserFileCachePath(user);
+      } else {// APPLICATION ONLY
+        cacheDirectory = getUserAppCachePath(user, appId.toString());
+      }
+      Path dirPath =
+          dirsHandler.getLocalPathForWrite(cacheDirectory,
+            ContainerLocalizer.getEstimatedSize(rsrc), false);
+      dirPath = tracker.getPathForLocalization(new LocalResourceRequest(rsrc),
+        dirPath);
+      return new Path (dirPath, Long.toString(tracker.nextUniqueNumber()));
+    }
+
     @Override
     @SuppressWarnings("unchecked") // dispatcher not typed
     public void run() {
@@ -1033,4 +1123,4 @@ public class ResourceLocalizationService extends CompositeService
     del.delete(null, dirPath, new Path[] {});
   }
 
-}
+}

+ 39 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerBuilderUtils.java

@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.nodemanager.util;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.server.nodemanager.api.ResourceLocalizationSpec;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.Records;
+
+public class NodeManagerBuilderUtils {
+
+  public static ResourceLocalizationSpec newResourceLocalizationSpec(
+      LocalResource rsrc, Path path) {
+    URL local = ConverterUtils.getYarnUrlFromPath(path);
+    ResourceLocalizationSpec resourceLocalizationSpec =
+        Records.newRecord(ResourceLocalizationSpec.class);
+    resourceLocalizationSpec.setDestinationDirectory(local);
+    resourceLocalizationSpec.setResource(rsrc);
+    return resourceLocalizationSpec;
+  }
+
+}

+ 6 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_service_protos.proto

@@ -47,7 +47,12 @@ enum LocalizerActionProto {
   DIE = 2;
 }
 
+message ResourceLocalizationSpecProto {
+  optional LocalResourceProto resource = 1;
+  optional URLProto destination_directory = 2;
+}
+
 message LocalizerHeartbeatResponseProto {
   optional LocalizerActionProto action = 1;
-  repeated LocalResourceProto resources = 2;
+  repeated ResourceLocalizationSpecProto resources = 2;
 }

+ 98 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java

@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
@@ -51,9 +52,11 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.server.api.ResourceTracker;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
@@ -167,6 +170,10 @@ public class TestNodeStatusUpdater {
         throws YarnRemoteException {
       NodeStatus nodeStatus = request.getNodeStatus();
       LOG.info("Got heartbeat number " + heartBeatID);
+      NodeManagerMetrics mockMetrics = mock(NodeManagerMetrics.class);
+      Dispatcher mockDispatcher = mock(Dispatcher.class);
+      EventHandler mockEventHandler = mock(EventHandler.class);
+      when(mockDispatcher.getEventHandler()).thenReturn(mockEventHandler);
       nodeStatus.setResponseId(heartBeatID++);
       Map<ApplicationId, List<ContainerStatus>> appToContainers =
           getAppToContainerStatusMap(nodeStatus.getContainersStatuses());
@@ -183,7 +190,8 @@ public class TestNodeStatusUpdater {
         launchContext.setContainerId(firstContainerID);
         launchContext.setResource(recordFactory.newRecordInstance(Resource.class));
         launchContext.getResource().setMemory(2);
-        Container container = new ContainerImpl(conf , null, launchContext, null, null);
+        Container container = new ContainerImpl(conf , mockDispatcher,
+            launchContext, null, mockMetrics);
         this.context.getContainers().put(firstContainerID, container);
       } else if (heartBeatID == 2) {
         // Checks on the RM end
@@ -207,7 +215,8 @@ public class TestNodeStatusUpdater {
         launchContext.setContainerId(secondContainerID);
         launchContext.setResource(recordFactory.newRecordInstance(Resource.class));
         launchContext.getResource().setMemory(3);
-        Container container = new ContainerImpl(conf, null, launchContext, null, null);
+        Container container = new ContainerImpl(conf, mockDispatcher,
+            launchContext, null, mockMetrics);
         this.context.getContainers().put(secondContainerID, container);
       } else if (heartBeatID == 3) {
         // Checks on the RM end
@@ -229,13 +238,14 @@ public class TestNodeStatusUpdater {
   }
 
   private class MyNodeStatusUpdater extends NodeStatusUpdaterImpl {
-    public ResourceTracker resourceTracker = new MyResourceTracker(this.context);
+    public ResourceTracker resourceTracker;
     private Context context;
 
     public MyNodeStatusUpdater(Context context, Dispatcher dispatcher,
         NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
       super(context, dispatcher, healthChecker, metrics);
       this.context = context;
+      resourceTracker = new MyResourceTracker(this.context);
     }
 
     @Override
@@ -312,6 +322,21 @@ public class TestNodeStatusUpdater {
     }
   }
 
+  private class MyNodeStatusUpdater5 extends NodeStatusUpdaterImpl {
+    private ResourceTracker resourceTracker;
+
+    public MyNodeStatusUpdater5(Context context, Dispatcher dispatcher,
+        NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
+      super(context, dispatcher, healthChecker, metrics);
+      resourceTracker = new MyResourceTracker5();
+    }
+
+    @Override
+    protected ResourceTracker getRMClient() {
+      return resourceTracker;
+    }
+  }
+
   private class MyNodeManager extends NodeManager {
     
     private MyNodeStatusUpdater3 nodeStatusUpdater;
@@ -328,6 +353,32 @@ public class TestNodeStatusUpdater {
     }
   }
   
+  private class MyNodeManager2 extends NodeManager {
+    public boolean isStopped = false;
+    private NodeStatusUpdater nodeStatusUpdater;
+    private CyclicBarrier syncBarrier;
+    public MyNodeManager2 (CyclicBarrier syncBarrier) {
+      this.syncBarrier = syncBarrier;
+    }
+    @Override
+    protected NodeStatusUpdater createNodeStatusUpdater(Context context,
+        Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
+      nodeStatusUpdater =
+          new MyNodeStatusUpdater5(context, dispatcher, healthChecker,
+                                     metrics);
+      return nodeStatusUpdater;
+    }
+
+    @Override
+    public void stop() {
+      super.stop();
+      isStopped = true;
+      try {
+        syncBarrier.await();
+      } catch (Exception e) {
+      }
+    }
+  }
   // 
   private class MyResourceTracker2 implements ResourceTracker {
     public NodeAction heartBeatNodeAction = NodeAction.NORMAL;
@@ -505,6 +556,26 @@ public class TestNodeStatusUpdater {
     }
   }
 
+  private class MyResourceTracker5 implements ResourceTracker {
+    public NodeAction registerNodeAction = NodeAction.NORMAL;
+    @Override
+    public RegisterNodeManagerResponse registerNodeManager(
+        RegisterNodeManagerRequest request) throws YarnRemoteException {
+      
+      RegisterNodeManagerResponse response = recordFactory
+          .newRecordInstance(RegisterNodeManagerResponse.class);
+      response.setNodeAction(registerNodeAction );
+      return response;
+    }
+    
+    @Override
+    public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
+        throws YarnRemoteException {
+      heartBeatID++;
+      throw RPCUtil.getRemoteException("NodeHeartbeat exception");
+    }
+  }
+
   @Before
   public void clearError() {
     nmStartError = null;
@@ -883,6 +954,30 @@ public class TestNodeStatusUpdater {
     nm.stop();
   }
 
+  @Test(timeout = 20000)
+  public void testNodeStatusUpdaterRetryAndNMShutdown() 
+      throws InterruptedException {
+    final long connectionWaitSecs = 1;
+    final long connectionRetryIntervalSecs = 1;
+    YarnConfiguration conf = createNMConfig();
+    conf.setLong(YarnConfiguration.RESOURCEMANAGER_CONNECT_WAIT_SECS,
+        connectionWaitSecs);
+    conf.setLong(YarnConfiguration
+        .RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_SECS,
+        connectionRetryIntervalSecs);
+    CyclicBarrier syncBarrier = new CyclicBarrier(2);
+    nm = new MyNodeManager2(syncBarrier);
+    nm.init(conf);
+    nm.start();
+    try {
+      syncBarrier.await();
+    } catch (Exception e) {
+    }
+    Assert.assertTrue(((MyNodeManager2) nm).isStopped);
+    Assert.assertTrue("calculate heartBeatCount based on" +
+        " connectionWaitSecs and RetryIntervalSecs", heartBeatID == 2);
+  }
+
   private class MyNMContext extends NMContext {
     ConcurrentMap<ContainerId, Container> containers =
         new ConcurrentSkipListMap<ContainerId, Container>();

+ 28 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java

@@ -17,6 +17,13 @@
 */
 package org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataInputBuffer;
@@ -31,15 +38,14 @@ import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalResourceStatusProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerHeartbeatResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerStatusProto;
+import org.apache.hadoop.yarn.server.nodemanager.api.ResourceLocalizationSpec;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalResourceStatus;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAction;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.ResourceStatusType;
 import org.apache.hadoop.yarn.util.ConverterUtils;
-
 import org.junit.Test;
-import static org.junit.Assert.*;
 
 public class TestPBRecordImpl {
 
@@ -54,9 +60,8 @@ public class TestPBRecordImpl {
   static LocalResource createResource() {
     LocalResource ret = recordFactory.newRecordInstance(LocalResource.class);
     assertTrue(ret instanceof LocalResourcePBImpl);
-    ret.setResource(
-        ConverterUtils.getYarnUrlFromPath(
-          new Path("hdfs://y.ak:8020/foo/bar")));
+    ret.setResource(ConverterUtils.getYarnUrlFromPath(new Path(
+      "hdfs://y.ak:8020/foo/bar")));
     ret.setSize(4344L);
     ret.setTimestamp(3141592653589793L);
     ret.setVisibility(LocalResourceVisibility.PUBLIC);
@@ -90,16 +95,27 @@ public class TestPBRecordImpl {
     return ret;
   }
 
-  static LocalizerHeartbeatResponse createLocalizerHeartbeatResponse() {
+  static LocalizerHeartbeatResponse createLocalizerHeartbeatResponse() 
+      throws URISyntaxException {
     LocalizerHeartbeatResponse ret =
       recordFactory.newRecordInstance(LocalizerHeartbeatResponse.class);
     assertTrue(ret instanceof LocalizerHeartbeatResponsePBImpl);
     ret.setLocalizerAction(LocalizerAction.LIVE);
-    ret.addResource(createResource());
+    LocalResource rsrc = createResource();
+    ArrayList<ResourceLocalizationSpec> rsrcs =
+      new ArrayList<ResourceLocalizationSpec>();
+    ResourceLocalizationSpec resource =
+      recordFactory.newRecordInstance(ResourceLocalizationSpec.class);
+    resource.setResource(rsrc);
+    resource.setDestinationDirectory(ConverterUtils
+      .getYarnUrlFromPath(new Path("/tmp" + System.currentTimeMillis())));
+    rsrcs.add(resource);
+    ret.setResourceSpecs(rsrcs);
+    System.out.println(resource);
     return ret;
   }
 
-  @Test
+  @Test(timeout=10000)
   public void testLocalResourceStatusSerDe() throws Exception {
     LocalResourceStatus rsrcS = createLocalResourceStatus();
     assertTrue(rsrcS instanceof LocalResourceStatusPBImpl);
@@ -119,7 +135,7 @@ public class TestPBRecordImpl {
     assertEquals(createResource(), rsrcD.getResource());
   }
 
-  @Test
+  @Test(timeout=10000)
   public void testLocalizerStatusSerDe() throws Exception {
     LocalizerStatus rsrcS = createLocalizerStatus();
     assertTrue(rsrcS instanceof LocalizerStatusPBImpl);
@@ -141,7 +157,7 @@ public class TestPBRecordImpl {
     assertEquals(createLocalResourceStatus(), rsrcD.getResourceStatus(0));
   }
 
-  @Test
+  @Test(timeout=10000)
   public void testLocalizerHeartbeatResponseSerDe() throws Exception {
     LocalizerHeartbeatResponse rsrcS = createLocalizerHeartbeatResponse();
     assertTrue(rsrcS instanceof LocalizerHeartbeatResponsePBImpl);
@@ -158,8 +174,8 @@ public class TestPBRecordImpl {
       new LocalizerHeartbeatResponsePBImpl(rsrcPbD);
 
     assertEquals(rsrcS, rsrcD);
-    assertEquals(createResource(), rsrcS.getLocalResource(0));
-    assertEquals(createResource(), rsrcD.getLocalResource(0));
+    assertEquals(createResource(), rsrcS.getResourceSpecs().get(0).getResource());
+    assertEquals(createResource(), rsrcD.getResourceSpecs().get(0).getResource());
   }
 
 }

+ 14 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/MockLocalizerHeartbeatResponse.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.server.nodemanager.api.ResourceLocalizationSpec;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAction;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
 
@@ -28,28 +28,30 @@ public class MockLocalizerHeartbeatResponse
     implements LocalizerHeartbeatResponse {
 
   LocalizerAction action;
-  List<LocalResource> rsrc;
+  List<ResourceLocalizationSpec> resourceSpecs;
 
   MockLocalizerHeartbeatResponse() {
-    rsrc = new ArrayList<LocalResource>();
+    resourceSpecs = new ArrayList<ResourceLocalizationSpec>();
   }
 
   MockLocalizerHeartbeatResponse(
-      LocalizerAction action, List<LocalResource> rsrc) {
+      LocalizerAction action, List<ResourceLocalizationSpec> resources) {
     this.action = action;
-    this.rsrc = rsrc;
+    this.resourceSpecs = resources;
   }
 
   public LocalizerAction getLocalizerAction() { return action; }
-  public List<LocalResource> getAllResources() { return rsrc; }
-  public LocalResource getLocalResource(int i) { return rsrc.get(i); }
   public void setLocalizerAction(LocalizerAction action) {
     this.action = action;
   }
-  public void addAllResources(List<LocalResource> resources) {
-    rsrc.addAll(resources);
+
+  @Override
+  public List<ResourceLocalizationSpec> getResourceSpecs() {
+    return resourceSpecs;
+}
+
+  @Override
+  public void setResourceSpecs(List<ResourceLocalizationSpec> resourceSpecs) {
+    this.resourceSpecs = resourceSpecs;
   }
-  public void addResource(LocalResource resource) { rsrc.add(resource); }
-  public void removeResource(int index) { rsrc.remove(index); }
-  public void clearResources() { rsrc.clear(); }
 }

+ 58 - 26
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestContainerLocalizer.java

@@ -50,7 +50,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.AbstractFileSystem;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileContext;
-import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.DataInputBuffer;
@@ -66,9 +65,11 @@ import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocol;
+import org.apache.hadoop.yarn.server.nodemanager.api.ResourceLocalizationSpec;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalResourceStatus;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAction;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.Test;
 import org.mockito.ArgumentMatcher;
 import org.mockito.invocation.InvocationOnMock;
@@ -95,12 +96,33 @@ public class TestContainerLocalizer {
   public void testContainerLocalizerMain() throws Exception {
     ContainerLocalizer localizer = setupContainerLocalizerForTest();
 
+    // verify created cache
+    List<Path> privCacheList = new ArrayList<Path>();
+    List<Path> appCacheList = new ArrayList<Path>();
+    for (Path p : localDirs) {
+      Path base = new Path(new Path(p, ContainerLocalizer.USERCACHE), appUser);
+      Path privcache = new Path(base, ContainerLocalizer.FILECACHE);
+      privCacheList.add(privcache);
+      Path appDir =
+          new Path(base, new Path(ContainerLocalizer.APPCACHE, appId));
+      Path appcache = new Path(appDir, ContainerLocalizer.FILECACHE);
+      appCacheList.add(appcache);
+    }
+
     // mock heartbeat responses from NM
-    LocalResource rsrcA = getMockRsrc(random, LocalResourceVisibility.PRIVATE);
-    LocalResource rsrcB = getMockRsrc(random, LocalResourceVisibility.PRIVATE);
-    LocalResource rsrcC = getMockRsrc(random,
-        LocalResourceVisibility.APPLICATION);
-    LocalResource rsrcD = getMockRsrc(random, LocalResourceVisibility.PRIVATE);
+    ResourceLocalizationSpec rsrcA =
+        getMockRsrc(random, LocalResourceVisibility.PRIVATE,
+          privCacheList.get(0));
+    ResourceLocalizationSpec rsrcB =
+        getMockRsrc(random, LocalResourceVisibility.PRIVATE,
+          privCacheList.get(0));
+    ResourceLocalizationSpec rsrcC =
+        getMockRsrc(random, LocalResourceVisibility.APPLICATION,
+          appCacheList.get(0));
+    ResourceLocalizationSpec rsrcD =
+        getMockRsrc(random, LocalResourceVisibility.PRIVATE,
+          privCacheList.get(0));
+    
     when(nmProxy.heartbeat(isA(LocalizerStatus.class)))
       .thenReturn(new MockLocalizerHeartbeatResponse(LocalizerAction.LIVE,
             Collections.singletonList(rsrcA)))
@@ -111,27 +133,33 @@ public class TestContainerLocalizer {
       .thenReturn(new MockLocalizerHeartbeatResponse(LocalizerAction.LIVE,
             Collections.singletonList(rsrcD)))
       .thenReturn(new MockLocalizerHeartbeatResponse(LocalizerAction.LIVE,
-            Collections.<LocalResource>emptyList()))
+            Collections.<ResourceLocalizationSpec>emptyList()))
       .thenReturn(new MockLocalizerHeartbeatResponse(LocalizerAction.DIE,
             null));
 
-    doReturn(new FakeDownload(rsrcA.getResource().getFile(), true)).when(
-        localizer).download(isA(LocalDirAllocator.class), eq(rsrcA),
+    LocalResource tRsrcA = rsrcA.getResource();
+    LocalResource tRsrcB = rsrcB.getResource();
+    LocalResource tRsrcC = rsrcC.getResource();
+    LocalResource tRsrcD = rsrcD.getResource();
+    doReturn(
+      new FakeDownload(rsrcA.getResource().getResource().getFile(), true))
+      .when(localizer).download(isA(Path.class), eq(tRsrcA),
         isA(UserGroupInformation.class));
-    doReturn(new FakeDownload(rsrcB.getResource().getFile(), true)).when(
-        localizer).download(isA(LocalDirAllocator.class), eq(rsrcB),
+    doReturn(
+      new FakeDownload(rsrcB.getResource().getResource().getFile(), true))
+      .when(localizer).download(isA(Path.class), eq(tRsrcB),
         isA(UserGroupInformation.class));
-    doReturn(new FakeDownload(rsrcC.getResource().getFile(), true)).when(
-        localizer).download(isA(LocalDirAllocator.class), eq(rsrcC),
+    doReturn(
+      new FakeDownload(rsrcC.getResource().getResource().getFile(), true))
+      .when(localizer).download(isA(Path.class), eq(tRsrcC),
         isA(UserGroupInformation.class));
-    doReturn(new FakeDownload(rsrcD.getResource().getFile(), true)).when(
-        localizer).download(isA(LocalDirAllocator.class), eq(rsrcD),
+    doReturn(
+      new FakeDownload(rsrcD.getResource().getResource().getFile(), true))
+      .when(localizer).download(isA(Path.class), eq(tRsrcD),
         isA(UserGroupInformation.class));
 
     // run localization
     assertEquals(0, localizer.runLocalization(nmAddr));
-
-    // verify created cache
     for (Path p : localDirs) {
       Path base = new Path(new Path(p, ContainerLocalizer.USERCACHE), appUser);
       Path privcache = new Path(base, ContainerLocalizer.FILECACHE);
@@ -143,15 +171,14 @@ public class TestContainerLocalizer {
       Path appcache = new Path(appDir, ContainerLocalizer.FILECACHE);
       verify(spylfs).mkdir(eq(appcache), isA(FsPermission.class), eq(false));
     }
-
     // verify tokens read at expected location
     verify(spylfs).open(tokenPath);
 
     // verify downloaded resources reported to NM
-    verify(nmProxy).heartbeat(argThat(new HBMatches(rsrcA)));
-    verify(nmProxy).heartbeat(argThat(new HBMatches(rsrcB)));
-    verify(nmProxy).heartbeat(argThat(new HBMatches(rsrcC)));
-    verify(nmProxy).heartbeat(argThat(new HBMatches(rsrcD)));
+    verify(nmProxy).heartbeat(argThat(new HBMatches(rsrcA.getResource())));
+    verify(nmProxy).heartbeat(argThat(new HBMatches(rsrcB.getResource())));
+    verify(nmProxy).heartbeat(argThat(new HBMatches(rsrcC.getResource())));
+    verify(nmProxy).heartbeat(argThat(new HBMatches(rsrcD.getResource())));
 
     // verify all HB use localizerID provided
     verify(nmProxy, never()).heartbeat(argThat(
@@ -306,10 +333,12 @@ public class TestContainerLocalizer {
     return mockRF;
   }
 
-  static LocalResource getMockRsrc(Random r,
-      LocalResourceVisibility vis) {
-    LocalResource rsrc = mock(LocalResource.class);
+  static ResourceLocalizationSpec getMockRsrc(Random r,
+      LocalResourceVisibility vis, Path p) {
+    ResourceLocalizationSpec resourceLocalizationSpec =
+      mock(ResourceLocalizationSpec.class);
 
+    LocalResource rsrc = mock(LocalResource.class);
     String name = Long.toHexString(r.nextLong());
     URL uri = mock(org.apache.hadoop.yarn.api.records.URL.class);
     when(uri.getScheme()).thenReturn("file");
@@ -322,7 +351,10 @@ public class TestContainerLocalizer {
     when(rsrc.getType()).thenReturn(LocalResourceType.FILE);
     when(rsrc.getVisibility()).thenReturn(vis);
 
-    return rsrc;
+    when(resourceLocalizationSpec.getResource()).thenReturn(rsrc);
+    when(resourceLocalizationSpec.getDestinationDirectory()).
+      thenReturn(ConverterUtils.getYarnUrlFromPath(p));
+    return resourceLocalizationSpec;
   }
 
   @SuppressWarnings({ "rawtypes", "unchecked" })

+ 73 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Matchers.anyLong;
@@ -35,6 +36,7 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.timeout;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.times;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -375,7 +377,7 @@ public class TestResourceLocalizationService {
     }
   }
   
-  @Test
+  @Test( timeout = 10000)
   @SuppressWarnings("unchecked") // mocked generics
   public void testLocalizationHeartbeat() throws Exception {
     Configuration conf = new YarnConfiguration();
@@ -386,12 +388,17 @@ public class TestResourceLocalizationService {
         isA(Path.class), isA(FsPermission.class), anyBoolean());
 
     List<Path> localDirs = new ArrayList<Path>();
-    String[] sDirs = new String[4];
-    for (int i = 0; i < 4; ++i) {
-      localDirs.add(lfs.makeQualified(new Path(basedir, i + "")));
-      sDirs[i] = localDirs.get(i).toString();
-    }
+    String[] sDirs = new String[1];
+    // Making sure that we have only one local disk so that it will only be
+    // selected for consecutive resource localization calls.  This is required
+    // to test LocalCacheDirectoryManager.
+    localDirs.add(lfs.makeQualified(new Path(basedir, 0 + "")));
+    sDirs[0] = localDirs.get(0).toString();
+
     conf.setStrings(YarnConfiguration.NM_LOCAL_DIRS, sDirs);
+    // Adding configuration to make sure there is only one file per
+    // directory
+    conf.set(YarnConfiguration.NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY, "37");
     String logDir = lfs.makeQualified(new Path(basedir, "logdir " )).toString();
     conf.set(YarnConfiguration.NM_LOG_DIRS, logDir);
     DrainDispatcher dispatcher = new DrainDispatcher();
@@ -452,12 +459,23 @@ public class TestResourceLocalizationService {
       doReturn(out).when(spylfs).createInternal(isA(Path.class),
           isA(EnumSet.class), isA(FsPermission.class), anyInt(), anyShort(),
           anyLong(), isA(Progressable.class), isA(ChecksumOpt.class), anyBoolean());
-      final LocalResource resource = getPrivateMockedResource(r);
-      final LocalResourceRequest req = new LocalResourceRequest(resource);
+      final LocalResource resource1 = getPrivateMockedResource(r);
+      LocalResource resource2 = null;
+      do {
+        resource2 = getPrivateMockedResource(r);
+      } while (resource2 == null || resource2.equals(resource1));
+      // above call to make sure we don't get identical resources.
+      
+      final LocalResourceRequest req1 = new LocalResourceRequest(resource1);
+      final LocalResourceRequest req2 = new LocalResourceRequest(resource2);
       Map<LocalResourceVisibility, Collection<LocalResourceRequest>> rsrcs =
         new HashMap<LocalResourceVisibility, 
                     Collection<LocalResourceRequest>>();
-      rsrcs.put(LocalResourceVisibility.PRIVATE, Collections.singletonList(req));
+      List<LocalResourceRequest> privateResourceList =
+          new ArrayList<LocalResourceRequest>();
+      privateResourceList.add(req1);
+      privateResourceList.add(req2);
+      rsrcs.put(LocalResourceVisibility.PRIVATE, privateResourceList);
       spyService.handle(new ContainerLocalizationRequestEvent(c, rsrcs));
       // Sigh. Thread init of private localizer not accessible
       Thread.sleep(1000);
@@ -471,33 +489,67 @@ public class TestResourceLocalizationService {
       Path localizationTokenPath = tokenPathCaptor.getValue();
 
       // heartbeat from localizer
-      LocalResourceStatus rsrcStat = mock(LocalResourceStatus.class);
+      LocalResourceStatus rsrcStat1 = mock(LocalResourceStatus.class);
+      LocalResourceStatus rsrcStat2 = mock(LocalResourceStatus.class);
       LocalizerStatus stat = mock(LocalizerStatus.class);
       when(stat.getLocalizerId()).thenReturn(ctnrStr);
-      when(rsrcStat.getResource()).thenReturn(resource);
-      when(rsrcStat.getLocalSize()).thenReturn(4344L);
+      when(rsrcStat1.getResource()).thenReturn(resource1);
+      when(rsrcStat2.getResource()).thenReturn(resource2);
+      when(rsrcStat1.getLocalSize()).thenReturn(4344L);
+      when(rsrcStat2.getLocalSize()).thenReturn(2342L);
       URL locPath = getPath("/cache/private/blah");
-      when(rsrcStat.getLocalPath()).thenReturn(locPath);
-      when(rsrcStat.getStatus()).thenReturn(ResourceStatusType.FETCH_SUCCESS);
+      when(rsrcStat1.getLocalPath()).thenReturn(locPath);
+      when(rsrcStat2.getLocalPath()).thenReturn(locPath);
+      when(rsrcStat1.getStatus()).thenReturn(ResourceStatusType.FETCH_SUCCESS);
+      when(rsrcStat2.getStatus()).thenReturn(ResourceStatusType.FETCH_SUCCESS);
       when(stat.getResources())
         .thenReturn(Collections.<LocalResourceStatus>emptyList())
-        .thenReturn(Collections.singletonList(rsrcStat))
+        .thenReturn(Collections.singletonList(rsrcStat1))
+        .thenReturn(Collections.singletonList(rsrcStat2))
         .thenReturn(Collections.<LocalResourceStatus>emptyList());
 
-      // get rsrc
+      String localPath = Path.SEPARATOR + ContainerLocalizer.USERCACHE +
+          Path.SEPARATOR + "user0" + Path.SEPARATOR +
+          ContainerLocalizer.FILECACHE;
+      
+      // get first resource
       LocalizerHeartbeatResponse response = spyService.heartbeat(stat);
       assertEquals(LocalizerAction.LIVE, response.getLocalizerAction());
-      assertEquals(req, new LocalResourceRequest(response.getLocalResource(0)));
+      assertEquals(1, response.getResourceSpecs().size());
+      assertEquals(req1,
+        new LocalResourceRequest(response.getResourceSpecs().get(0).getResource()));
+      URL localizedPath =
+          response.getResourceSpecs().get(0).getDestinationDirectory();
+      // Appending to local path unique number(10) generated as a part of
+      // LocalResourcesTracker
+      assertTrue(localizedPath.getFile().endsWith(
+        localPath + Path.SEPARATOR + "10"));
+
+      // get second resource
+      response = spyService.heartbeat(stat);
+      assertEquals(LocalizerAction.LIVE, response.getLocalizerAction());
+      assertEquals(1, response.getResourceSpecs().size());
+      assertEquals(req2, new LocalResourceRequest(response.getResourceSpecs()
+        .get(0).getResource()));
+      localizedPath =
+          response.getResourceSpecs().get(0).getDestinationDirectory();
+      // Resource's destination path should be now inside sub directory 0 as
+      // LocalCacheDirectoryManager will be used and we have restricted number
+      // of files per directory to 1.
+      assertTrue(localizedPath.getFile().endsWith(
+        localPath + Path.SEPARATOR + "0" + Path.SEPARATOR + "11"));
 
       // empty rsrc
       response = spyService.heartbeat(stat);
       assertEquals(LocalizerAction.LIVE, response.getLocalizerAction());
-      assertEquals(0, response.getAllResources().size());
+      assertEquals(0, response.getResourceSpecs().size());
 
       // get shutdown
       response = spyService.heartbeat(stat);
       assertEquals(LocalizerAction.DIE, response.getLocalizerAction());
 
+
+      dispatcher.await();
       // verify container notification
       ArgumentMatcher<ContainerEvent> matchesContainerLoc =
         new ArgumentMatcher<ContainerEvent>() {
@@ -508,9 +560,9 @@ public class TestResourceLocalizationService {
               && c.getContainerID() == evt.getContainerID();
           }
         };
-      dispatcher.await();
-      verify(containerBus).handle(argThat(matchesContainerLoc));
-      
+      // total 2 resource localzation calls. one for each resource.
+      verify(containerBus, times(2)).handle(argThat(matchesContainerLoc));
+        
       // Verify deletion of localization token.
       verify(delService).delete((String)isNull(), eq(localizationTokenPath));
     } finally {

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

@@ -54,15 +54,17 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStatusupdateEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
 import org.apache.hadoop.yarn.service.AbstractService;
@@ -276,6 +278,14 @@ public class ApplicationMasterService extends AbstractService implements
       List<ResourceRequest> ask = request.getAskList();
       List<ContainerId> release = request.getReleaseList();
 
+      // sanity check
+      try {
+        SchedulerUtils.validateResourceRequests(ask,
+            rScheduler.getMaximumResourceCapability());
+      } catch (InvalidResourceRequestException e) {
+        LOG.warn("Invalid resource ask by application " + appAttemptId, e);
+        throw RPCUtil.getRemoteException(e);
+      }
       // Send new requests to appAttempt.
       Allocation allocation =
           this.rScheduler.allocate(appAttemptId, ask, release);

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

@@ -72,6 +72,7 @@ import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
@@ -86,8 +87,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
@@ -273,6 +277,21 @@ public class ClientRMService extends AbstractService implements
       // Safety 
       submissionContext.setUser(user);
 
+      // Check whether AM resource requirements are within required limits
+      if (!submissionContext.getUnmanagedAM()) {
+        ResourceRequest amReq = BuilderUtils.newResourceRequest(
+            RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
+            submissionContext.getAMContainerSpec().getResource(), 1);
+        try {
+          SchedulerUtils.validateResourceRequest(amReq,
+              scheduler.getMaximumResourceCapability());
+        } catch (InvalidResourceRequestException e) {
+          LOG.warn("RM app submission failed in validating AM resource request"
+              + " for application " + applicationId, e);
+          throw RPCUtil.getRemoteException(e);
+        }
+      }
+
       // This needs to be synchronous as the client can query 
       // immediately following the submission to get the application status.
       // So call handle directly and do not send an event.

+ 30 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java

@@ -57,6 +57,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
   private static final Log LOG = LogFactory.getLog(RMAppManager.class);
 
   private int completedAppsMax = YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS;
+  private int globalMaxAppAttempts;
   private LinkedList<ApplicationId> completedApps = new LinkedList<ApplicationId>();
 
   private final RMContext rmContext;
@@ -76,6 +77,8 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     setCompletedAppsMax(conf.getInt(
         YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS,
         YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS));
+    globalMaxAppAttempts = conf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+        YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
   }
 
   /**
@@ -308,6 +311,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     Map<ApplicationId, ApplicationState> appStates = state.getApplicationState();
     LOG.info("Recovering " + appStates.size() + " applications");
     for(ApplicationState appState : appStates.values()) {
+      boolean shouldRecover = true;
       // re-submit the application
       // this is going to send an app start event but since the async dispatcher 
       // has not started that event will be queued until we have completed re
@@ -318,16 +322,39 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
         // This will need to be changed in work preserving recovery in which 
         // RM will re-connect with the running AM's instead of restarting them
         LOG.info("Not recovering unmanaged application " + appState.getAppId());
-        store.removeApplication(appState);
+        shouldRecover = false;
+      }
+      int individualMaxAppAttempts = appState.getApplicationSubmissionContext()
+          .getMaxAppAttempts();
+      int maxAppAttempts;
+      if (individualMaxAppAttempts <= 0 ||
+          individualMaxAppAttempts > globalMaxAppAttempts) {
+        maxAppAttempts = globalMaxAppAttempts;
+        LOG.warn("The specific max attempts: " + individualMaxAppAttempts
+            + " for application: " + appState.getAppId()
+            + " is invalid, because it is out of the range [1, "
+            + globalMaxAppAttempts + "]. Use the global max attempts instead.");
       } else {
+        maxAppAttempts = individualMaxAppAttempts;
+      }
+      if(appState.getAttemptCount() >= maxAppAttempts) {
+        LOG.info("Not recovering application " + appState.getAppId() +
+            " due to recovering attempt is beyond maxAppAttempt limit");
+        shouldRecover = false;
+      }
+
+      if(shouldRecover) {
         LOG.info("Recovering application " + appState.getAppId());
         submitApplication(appState.getApplicationSubmissionContext(), 
-                          appState.getSubmitTime());
+                        appState.getSubmitTime());
         // re-populate attempt information in application
         RMAppImpl appImpl = (RMAppImpl) rmContext.getRMApps().get(
-                                                          appState.getAppId());
+                                                        appState.getAppId());
         appImpl.recover(state);
       }
+      else {
+        store.removeApplication(appState);
+      }
     }
   }
 

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

@@ -327,12 +327,52 @@ public class ResourceManager extends CompositeService implements Recoverable {
       this.applicationACLsManager, this.conf);
   }
 
+  // sanity check for configurations
   protected static void validateConfigs(Configuration conf) {
-    int globalMaxAppAttempts = conf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+    // validate max-attempts
+    int globalMaxAppAttempts =
+        conf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
     if (globalMaxAppAttempts <= 0) {
-      throw new YarnException(
-          "The global max attempts should be a positive integer.");
+      throw new YarnException("Invalid global max attempts configuration"
+          + ", " + YarnConfiguration.RM_AM_MAX_ATTEMPTS
+          + "=" + globalMaxAppAttempts + ", it should be a positive integer.");
+    }
+
+    // validate scheduler memory allocation setting
+    int minMem = conf.getInt(
+        YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
+    int maxMem = conf.getInt(
+        YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
+
+    if (minMem <= 0 || minMem > maxMem) {
+      throw new YarnException("Invalid resource scheduler memory"
+          + " allocation configuration"
+          + ", " + YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB
+          + "=" + minMem
+          + ", " + YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB
+          + "=" + maxMem + ", min and max should be greater than 0"
+          + ", max should be no smaller than min.");
+    }
+
+    // validate scheduler vcores allocation setting
+    int minVcores = conf.getInt(
+        YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+    int maxVcores = conf.getInt(
+        YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
+
+    if (minVcores <= 0 || minVcores > maxVcores) {
+      throw new YarnException("Invalid resource scheduler vcores"
+          + " allocation configuration"
+          + ", " + YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES
+          + "=" + minVcores
+          + ", " + YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES
+          + "=" + maxVcores + ", min and max should be greater than 0"
+          + ", max should be no smaller than min.");
     }
   }
 

+ 6 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DefaultResourceCalculator.java

@@ -53,12 +53,14 @@ public class DefaultResourceCalculator extends ResourceCalculator {
   }
 
   @Override
-  public Resource normalize(Resource r, Resource minimumResource) {
-    return Resources.createResource(
+  public Resource normalize(Resource r, Resource minimumResource,
+      Resource maximumResource) {
+    int normalizedMemory = Math.min(
         roundUp(
             Math.max(r.getMemory(), minimumResource.getMemory()),
-            minimumResource.getMemory())
-        );
+            minimumResource.getMemory()),
+            maximumResource.getMemory());
+    return Resources.createResource(normalizedMemory);
   }
 
   @Override

+ 10 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DominantResourceCalculator.java

@@ -123,15 +123,20 @@ public class DominantResourceCalculator extends ResourceCalculator {
   }
 
   @Override
-  public Resource normalize(Resource r, Resource minimumResource) {
-    return Resources.createResource(
+  public Resource normalize(Resource r, Resource minimumResource,
+      Resource maximumResource) {
+    int normalizedMemory = Math.min(
         roundUp(
-            Math.max(r.getMemory(), minimumResource.getMemory()), 
+            Math.max(r.getMemory(), minimumResource.getMemory()),
             minimumResource.getMemory()),
+            maximumResource.getMemory());
+    int normalizedCores = Math.min(
         roundUp(
             Math.max(r.getVirtualCores(), minimumResource.getVirtualCores()),
-            minimumResource.getVirtualCores())
-        );
+            minimumResource.getVirtualCores()),
+            maximumResource.getVirtualCores());
+    return Resources.createResource(normalizedMemory,
+        normalizedCores);
   }
 
   @Override

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

@@ -88,13 +88,16 @@ public abstract class ResourceCalculator {
   
   /**
    * Normalize resource <code>r</code> given the base 
-   * <code>minimumResource</code>.
+   * <code>minimumResource</code> and verify against max allowed
+   * <code>maximumResource</code>
    * 
    * @param r resource
    * @param minimumResource step-factor
+   * @param maximumResource the upper bound of the resource to be allocated
    * @return normalized resource
    */
-  public abstract Resource normalize(Resource r, Resource minimumResource);
+  public abstract Resource normalize(Resource r, Resource minimumResource,
+      Resource maximumResource);
 
   /**
    * Round-up resource <code>r</code> given factor <code>stepFactor</code>.

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java

@@ -132,8 +132,9 @@ public class Resources {
   }
   
   public static Resource normalize(
-      ResourceCalculator calculator, Resource lhs, Resource factor) {
-    return calculator.normalize(lhs, factor);
+      ResourceCalculator calculator, Resource lhs, Resource factor,
+      Resource limit) {
+    return calculator.normalize(lhs, factor, limit);
   }
   
   public static Resource roundUp(

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -740,6 +740,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
             AM_CONTAINER_PRIORITY, ResourceRequest.ANY, appAttempt.submissionContext
                 .getAMContainerSpec().getResource(), 1);
 
+        // SchedulerUtils.validateResourceRequests is not necessary because
+        // AM resource has been checked when submission
         Allocation amContainerAllocation = appAttempt.scheduler.allocate(
             appAttempt.applicationAttemptId,
             Collections.singletonList(request), EMPTY_CONTAINER_RELEASE_LIST);

+ 42 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/InvalidResourceRequestException.java

@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
+
+import org.apache.hadoop.yarn.YarnException;
+
+/**
+ * The exception is thrown when the requested resource is out of the range
+ * of the configured lower and upper resource boundaries.
+ *
+ */
+public class InvalidResourceRequestException extends YarnException {
+
+  public InvalidResourceRequestException(Throwable cause) {
+    super(cause);
+  }
+
+  public InvalidResourceRequestException(String message) {
+    super(message);
+  }
+
+  public InvalidResourceRequestException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+}

+ 45 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java

@@ -89,10 +89,12 @@ public class SchedulerUtils {
       List<ResourceRequest> asks,
       ResourceCalculator resourceCalculator, 
       Resource clusterResource,
-      Resource minimumResource) {
+      Resource minimumResource,
+      Resource maximumResource) {
     for (ResourceRequest ask : asks) {
       normalizeRequest(
-          ask, resourceCalculator, clusterResource, minimumResource);
+          ask, resourceCalculator, clusterResource, minimumResource,
+          maximumResource);
     }
   }
 
@@ -104,11 +106,50 @@ public class SchedulerUtils {
       ResourceRequest ask, 
       ResourceCalculator resourceCalculator, 
       Resource clusterResource,
-      Resource minimumResource) {
+      Resource minimumResource,
+      Resource maximumResource) {
     Resource normalized = 
         Resources.normalize(
-            resourceCalculator, ask.getCapability(), minimumResource);
+            resourceCalculator, ask.getCapability(), minimumResource,
+            maximumResource);
     ask.setCapability(normalized);
   }
 
+  /**
+   * Utility method to validate a resource request, by insuring that the
+   * requested memory/vcore is non-negative and not greater than max
+   */
+  public static void validateResourceRequest(ResourceRequest resReq,
+      Resource maximumResource) throws InvalidResourceRequestException {
+    if (resReq.getCapability().getMemory() < 0 ||
+        resReq.getCapability().getMemory() > maximumResource.getMemory()) {
+      throw new InvalidResourceRequestException("Invalid resource request"
+          + ", requested memory < 0"
+          + ", or requested memory > max configured"
+          + ", requestedMemory=" + resReq.getCapability().getMemory()
+          + ", maxMemory=" + maximumResource.getMemory());
+    }
+    if (resReq.getCapability().getVirtualCores() < 0 ||
+        resReq.getCapability().getVirtualCores() >
+        maximumResource.getVirtualCores()) {
+      throw new InvalidResourceRequestException("Invalid resource request"
+          + ", requested virtual cores < 0"
+          + ", or requested virtual cores > max configured"
+          + ", requestedVirtualCores="
+          + resReq.getCapability().getVirtualCores()
+          + ", maxVirtualCores=" + maximumResource.getVirtualCores());
+    }
+  }
+
+  /**
+   * Utility method to validate a list resource requests, by insuring that the
+   * requested memory/vcore is non-negative and not greater than max
+   */
+  public static void validateResourceRequests(List<ResourceRequest> ask,
+      Resource maximumResource) throws InvalidResourceRequestException {
+    for (ResourceRequest resReq : ask) {
+      validateResourceRequest(resReq, maximumResource);
+    }
+  }
+
 }

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

@@ -483,7 +483,8 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
     
     // Sanity check
     SchedulerUtils.normalizeRequests(
-        ask, calculator, getClusterResources(), minimumAllocation);
+        ask, calculator, getClusterResources(), minimumAllocation,
+        maximumAllocation);
 
     // Release containers
     for (ContainerId releasedContainerId : release) {

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java

@@ -315,8 +315,8 @@ public class CapacitySchedulerConfiguration extends Configuration {
         YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
     int maximumCores = getInt(
-        YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_CORES,
-        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_CORES);
+        YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
     return Resources.createResource(maximumMemory, maximumCores);
   }
 

Một số tệp đã không được hiển thị bởi vì quá nhiều tập tin thay đổi trong này khác