Bladeren bron

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 jaren geleden
bovenliggende
commit
3209d4a3c4
100 gewijzigde bestanden met toevoegingen van 2947 en 572 verwijderingen
  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.
     HADOOP-9353. Activate native-win maven profile by default on Windows.
     (Arpit Agarwal via szetszwo)
     (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
 Release 2.0.5-beta - UNRELEASED
 
 
@@ -625,6 +629,9 @@ Release 2.0.4-alpha - UNRELEASED
 
 
   BUG FIXES
   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-9406. hadoop-client leaks dependency on JDK tools jar. (tucu)
 
 
     HADOOP-9301. hadoop client servlet/jsp/jetty/tomcat JARs creating
     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.
    * @return  true to accept; false otherwise.
    */
    */
   public boolean accepts(MetricsRecord record) {
   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, 
 Java_org_apache_hadoop_io_nativeio_NativeIO_renameTo0(JNIEnv *env, 
 jclass clazz, jstring jsrc, jstring jdst)
 jclass clazz, jstring jsrc, jstring jdst)
 {
 {
+#ifdef UNIX
   const char *src = NULL, *dst = NULL;
   const char *src = NULL, *dst = NULL;
   
   
   src = (*env)->GetStringUTFChars(env, jsrc, NULL);
   src = (*env)->GetStringUTFChars(env, jsrc, NULL);
@@ -829,6 +830,23 @@ jclass clazz, jstring jsrc, jstring jdst)
 done:
 done:
   if (src) (*env)->ReleaseStringUTFChars(env, jsrc, src);
   if (src) (*env)->ReleaseStringUTFChars(env, jsrc, src);
   if (dst) (*env)->ReleaseStringUTFChars(env, jdst, dst);
   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);
       NativeIO.renameTo(nonExistentFile, targetFile);
       Assert.fail();
       Assert.fail();
     } catch (NativeIOException e) {
     } 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.
     // Test renaming a file to itself.  It should succeed and do nothing.
@@ -465,7 +471,13 @@ public class TestNativeIO {
       NativeIO.renameTo(sourceFile, badTarget);
       NativeIO.renameTo(sourceFile, badTarget);
       Assert.fail();
       Assert.fail();
     } catch (NativeIOException e) {
     } 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);
     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.apache.commons.configuration.SubsetConfiguration;
 import org.junit.Test;
 import org.junit.Test;
 import static org.junit.Assert.*;
 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.MetricsTag;
 import org.apache.hadoop.metrics2.impl.ConfigBuilder;
 import org.apache.hadoop.metrics2.impl.ConfigBuilder;
 import static org.apache.hadoop.metrics2.lib.Interns.*;
 import static org.apache.hadoop.metrics2.lib.Interns.*;
@@ -38,6 +40,8 @@ public class TestPatternFilter {
     SubsetConfiguration empty = new ConfigBuilder().subset("");
     SubsetConfiguration empty = new ConfigBuilder().subset("");
     shouldAccept(empty, "anything");
     shouldAccept(empty, "anything");
     shouldAccept(empty, Arrays.asList(tag("key", "desc", "value")));
     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, "foo");
     shouldAccept(wl, Arrays.asList(tag("bar", "", ""),
     shouldAccept(wl, Arrays.asList(tag("bar", "", ""),
                                    tag("foo", "", "f")));
                                    tag("foo", "", "f")));
+    shouldAccept(wl, mockMetricsRecord("foo", Arrays.asList(
+      tag("bar", "", ""), tag("foo", "", "f"))));
     shouldReject(wl, "bar");
     shouldReject(wl, "bar");
     shouldReject(wl, Arrays.asList(tag("bar", "", "")));
     shouldReject(wl, Arrays.asList(tag("bar", "", "")));
     shouldReject(wl, Arrays.asList(tag("foo", "", "boo")));
     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");
         .add("p.exclude.tags", "foo:f").subset("p");
     shouldAccept(bl, "bar");
     shouldAccept(bl, "bar");
     shouldAccept(bl, Arrays.asList(tag("bar", "", "")));
     shouldAccept(bl, Arrays.asList(tag("bar", "", "")));
+    shouldAccept(bl, mockMetricsRecord("bar", Arrays.asList(
+      tag("bar", "", ""))));
     shouldReject(bl, "foo");
     shouldReject(bl, "foo");
     shouldReject(bl, Arrays.asList(tag("bar", "", ""),
     shouldReject(bl, Arrays.asList(tag("bar", "", ""),
                                    tag("foo", "", "f")));
                                    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");
         .add("p.exclude.tags", "bar:b").subset("p");
     shouldAccept(c, "foo");
     shouldAccept(c, "foo");
     shouldAccept(c, Arrays.asList(tag("foo", "", "f")));
     shouldAccept(c, Arrays.asList(tag("foo", "", "f")));
+    shouldAccept(c, mockMetricsRecord("foo", Arrays.asList(
+      tag("foo", "", "f"))));
     shouldReject(c, "bar");
     shouldReject(c, "bar");
     shouldReject(c, Arrays.asList(tag("bar", "", "b")));
     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, "foobar");
     shouldAccept(c, Arrays.asList(tag("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");
         .add("p.exclude.tags", "foo:f").subset("p");
     shouldAccept(c, "foo");
     shouldAccept(c, "foo");
     shouldAccept(c, Arrays.asList(tag("foo", "", "f")));
     shouldAccept(c, Arrays.asList(tag("foo", "", "f")));
+    shouldAccept(c, mockMetricsRecord("foo", Arrays.asList(
+      tag("foo", "", "f"))));
   }
   }
 
 
   static void shouldAccept(SubsetConfiguration conf, String s) {
   static void shouldAccept(SubsetConfiguration conf, String s) {
@@ -110,6 +136,17 @@ public class TestPatternFilter {
     assertTrue("accepts "+ tags, newRegexFilter(conf).accepts(tags));
     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) {
   static void shouldReject(SubsetConfiguration conf, String s) {
     assertTrue("rejects "+ s, !newGlobFilter(conf).accepts(s));
     assertTrue("rejects "+ s, !newGlobFilter(conf).accepts(s));
     assertTrue("rejects "+ s, !newRegexFilter(conf).accepts(s));
     assertTrue("rejects "+ s, !newRegexFilter(conf).accepts(s));
@@ -120,6 +157,17 @@ public class TestPatternFilter {
     assertTrue("rejects "+ tags, !newRegexFilter(conf).accepts(tags));
     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
    * Create a new glob filter with a config object
    * @param conf  the config object
    * @param conf  the config object
@@ -141,4 +189,19 @@ public class TestPatternFilter {
     f.init(conf);
     f.init(conf);
     return f;
     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.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.StringWriter;
 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.lang.reflect.InvocationTargetException;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Random;
 import java.util.Random;
@@ -330,4 +333,33 @@ public abstract class GenericTestUtils {
         " but got:\n" + output,
         " but got:\n" + output,
         Pattern.compile(pattern).matcher(output).find());
         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 java.io.Closeable;
 
 
 import org.mockito.Mockito;
 import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.mockito.stubbing.Stubber;
 
 
 public abstract class MockitoUtil {
 public abstract class MockitoUtil {
 
 
@@ -33,4 +36,29 @@ public abstract class MockitoUtil {
     return Mockito.mock(clazz,
     return Mockito.mock(clazz,
         Mockito.withSettings().extraInterfaces(Closeable.class));
         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
     HDFS-4586. TestDataDirs.testGetDataDirsFromURIs fails with all directories
     in dfs.datanode.data.dir are invalid. (Ivan Mitic via atm)
     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
   BREAKDOWN OF HADOOP-8562 SUBTASKS AND RELATED JIRAS
 
 
     HDFS-4145. Merge hdfs cmd line scripts from branch-1-win. (David Lao,
     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
     HDFS-4625. Make TestNNWithQJM#testNewNamenodeTakesOverWriter work on
     Windows. (Ivan Mitic via suresh)
     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
 Release 2.0.5-beta - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
 
 
   NEW FEATURES
   NEW FEATURES
 
 
+    HDFS-1804. Add a new block-volume device choosing policy that looks at
+    free space. (atm)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HDFS-4222. NN is unresponsive and loses heartbeats from DNs when 
     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-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
   OPTIMIZATIONS
 
 
   BUG FIXES
   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
     HDFS-4658. Standby NN will log that it has received a block report "after
     becoming active" (atm)
     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
 Release 2.0.4-alpha - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   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);
                                      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
    * Get the file info for a specific file or directory. If src
    * refers to a symlink then the FileStatus of the link is returned.
    * 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_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_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_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_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_DATANODE_STARTUP_KEY = "dfs.datanode.startup";
   public static final String  DFS_NAMENODE_PLUGINS_KEY = "dfs.namenode.plugins";
   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>]
     // Look for configurations of the form <key>[.<nameserviceId>][.<namenodeId>]
     // across all of the configured nameservices and namenodes.
     // 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)) {
     for (String nsId : emptyAsSingletonNull(nameserviceIds)) {
       Map<String, InetSocketAddress> isas =
       Map<String, InetSocketAddress> isas =
         getAddressesForNameserviceId(conf, nsId, defaultAddress, keys);
         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 {
       String fromSnapshot, String toSnapshot) throws IOException {
     return dfs.getSnapshotDiffReport(snapshotDir, fromSnapshot, toSnapshot);
     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);
     final long version = RPC.getProtocolVersion(ClientNamenodeProtocolPB.class);
     ClientNamenodeProtocolPB proxy = RPC.getProtocolProxy(
     ClientNamenodeProtocolPB proxy = RPC.getProtocolProxy(
         ClientNamenodeProtocolPB.class, version, address, ugi, conf,
         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
     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
   @Idempotent
   public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
   public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
       FileNotFoundException, UnresolvedLinkException, IOException;
       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 
    * 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.
    * 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, " +
         "Serialize block lists with delta-encoded variable length ints, " +
         "add OP_UPDATE_BLOCKS"),
         "add OP_UPDATE_BLOCKS"),
     RESERVED_REL1_2_0(-41, -32, "Reserved for release 1.2.0", true, CONCAT),
     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 lv;
     final int ancestorLV;
     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.GetSnapshotDiffReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
 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.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.ListCorruptFileBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
@@ -984,4 +986,17 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
       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.GetSnapshotDiffReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
 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.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.ListCorruptFileBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
@@ -861,6 +862,19 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
       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
   @Override
   public Object getUnderlyingProxyObject() {
   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
     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() {
   void close() {
-    // Empty blocks once GSet#clear is implemented (HDFS-3940)
+    blocks.clear();
   }
   }
 
 
   BlockCollection getBlockCollection(Block b) {
   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 {
       Map<String, Map<String, InetSocketAddress>> addrMap) throws IOException {
     assert Thread.holdsLock(refreshNamenodesLock);
     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;
     Set<String> toRemove;
     
     
     synchronized (this) {
     synchronized (this) {
@@ -239,4 +239,4 @@ class BlockPoolManager {
   protected BPOfferService createBPOS(List<InetSocketAddress> nnAddrs) {
   protected BPOfferService createBPOS(List<InetSocketAddress> nnAddrs) {
     return new BPOfferService(nnAddrs, dn);
     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());
       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) {
   public void logOpenFile(String path, INodeFileUnderConstruction newNode) {
     AddOp op = AddOp.getInstance(cache.get())
     AddOp op = AddOp.getInstance(cache.get())
+      .setInodeId(newNode.getId())
       .setPath(path)
       .setPath(path)
       .setReplication(newNode.getFileReplication())
       .setReplication(newNode.getFileReplication())
       .setModificationTime(newNode.getModificationTime())
       .setModificationTime(newNode.getModificationTime())
@@ -702,6 +703,7 @@ public class FSEditLog implements LogsPurgeable {
    */
    */
   public void logMkDir(String path, INode newNode) {
   public void logMkDir(String path, INode newNode) {
     MkdirOp op = MkdirOp.getInstance(cache.get())
     MkdirOp op = MkdirOp.getInstance(cache.get())
+      .setInodeId(newNode.getId())
       .setPath(path)
       .setPath(path)
       .setTimestamp(newNode.getModificationTime())
       .setTimestamp(newNode.getModificationTime())
       .setPermissionStatus(newNode.getPermissionStatus());
       .setPermissionStatus(newNode.getPermissionStatus());
@@ -819,6 +821,7 @@ public class FSEditLog implements LogsPurgeable {
   void logSymlink(String path, String value, long mtime, 
   void logSymlink(String path, String value, long mtime, 
                   long atime, INodeSymlink node) {
                   long atime, INodeSymlink node) {
     SymlinkOp op = SymlinkOp.getInstance(cache.get())
     SymlinkOp op = SymlinkOp.getInstance(cache.get())
+      .setId(node.getId())
       .setPath(path)
       .setPath(path)
       .setValue(value)
       .setValue(value)
       .setModificationTime(mtime)
       .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.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 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.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -174,7 +175,7 @@ public class FSEditLogLoader {
             }
             }
           }
           }
           try {
           try {
-            long inodeId = applyEditLogOp(op, fsDir, in.getVersion());
+            long inodeId = applyEditLogOp(op, fsDir, in.getVersion(), lastInodeId);
             if (lastInodeId < inodeId) {
             if (lastInodeId < inodeId) {
               lastInodeId = inodeId;
               lastInodeId = inodeId;
             }
             }
@@ -230,9 +231,30 @@ public class FSEditLogLoader {
     return numEdits;
     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")
   @SuppressWarnings("deprecation")
   private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
   private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
-      int logVersion) throws IOException {
+      int logVersion, long lastInodeId) throws IOException {
     long inodeId = INodeId.GRANDFATHER_INODE_ID;
     long inodeId = INodeId.GRANDFATHER_INODE_ID;
     if (LOG.isTraceEnabled()) {
     if (LOG.isTraceEnabled()) {
       LOG.trace("replaying edit log: " + op);
       LOG.trace("replaying edit log: " + op);
@@ -265,7 +287,8 @@ public class FSEditLogLoader {
         assert addCloseOp.blocks.length == 0;
         assert addCloseOp.blocks.length == 0;
 
 
         // add to the file tree
         // add to the file tree
-        inodeId = fsNamesys.allocateNewInodeId();
+        inodeId = getAndUpdateLastInodeId(addCloseOp.inodeId, logVersion,
+            lastInodeId);
         newFile = fsDir.unprotectedAddFile(inodeId,
         newFile = fsDir.unprotectedAddFile(inodeId,
             addCloseOp.path, addCloseOp.permissions, replication,
             addCloseOp.path, addCloseOp.permissions, replication,
             addCloseOp.mtime, addCloseOp.atime, addCloseOp.blockSize, true,
             addCloseOp.mtime, addCloseOp.atime, addCloseOp.blockSize, true,
@@ -373,7 +396,8 @@ public class FSEditLogLoader {
     }
     }
     case OP_MKDIR: {
     case OP_MKDIR: {
       MkdirOp mkdirOp = (MkdirOp)op;
       MkdirOp mkdirOp = (MkdirOp)op;
-      inodeId = fsNamesys.allocateNewInodeId();
+      inodeId = getAndUpdateLastInodeId(mkdirOp.inodeId, logVersion,
+          lastInodeId);
       fsDir.unprotectedMkdir(inodeId, mkdirOp.path, mkdirOp.permissions,
       fsDir.unprotectedMkdir(inodeId, mkdirOp.path, mkdirOp.permissions,
                              mkdirOp.timestamp);
                              mkdirOp.timestamp);
       break;
       break;
@@ -427,7 +451,8 @@ public class FSEditLogLoader {
     }
     }
     case OP_SYMLINK: {
     case OP_SYMLINK: {
       SymlinkOp symlinkOp = (SymlinkOp)op;
       SymlinkOp symlinkOp = (SymlinkOp)op;
-      inodeId = fsNamesys.allocateNewInodeId();
+      inodeId = getAndUpdateLastInodeId(symlinkOp.inodeId, logVersion,
+          lastInodeId);
       fsDir.unprotectedAddSymlink(inodeId, symlinkOp.path,
       fsDir.unprotectedAddSymlink(inodeId, symlinkOp.path,
                                   symlinkOp.value, symlinkOp.mtime, 
                                   symlinkOp.value, symlinkOp.mtime, 
                                   symlinkOp.atime, symlinkOp.permissionStatus);
                                   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")
   @SuppressWarnings("unchecked")
   static abstract class AddCloseOp extends FSEditLogOp implements BlockListUpdatingOp {
   static abstract class AddCloseOp extends FSEditLogOp implements BlockListUpdatingOp {
     int length;
     int length;
+    long inodeId;
     String path;
     String path;
     short replication;
     short replication;
     long mtime;
     long mtime;
@@ -178,6 +179,11 @@ public abstract class FSEditLogOp {
       super(opCode);
       super(opCode);
       assert(opCode == OP_ADD || opCode == OP_CLOSE);
       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) {
     <T extends AddCloseOp> T setPath(String path) {
       this.path = path;
       this.path = path;
@@ -241,6 +247,7 @@ public abstract class FSEditLogOp {
     @Override
     @Override
     public 
     public 
     void writeFields(DataOutputStream out) throws IOException {
     void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeLong(inodeId, out);
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeShort(replication, out);
       FSImageSerialization.writeShort(replication, out);
       FSImageSerialization.writeLong(mtime, out);
       FSImageSerialization.writeLong(mtime, out);
@@ -261,6 +268,12 @@ public abstract class FSEditLogOp {
       if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
       if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.length = in.readInt();
         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) ||
       if ((-17 < logVersion && length != 4) ||
           (logVersion <= -17 && length != 5 && !LayoutVersion.supports(
           (logVersion <= -17 && length != 5 && !LayoutVersion.supports(
               Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) {
               Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) {
@@ -333,6 +346,8 @@ public abstract class FSEditLogOp {
       StringBuilder builder = new StringBuilder();
       StringBuilder builder = new StringBuilder();
       builder.append("[length=");
       builder.append("[length=");
       builder.append(length);
       builder.append(length);
+      builder.append(", inodeId=");
+      builder.append(inodeId);
       builder.append(", path=");
       builder.append(", path=");
       builder.append(path);
       builder.append(path);
       builder.append(", replication=");
       builder.append(", replication=");
@@ -363,6 +378,8 @@ public abstract class FSEditLogOp {
     protected void toXml(ContentHandler contentHandler) throws SAXException {
     protected void toXml(ContentHandler contentHandler) throws SAXException {
       XMLUtils.addSaxString(contentHandler, "LENGTH",
       XMLUtils.addSaxString(contentHandler, "LENGTH",
           Integer.valueOf(length).toString());
           Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "INODEID",
+          Long.valueOf(inodeId).toString());
       XMLUtils.addSaxString(contentHandler, "PATH", path);
       XMLUtils.addSaxString(contentHandler, "PATH", path);
       XMLUtils.addSaxString(contentHandler, "REPLICATION",
       XMLUtils.addSaxString(contentHandler, "REPLICATION",
           Short.valueOf(replication).toString());
           Short.valueOf(replication).toString());
@@ -382,6 +399,7 @@ public abstract class FSEditLogOp {
 
 
     @Override void fromXml(Stanza st) throws InvalidXmlException {
     @Override void fromXml(Stanza st) throws InvalidXmlException {
       this.length = Integer.valueOf(st.getValue("LENGTH"));
       this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.inodeId = Long.valueOf(st.getValue("INODEID"));
       this.path = st.getValue("PATH");
       this.path = st.getValue("PATH");
       this.replication = Short.valueOf(st.getValue("REPLICATION"));
       this.replication = Short.valueOf(st.getValue("REPLICATION"));
       this.mtime = Long.valueOf(st.getValue("MTIME"));
       this.mtime = Long.valueOf(st.getValue("MTIME"));
@@ -913,6 +931,7 @@ public abstract class FSEditLogOp {
     
     
   static class MkdirOp extends FSEditLogOp {
   static class MkdirOp extends FSEditLogOp {
     int length;
     int length;
+    long inodeId;
     String path;
     String path;
     long timestamp;
     long timestamp;
     PermissionStatus permissions;
     PermissionStatus permissions;
@@ -925,6 +944,11 @@ public abstract class FSEditLogOp {
       return (MkdirOp)cache.get(OP_MKDIR);
       return (MkdirOp)cache.get(OP_MKDIR);
     }
     }
 
 
+    MkdirOp setInodeId(long inodeId) {
+      this.inodeId = inodeId;
+      return this;
+    }
+    
     MkdirOp setPath(String path) {
     MkdirOp setPath(String path) {
       this.path = path;
       this.path = path;
       return this;
       return this;
@@ -943,6 +967,7 @@ public abstract class FSEditLogOp {
     @Override
     @Override
     public 
     public 
     void writeFields(DataOutputStream out) throws IOException {
     void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeLong(inodeId, out);
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeLong(timestamp, out); // mtime
       FSImageSerialization.writeLong(timestamp, out); // mtime
       FSImageSerialization.writeLong(timestamp, out); // atime, unused at this
       FSImageSerialization.writeLong(timestamp, out); // atime, unused at this
@@ -959,6 +984,12 @@ public abstract class FSEditLogOp {
           && !LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
           && !LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         throw new IOException("Incorrect data format. Mkdir operation.");
         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);
       this.path = FSImageSerialization.readString(in);
       if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
       if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.timestamp = FSImageSerialization.readLong(in);
         this.timestamp = FSImageSerialization.readLong(in);
@@ -985,6 +1016,8 @@ public abstract class FSEditLogOp {
       StringBuilder builder = new StringBuilder();
       StringBuilder builder = new StringBuilder();
       builder.append("MkdirOp [length=");
       builder.append("MkdirOp [length=");
       builder.append(length);
       builder.append(length);
+      builder.append(", inodeId=");
+      builder.append(inodeId);
       builder.append(", path=");
       builder.append(", path=");
       builder.append(path);
       builder.append(path);
       builder.append(", timestamp=");
       builder.append(", timestamp=");
@@ -1003,6 +1036,8 @@ public abstract class FSEditLogOp {
     protected void toXml(ContentHandler contentHandler) throws SAXException {
     protected void toXml(ContentHandler contentHandler) throws SAXException {
       XMLUtils.addSaxString(contentHandler, "LENGTH",
       XMLUtils.addSaxString(contentHandler, "LENGTH",
           Integer.valueOf(length).toString());
           Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "INODEID",
+          Long.valueOf(inodeId).toString());
       XMLUtils.addSaxString(contentHandler, "PATH", path);
       XMLUtils.addSaxString(contentHandler, "PATH", path);
       XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
       XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
           Long.valueOf(timestamp).toString());
           Long.valueOf(timestamp).toString());
@@ -1011,6 +1046,7 @@ public abstract class FSEditLogOp {
     
     
     @Override void fromXml(Stanza st) throws InvalidXmlException {
     @Override void fromXml(Stanza st) throws InvalidXmlException {
       this.length = Integer.valueOf(st.getValue("LENGTH"));
       this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.inodeId = Long.valueOf(st.getValue("INODEID"));
       this.path = st.getValue("PATH");
       this.path = st.getValue("PATH");
       this.timestamp = Long.valueOf(st.getValue("TIMESTAMP"));
       this.timestamp = Long.valueOf(st.getValue("TIMESTAMP"));
       this.permissions =
       this.permissions =
@@ -1489,6 +1525,7 @@ public abstract class FSEditLogOp {
 
 
   static class SymlinkOp extends FSEditLogOp {
   static class SymlinkOp extends FSEditLogOp {
     int length;
     int length;
+    long inodeId;
     String path;
     String path;
     String value;
     String value;
     long mtime;
     long mtime;
@@ -1503,6 +1540,11 @@ public abstract class FSEditLogOp {
       return (SymlinkOp)cache.get(OP_SYMLINK);
       return (SymlinkOp)cache.get(OP_SYMLINK);
     }
     }
 
 
+    SymlinkOp setId(long inodeId) {
+      this.inodeId = inodeId;
+      return this;
+    }
+    
     SymlinkOp setPath(String path) {
     SymlinkOp setPath(String path) {
       this.path = path;
       this.path = path;
       return this;
       return this;
@@ -1531,6 +1573,7 @@ public abstract class FSEditLogOp {
     @Override
     @Override
     public 
     public 
     void writeFields(DataOutputStream out) throws IOException {
     void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeLong(inodeId, out);      
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeString(path, out);
       FSImageSerialization.writeString(value, out);
       FSImageSerialization.writeString(value, out);
       FSImageSerialization.writeLong(mtime, out);
       FSImageSerialization.writeLong(mtime, out);
@@ -1548,6 +1591,12 @@ public abstract class FSEditLogOp {
               + "symlink operation.");
               + "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.path = FSImageSerialization.readString(in);
       this.value = FSImageSerialization.readString(in);
       this.value = FSImageSerialization.readString(in);
 
 
@@ -1566,6 +1615,8 @@ public abstract class FSEditLogOp {
       StringBuilder builder = new StringBuilder();
       StringBuilder builder = new StringBuilder();
       builder.append("SymlinkOp [length=");
       builder.append("SymlinkOp [length=");
       builder.append(length);
       builder.append(length);
+      builder.append(", inodeId=");
+      builder.append(inodeId);
       builder.append(", path=");
       builder.append(", path=");
       builder.append(path);
       builder.append(path);
       builder.append(", value=");
       builder.append(", value=");
@@ -1588,6 +1639,8 @@ public abstract class FSEditLogOp {
     protected void toXml(ContentHandler contentHandler) throws SAXException {
     protected void toXml(ContentHandler contentHandler) throws SAXException {
       XMLUtils.addSaxString(contentHandler, "LENGTH",
       XMLUtils.addSaxString(contentHandler, "LENGTH",
           Integer.valueOf(length).toString());
           Integer.valueOf(length).toString());
+      XMLUtils.addSaxString(contentHandler, "INODEID",
+          Long.valueOf(inodeId).toString());
       XMLUtils.addSaxString(contentHandler, "PATH", path);
       XMLUtils.addSaxString(contentHandler, "PATH", path);
       XMLUtils.addSaxString(contentHandler, "VALUE", value);
       XMLUtils.addSaxString(contentHandler, "VALUE", value);
       XMLUtils.addSaxString(contentHandler, "MTIME",
       XMLUtils.addSaxString(contentHandler, "MTIME",
@@ -1599,6 +1652,7 @@ public abstract class FSEditLogOp {
 
 
     @Override void fromXml(Stanza st) throws InvalidXmlException {
     @Override void fromXml(Stanza st) throws InvalidXmlException {
       this.length = Integer.valueOf(st.getValue("LENGTH"));
       this.length = Integer.valueOf(st.getValue("LENGTH"));
+      this.inodeId = Long.valueOf(st.getValue("INODEID"));
       this.path = st.getValue("PATH");
       this.path = st.getValue("PATH");
       this.value = st.getValue("VALUE");
       this.value = st.getValue("VALUE");
       this.mtime = Long.valueOf(st.getValue("MTIME"));
       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);
           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
         // read compression related info
         FSImageCompression compression;
         FSImageCompression compression;
         if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imgVersion)) {
         if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imgVersion)) {
@@ -285,8 +299,7 @@ public class FSImageFormat {
         in = compression.unwrapInputStream(fin);
         in = compression.unwrapInputStream(fin);
 
 
         LOG.info("Loading image file " + curFile + " using " + compression);
         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
         // load all inodes
         LOG.info("Number of files = " + numFiles);
         LOG.info("Number of files = " + numFiles);
         if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
         if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
@@ -356,7 +369,7 @@ public class FSImageFormat {
    * @throws IOException
    * @throws IOException
    */  
    */  
    private void loadLocalNameINodes(long numFiles, DataInput in) 
    private void loadLocalNameINodes(long numFiles, DataInput in) 
-   throws IOException {
+       throws IOException {
      assert LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
      assert LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
          getLayoutVersion());
          getLayoutVersion());
      assert numFiles > 0;
      assert numFiles > 0;
@@ -546,7 +559,8 @@ public class FSImageFormat {
   INode loadINode(final byte[] localName, boolean isSnapshotINode,
   INode loadINode(final byte[] localName, boolean isSnapshotINode,
       DataInput in) throws IOException {
       DataInput in) throws IOException {
     final int imgVersion = getLayoutVersion();
     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(
     final short replication = namesystem.getBlockManager().adjustReplication(
         in.readShort());
         in.readShort());
@@ -662,8 +676,8 @@ public class FSImageFormat {
       LOG.info("Number of files under construction = " + size);
       LOG.info("Number of files under construction = " + size);
 
 
       for (int i = 0; i < size; i++) {
       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
         // verify that file exists in namespace
         String path = cons.getLocalName();
         String path = cons.getLocalName();
@@ -804,6 +818,8 @@ public class FSImageFormat {
         out.writeLong(fsDir.rootDir.numItemsInTree());
         out.writeLong(fsDir.rootDir.numItemsInTree());
         out.writeLong(sourceNamesystem.getGenerationStamp());
         out.writeLong(sourceNamesystem.getGenerationStamp());
         out.writeLong(context.getTxId());
         out.writeLong(context.getTxId());
+        out.writeLong(sourceNamesystem.getLastInodeId());
+        
         sourceNamesystem.getSnapshotManager().write(out);
         sourceNamesystem.getSnapshotManager().write(out);
         
         
         // write compression info and set up compressed stream
         // 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.DFSUtil;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.hdfs.protocol.Block;
 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.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@@ -103,8 +105,11 @@ public class FSImageSerialization {
   // from the input stream
   // from the input stream
   //
   //
   static INodeFileUnderConstruction readINodeUnderConstruction(
   static INodeFileUnderConstruction readINodeUnderConstruction(
-      DataInput in) throws IOException {
+      DataInput in, FSNamesystem fsNamesys, int imgVersion)
+      throws IOException {
     byte[] name = readBytes(in);
     byte[] name = readBytes(in);
+    long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? in
+        .readLong() : fsNamesys.allocateNewInodeId();
     short blockReplication = in.readShort();
     short blockReplication = in.readShort();
     long modificationTime = in.readLong();
     long modificationTime = in.readLong();
     long preferredBlockSize = in.readLong();
     long preferredBlockSize = in.readLong();
@@ -132,10 +137,16 @@ public class FSImageSerialization {
     int numLocs = in.readInt();
     int numLocs = in.readInt();
     assert numLocs == 0 : "Unexpected block locations";
     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
   // Helper function that writes an INodeUnderConstruction
@@ -146,6 +157,7 @@ public class FSImageSerialization {
                                            String path) 
                                            String path) 
                                            throws IOException {
                                            throws IOException {
     writeString(path, out);
     writeString(path, out);
+    out.writeLong(cons.getId());
     out.writeShort(cons.getFileReplication());
     out.writeShort(cons.getFileReplication());
     out.writeLong(cons.getModificationTime());
     out.writeLong(cons.getModificationTime());
     out.writeLong(cons.getPreferredBlockSize());
     out.writeLong(cons.getPreferredBlockSize());
@@ -168,6 +180,7 @@ public class FSImageSerialization {
   public static void writeINodeFile(INodeFile file, DataOutput out,
   public static void writeINodeFile(INodeFile file, DataOutput out,
       boolean writeUnderConstruction) throws IOException {
       boolean writeUnderConstruction) throws IOException {
     writeLocalName(file, out);
     writeLocalName(file, out);
+    out.writeLong(file.getId());
     out.writeShort(file.getFileReplication());
     out.writeShort(file.getFileReplication());
     out.writeLong(file.getModificationTime());
     out.writeLong(file.getModificationTime());
     out.writeLong(file.getAccessTime());
     out.writeLong(file.getAccessTime());
@@ -198,6 +211,7 @@ public class FSImageSerialization {
   public static void writeINodeDirectory(INodeDirectory node, DataOutput out)
   public static void writeINodeDirectory(INodeDirectory node, DataOutput out)
       throws IOException {
       throws IOException {
     writeLocalName(node, out);
     writeLocalName(node, out);
+    out.writeLong(node.getId());
     out.writeShort(0);  // replication
     out.writeShort(0);  // replication
     out.writeLong(node.getModificationTime());
     out.writeLong(node.getModificationTime());
     out.writeLong(0);   // access time
     out.writeLong(0);   // access time
@@ -224,6 +238,7 @@ public class FSImageSerialization {
   private static void writeINodeSymlink(INodeSymlink node, DataOutput out)
   private static void writeINodeSymlink(INodeSymlink node, DataOutput out)
       throws IOException {
       throws IOException {
     writeLocalName(node, out);
     writeLocalName(node, out);
+    out.writeLong(node.getId());
     out.writeShort(0);  // replication
     out.writeShort(0);  // replication
     out.writeLong(0);   // modification time
     out.writeLong(0);   // modification time
     out.writeLong(0);   // access time
     out.writeLong(0);   // access time
@@ -239,6 +254,7 @@ public class FSImageSerialization {
       boolean writeUnderConstruction, ReferenceMap referenceMap
       boolean writeUnderConstruction, ReferenceMap referenceMap
       ) throws IOException {
       ) throws IOException {
     writeLocalName(ref, out);
     writeLocalName(ref, out);
+    out.writeLong(ref.getId());
     out.writeShort(0);  // replication
     out.writeShort(0);  // replication
     out.writeLong(0);   // modification time
     out.writeLong(0);   // modification time
     out.writeLong(0);   // access time
     out.writeLong(0);   // access time
@@ -403,4 +419,4 @@ public class FSImageSerialization {
     }
     }
     return ret;
     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.
         if (!iip.isSnapshot() //snapshots are readonly, so don't update atime.
             && doAccessTime && isAccessTimeSupported()) {
             && doAccessTime && isAccessTimeSupported()) {
           final long now = now();
           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
             // if we have to set access time but we only have the readlock, then
             // restart this entire operation with the writeLock.
             // restart this entire operation with the writeLock.
             if (isReadOp) {
             if (isReadOp) {
               continue;
               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?
         final long fileSize = iip.getPathSnapshot() != null?
             inode.computeFileSize(iip.getPathSnapshot())
             inode.computeFileSize(iip.getPathSnapshot())
@@ -2992,6 +2992,33 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     logAuditEvent(true, "getfileinfo", src);
     logAuditEvent(true, "getfileinfo", src);
     return stat;
     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
    * Create all the necessary directories
@@ -4828,8 +4855,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * shutdown FSNamesystem
    * shutdown FSNamesystem
    */
    */
   void shutdown() {
   void shutdown() {
-    if (mbeanName != null)
+    if (mbeanName != null) {
       MBeans.unregister(mbeanName);
       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();
     metrics.incrFileInfoOps();
     return namesystem.getFileInfo(src, true);
     return namesystem.getFileInfo(src, true);
   }
   }
-
+  
+  @Override // ClientProtocol
+  public boolean isFileClosed(String src) throws IOException{
+    return namesystem.isFileClosed(src);
+  }
+  
   @Override // ClientProtocol
   @Override // ClientProtocol
   public HdfsFileStatus getFileLinkInfo(String src) throws IOException { 
   public HdfsFileStatus getFileLinkInfo(String src) throws IOException { 
     metrics.incrFileInfoOps();
     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());
         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)) {
       if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imageVersion)) {
         boolean isCompressed = in.readBoolean();
         boolean isCompressed = in.readBoolean();
         v.visit(ImageElement.IS_COMPRESSED, String.valueOf(isCompressed));
         v.visit(ImageElement.IS_COMPRESSED, String.valueOf(isCompressed));
@@ -577,6 +581,9 @@ class ImageLoaderCurrent implements ImageLoader {
     }
     }
 
 
     v.visit(ImageElement.INODE_PATH, pathName);
     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.REPLICATION, in.readShort());
     v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
     v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
     if(LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imageVersion))
     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_EXPIRY_TIME,
     DELEGATION_TOKEN_IDENTIFIER_MASTER_KEY_ID,
     DELEGATION_TOKEN_IDENTIFIER_MASTER_KEY_ID,
     TRANSACTION_ID,
     TRANSACTION_ID,
+    LAST_INODE_ID,
+    INODE_ID,
+
     SNAPSHOT_COUNTER,
     SNAPSHOT_COUNTER,
     NUM_SNAPSHOTS_TOTAL,
     NUM_SNAPSHOTS_TOTAL,
     NUM_SNAPSHOTTABLE_DIRS,
     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.
     * @throws NullPointerException if key == null.
   */
   */
   E remove(K key);
   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() {
   public Iterator<E> iterator() {
     return m.values().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()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("recommended=" + recommended_length + ", actual=" + actual);
       LOG.debug("recommended=" + recommended_length + ", actual=" + actual);
     }
     }
-
     entries = new LinkedElement[actual];
     entries = new LinkedElement[actual];
     hash_mask = entries.length - 1;
     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 exponent = e2 < 0? 0: e2 > 30? 30: e2;
     final int c = 1 << exponent;
     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;
     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;
   optional HdfsFileStatusProto fs = 1;
 }
 }
 
 
+message IsFileClosedRequestProto {
+  required string src = 1;
+}
+
+message IsFileClosedResponseProto {
+  required bool result = 1;
+}
+
 message GetFileLinkInfoRequestProto {
 message GetFileLinkInfoRequestProto {
   required string src = 1;
   required string src = 1;
 }
 }
@@ -566,4 +574,6 @@ service ClientNamenodeProtocol {
       returns(DeleteSnapshotResponseProto);
       returns(DeleteSnapshotResponseProto);
   rpc getSnapshotDiffReport(GetSnapshotDiffReportRequestProto)
   rpc getSnapshotDiffReport(GetSnapshotDiffReportRequestProto)
       returns(GetSnapshotDiffReportResponseProto);
       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>
   </description>
 </property>
 </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>
 </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 {
   protected void tearDown() throws Exception {
     super.tearDown();
     super.tearDown();
     cluster.shutdown();
     cluster.shutdown();
+    cluster = null;
   }
   }
 
 
   @Override
   @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.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.Date;
 import java.util.Random;
 import java.util.Random;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 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.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 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.apache.hadoop.util.Time;
 import org.junit.Test;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 
 /**
 /**
  * This class tests the access time on files.
  * This class tests the access time on files.
@@ -273,6 +277,37 @@ public class TestSetTimes {
       cluster.shutdown();
       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 {
   public static void main(String[] args) throws Exception {
     new TestSetTimes().testTimes();
     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 Configuration conf;
   private QuorumJournalManager qjm;
   private QuorumJournalManager qjm;
   private List<AsyncLogger> spies;
   private List<AsyncLogger> spies;
+
+  private List<QuorumJournalManager> toClose = Lists.newLinkedList();
   
   
   static {
   static {
     ((Log4JLogger)ProtobufRpcEngine.LOG).getLogger().setLevel(Level.ALL);
     ((Log4JLogger)ProtobufRpcEngine.LOG).getLogger().setLevel(Level.ALL);
@@ -98,11 +100,26 @@ public class TestQuorumJournalManager {
   
   
   @After
   @After
   public void shutdown() throws IOException {
   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) {
     if (cluster != null) {
       cluster.shutdown();
       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
   @Test
   public void testSingleWriter() throws Exception {
   public void testSingleWriter() throws Exception {
     writeSegment(cluster, qjm, 1, 3, true);
     writeSegment(cluster, qjm, 1, 3, true);
@@ -119,8 +136,8 @@ public class TestQuorumJournalManager {
   
   
   @Test
   @Test
   public void testFormat() throws Exception {
   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());
     assertFalse(qjm.hasSomeData());
     qjm.format(FAKE_NSINFO);
     qjm.format(FAKE_NSINFO);
     assertTrue(qjm.hasSomeData());
     assertTrue(qjm.hasSomeData());
@@ -128,8 +145,7 @@ public class TestQuorumJournalManager {
   
   
   @Test
   @Test
   public void testReaderWhileAnotherWrites() throws Exception {
   public void testReaderWhileAnotherWrites() throws Exception {
-    
-    QuorumJournalManager readerQjm = createSpyingQJM();
+    QuorumJournalManager readerQjm = closeLater(createSpyingQJM());
     List<EditLogInputStream> streams = Lists.newArrayList();
     List<EditLogInputStream> streams = Lists.newArrayList();
     readerQjm.selectInputStreams(streams, 0, false);
     readerQjm.selectInputStreams(streams, 0, false);
     assertEquals(0, streams.size());
     assertEquals(0, streams.size());
@@ -251,8 +267,8 @@ public class TestQuorumJournalManager {
     
     
     
     
     // Make a new QJM
     // 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();
     qjm.recoverUnfinalizedSegments();
     checkRecovery(cluster, 1, 3);
     checkRecovery(cluster, 1, 3);
 
 
@@ -364,8 +380,8 @@ public class TestQuorumJournalManager {
         NNStorage.getInProgressEditsFileName(1));
         NNStorage.getInProgressEditsFileName(1));
 
 
     // Make a new QJM
     // 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();
     qjm.recoverUnfinalizedSegments();
     checkRecovery(cluster, 1, 3);
     checkRecovery(cluster, 1, 3);
   }
   }
@@ -902,8 +918,8 @@ public class TestQuorumJournalManager {
         return Mockito.spy(logger);
         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)
   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;
 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.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertSame;
 
 
+import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.util.Map;
 import java.util.Map;
@@ -68,6 +70,8 @@ public class TestBPOfferService {
       TestBPOfferService.class);
       TestBPOfferService.class);
   private static final ExtendedBlock FAKE_BLOCK =
   private static final ExtendedBlock FAKE_BLOCK =
     new ExtendedBlock(FAKE_BPID, 12345L);
     new ExtendedBlock(FAKE_BPID, 12345L);
+  private static final String TEST_BUILD_DATA = System.getProperty(
+    "test.build.data", "build/test/data");
 
 
   static {
   static {
     ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);
     ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);
@@ -90,6 +94,8 @@ public class TestBPOfferService {
     mockDn = Mockito.mock(DataNode.class);
     mockDn = Mockito.mock(DataNode.class);
     Mockito.doReturn(true).when(mockDn).shouldRun();
     Mockito.doReturn(true).when(mockDn).shouldRun();
     Configuration conf = new Configuration();
     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(conf).when(mockDn).getConf();
     Mockito.doReturn(new DNConf(conf)).when(mockDn).getDnConf();
     Mockito.doReturn(new DNConf(conf)).when(mockDn).getDnConf();
     Mockito.doReturn(DataNodeMetrics.create(conf, "fake dn"))
     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
     // Remove the first NS
     conf.set(DFSConfigKeys.DFS_NAMESERVICES,
     conf.set(DFSConfigKeys.DFS_NAMESERVICES,
-        "ns1");
+        "ns2");
     bpm.refreshNamenodes(conf);
     bpm.refreshNamenodes(conf);
     assertEquals(
     assertEquals(
         "stop #1\n" +
         "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 the Round-Robin block-volume choosing algorithm.
   @Test
   @Test
   public void testRR() throws Exception {
   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>();
     final List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
 
 
     // First volume, with 100 bytes of space.
     // First volume, with 100 bytes of space.
@@ -41,10 +49,6 @@ public class TestRoundRobinVolumeChoosingPolicy {
     // Second volume, with 200 bytes of space.
     // Second volume, with 200 bytes of space.
     volumes.add(Mockito.mock(FsVolumeSpi.class));
     volumes.add(Mockito.mock(FsVolumeSpi.class));
     Mockito.when(volumes.get(1).getAvailable()).thenReturn(200L);
     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
     // Test two rounds of round-robin choosing
     Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 0));
     Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 0));
@@ -69,6 +73,13 @@ public class TestRoundRobinVolumeChoosingPolicy {
   // with volume and block sizes in exception message.
   // with volume and block sizes in exception message.
   @Test
   @Test
   public void testRRPolicyExceptionMessage() throws Exception {
   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>();
     final List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
 
 
     // First volume, with 500 bytes of space.
     // First volume, with 500 bytes of space.
@@ -79,8 +90,6 @@ public class TestRoundRobinVolumeChoosingPolicy {
     volumes.add(Mockito.mock(FsVolumeSpi.class));
     volumes.add(Mockito.mock(FsVolumeSpi.class));
     Mockito.when(volumes.get(1).getAvailable()).thenReturn(600L);
     Mockito.when(volumes.get(1).getAvailable()).thenReturn(600L);
 
 
-    final RoundRobinVolumeChoosingPolicy<FsVolumeSpi> policy
-        = new RoundRobinVolumeChoosingPolicy<FsVolumeSpi>();
     int blockSize = 700;
     int blockSize = 700;
     try {
     try {
       policy.chooseVolume(volumes, blockSize);
       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));
         FsPermission.createImmutable((short)0755));
     for (int i = 1; i <= numDirs; i++) {
     for (int i = 1; i <= numDirs; i++) {
       String dirName = "dir" + i;
       String dirName = "dir" + i;
-      INodeDirectory dir = new INodeDirectory(newInodeId + i -1,
+      INodeDirectory dir = new INodeDirectory(newInodeId + i - 1,
           DFSUtil.string2Bytes(dirName), perms, 0L);
           DFSUtil.string2Bytes(dirName), perms, 0L);
       editLog.logMkDir("/" + dirName, dir);
       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.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.management.ManagementFactory;
-import java.lang.management.ThreadInfo;
-import java.lang.management.ThreadMXBean;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.ArrayList;
@@ -122,18 +120,7 @@ public class TestCheckpoint {
   
   
   @After
   @After
   public void checkForSNNThreads() {
   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)
   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.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
-import java.util.EnumSet;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Options;
 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.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 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.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
-import org.apache.hadoop.io.EnumSetWritable;
 import org.junit.Test;
 import org.junit.Test;
 
 
 public class TestINodeFile {
 public class TestINodeFile {
@@ -366,9 +361,7 @@ public class TestINodeFile {
       cluster.waitActive();
       cluster.waitActive();
 
 
       FSNamesystem fsn = cluster.getNamesystem();
       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
       // Create one directory and the last inode id should increase to 1002
       FileSystem fs = cluster.getFileSystem();
       FileSystem fs = cluster.getFileSystem();
@@ -376,14 +369,10 @@ public class TestINodeFile {
       assertTrue(fs.mkdirs(path));
       assertTrue(fs.mkdirs(path));
       assertTrue(fsn.getLastInodeId() == 1002);
       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(fsn.getLastInodeId() == 1003);
-      assertTrue(fileStatus.getFileId() == 1003);
 
 
       // Rename doesn't increase inode id
       // Rename doesn't increase inode id
       Path renamedPath = new Path("/test2");
       Path renamedPath = new Path("/test2");
@@ -395,7 +384,32 @@ public class TestINodeFile {
       // Make sure empty editlog can be handled
       // Make sure empty editlog can be handled
       cluster.restartNameNode();
       cluster.restartNameNode();
       cluster.waitActive();
       cluster.waitActive();
+      fsn = cluster.getNamesystem();
       assertTrue(fsn.getLastInodeId() == 1003);
       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 {
     } finally {
       if (cluster != null) {
       if (cluster != null) {
         cluster.shutdown();
         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",
       return String.format(" iterate=%5d, contain=%5d, time elapsed=%5d.%03ds",
           iterate_count, contain_count, t/1000, t%1000);
           iterate_count, contain_count, t/1000, t%1000);
     }
     }
+
+    @Override
+    public void clear() {
+      gset.clear();
+    }
   }
   }
 
 
   /** Test data set */
   /** 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"?>
 <?xml version="1.0" encoding="UTF-8"?>
 <EDITS>
 <EDITS>
-  <EDITS_VERSION>-41</EDITS_VERSION>
+  <EDITS_VERSION>-42</EDITS_VERSION>
   <RECORD>
   <RECORD>
     <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <DATA>
     <DATA>
@@ -41,6 +41,7 @@
     <DATA>
     <DATA>
       <TXID>5</TXID>
       <TXID>5</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
+      <INODEID>1002</INODEID>
       <PATH>/file_create</PATH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709370</MTIME>
       <MTIME>1357889709370</MTIME>
@@ -60,6 +61,7 @@
     <DATA>
     <DATA>
       <TXID>6</TXID>
       <TXID>6</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709398</MTIME>
       <MTIME>1357889709398</MTIME>
@@ -98,6 +100,7 @@
     <DATA>
     <DATA>
       <TXID>9</TXID>
       <TXID>9</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
+      <INODEID>1003</INODEID>
       <PATH>/directory_mkdir</PATH>
       <PATH>/directory_mkdir</PATH>
       <TIMESTAMP>1357889709598</TIMESTAMP>
       <TIMESTAMP>1357889709598</TIMESTAMP>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
@@ -165,6 +168,7 @@
     <DATA>
     <DATA>
       <TXID>17</TXID>
       <TXID>17</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
+      <INODEID>1004</INODEID>
       <PATH>/file_create</PATH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709615</MTIME>
       <MTIME>1357889709615</MTIME>
@@ -184,6 +188,7 @@
     <DATA>
     <DATA>
       <TXID>18</TXID>
       <TXID>18</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709616</MTIME>
       <MTIME>1357889709616</MTIME>
@@ -264,6 +269,7 @@
     <DATA>
     <DATA>
       <TXID>26</TXID>
       <TXID>26</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
+      <INODEID>1005</INODEID>
       <PATH>/file_concat_target</PATH>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709631</MTIME>
       <MTIME>1357889709631</MTIME>
@@ -355,6 +361,7 @@
     <DATA>
     <DATA>
       <TXID>33</TXID>
       <TXID>33</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/file_concat_target</PATH>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709689</MTIME>
       <MTIME>1357889709689</MTIME>
@@ -396,6 +403,7 @@
     <DATA>
     <DATA>
       <TXID>35</TXID>
       <TXID>35</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
+      <INODEID>1006</INODEID>
       <PATH>/file_concat_0</PATH>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709692</MTIME>
       <MTIME>1357889709692</MTIME>
@@ -487,6 +495,7 @@
     <DATA>
     <DATA>
       <TXID>42</TXID>
       <TXID>42</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/file_concat_0</PATH>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709710</MTIME>
       <MTIME>1357889709710</MTIME>
@@ -528,6 +537,7 @@
     <DATA>
     <DATA>
       <TXID>44</TXID>
       <TXID>44</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
+      <INODEID>1007</INODEID>
       <PATH>/file_concat_1</PATH>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709713</MTIME>
       <MTIME>1357889709713</MTIME>
@@ -619,6 +629,7 @@
     <DATA>
     <DATA>
       <TXID>51</TXID>
       <TXID>51</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/file_concat_1</PATH>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709744</MTIME>
       <MTIME>1357889709744</MTIME>
@@ -666,6 +677,7 @@
     <DATA>
     <DATA>
       <TXID>53</TXID>
       <TXID>53</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
+      <INODEID>1008</INODEID>
       <PATH>/file_symlink</PATH>
       <PATH>/file_symlink</PATH>
       <VALUE>/file_concat_target</VALUE>
       <VALUE>/file_concat_target</VALUE>
       <MTIME>1357889709751</MTIME>
       <MTIME>1357889709751</MTIME>
@@ -739,6 +751,7 @@
     <DATA>
     <DATA>
       <TXID>58</TXID>
       <TXID>58</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
+      <INODEID>1009</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889709825</MTIME>
       <MTIME>1357889709825</MTIME>
@@ -805,6 +818,7 @@
     <DATA>
     <DATA>
       <TXID>64</TXID>
       <TXID>64</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
       <MTIME>1357889712257</MTIME>
       <MTIME>1357889712257</MTIME>

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

@@ -166,6 +166,9 @@ Release 2.0.5-alpha - UNRELEASED
 
 
   IMPROVEMENTS
   IMPROVEMENTS
 
 
+    MAPREDUCE-5129. Allow tags to JobHistory for deeper analytics. (billie via
+    acmurthy)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -222,6 +225,9 @@ Release 2.0.4-beta - UNRELEASED
     MAPREDUCE-5088. MR Client gets an renewer token exception while Oozie is
     MAPREDUCE-5088. MR Client gets an renewer token exception while Oozie is
     submitting a job (Daryn Sharp via cos)
     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 
 Release 2.0.3-alpha - 2013-02-06 
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -773,6 +779,9 @@ Release 0.23.7 - UNRELEASED
 
 
     MAPREDUCE-4991. coverage for gridmix (Aleksey Gorshkov via tgraves)
     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
 Release 0.23.6 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   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_ID, ""),
             job.conf.get(MRJobConfig.WORKFLOW_NAME, ""),
             job.conf.get(MRJobConfig.WORKFLOW_NAME, ""),
             job.conf.get(MRJobConfig.WORKFLOW_NODE_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));
         job.eventHandler.handle(new JobHistoryEvent(job.jobId, jse));
         //TODO JH Verify jobACLs, UserName via UGI?
         //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_NODE_NAME, "testNodeName");
     conf.set(MRJobConfig.WORKFLOW_ADJACENCY_PREFIX_STRING + "key1", "value1");
     conf.set(MRJobConfig.WORKFLOW_ADJACENCY_PREFIX_STRING + "key1", "value1");
     conf.set(MRJobConfig.WORKFLOW_ADJACENCY_PREFIX_STRING + "key2", "value2");
     conf.set(MRJobConfig.WORKFLOW_ADJACENCY_PREFIX_STRING + "key2", "value2");
+    conf.set(MRJobConfig.WORKFLOW_TAGS, "tag1,tag2");
     
     
  
  
     AsyncDispatcher dispatcher = new AsyncDispatcher();
     AsyncDispatcher dispatcher = new AsyncDispatcher();
@@ -126,7 +127,8 @@ public class TestJobImpl {
     commitHandler.start();
     commitHandler.start();
 
 
     JobSubmittedEventHandler jseHandler = new JobSubmittedEventHandler("testId",
     JobSubmittedEventHandler jseHandler = new JobSubmittedEventHandler("testId",
-        "testName", "testNodeName", "\"key2\"=\"value2\" \"key1\"=\"value1\" ");
+        "testName", "testNodeName", "\"key2\"=\"value2\" \"key1\"=\"value1\" ",
+        "tag1,tag2");
     dispatcher.register(EventType.class, jseHandler);
     dispatcher.register(EventType.class, jseHandler);
     JobImpl job = createStubbedJob(conf, dispatcher, 0);
     JobImpl job = createStubbedJob(conf, dispatcher, 0);
     job.handle(new JobEvent(job.getID(), JobEventType.JOB_INIT));
     job.handle(new JobEvent(job.getID(), JobEventType.JOB_INIT));
@@ -706,14 +708,18 @@ public class TestJobImpl {
     
     
     private String workflowAdjacencies;
     private String workflowAdjacencies;
     
     
+    private String workflowTags;
+    
     private Boolean assertBoolean;
     private Boolean assertBoolean;
 
 
     public JobSubmittedEventHandler(String workflowId, String workflowName,
     public JobSubmittedEventHandler(String workflowId, String workflowName,
-        String workflowNodeName, String workflowAdjacencies) {
+        String workflowNodeName, String workflowAdjacencies,
+        String workflowTags) {
       this.workflowId = workflowId;
       this.workflowId = workflowId;
       this.workflowName = workflowName;
       this.workflowName = workflowName;
       this.workflowNodeName = workflowNodeName;
       this.workflowNodeName = workflowNodeName;
       this.workflowAdjacencies = workflowAdjacencies;
       this.workflowAdjacencies = workflowAdjacencies;
+      this.workflowTags = workflowTags;
       assertBoolean = null;
       assertBoolean = null;
     }
     }
 
 
@@ -739,6 +745,10 @@ public class TestJobImpl {
         setAssertValue(false);
         setAssertValue(false);
         return;
         return;
       }
       }
+      if (!workflowTags.equals(jsEvent.getWorkflowTags())) {
+        setAssertValue(false);
+        return;
+      }
       setAssertValue(true);
       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.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Map.Entry;
-import java.util.Random;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.Future;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicLong;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -91,6 +91,9 @@ class LocalDistributedCacheManager {
     Map<String, LocalResource> localResources = 
     Map<String, LocalResource> localResources = 
       new LinkedHashMap<String, LocalResource>();
       new LinkedHashMap<String, LocalResource>();
     MRApps.setupDistributedCache(conf, localResources);
     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
     // Find which resources are to be put on the local classpath
     Map<String, Path> classpaths = new HashMap<String, Path>();
     Map<String, Path> classpaths = new HashMap<String, Path>();
@@ -128,8 +131,10 @@ class LocalDistributedCacheManager {
       Path destPath = localDirAllocator.getLocalPathForWrite(".", conf);
       Path destPath = localDirAllocator.getLocalPathForWrite(".", conf);
       Map<LocalResource, Future<Path>> resourcesToPaths = Maps.newHashMap();
       Map<LocalResource, Future<Path>> resourcesToPaths = Maps.newHashMap();
       for (LocalResource resource : localResources.values()) {
       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);
         Future<Path> future = exec.submit(download);
         resourcesToPaths.put(resource, future);
         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": "workflowId", "type": "string"},
           {"name": "workflowName", "type": "string"},
           {"name": "workflowName", "type": "string"},
           {"name": "workflowNodeName", "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 =
   public static final String WORKFLOW_ADJACENCY_PREFIX_PATTERN =
       "^mapreduce\\.workflow\\.adjacency\\..+";
       "^mapreduce\\.workflow\\.adjacency\\..+";
 
 
+  public static final String WORKFLOW_TAGS = "mapreduce.workflow.tags";
+
   /**
   /**
    * The maximum number of application attempts.
    * The maximum number of application attempts.
    * It is a application-specific setting.
    * 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,
       Map<JobACL, AccessControlList> jobACLs, String jobQueueName,
       String workflowId, String workflowName, String workflowNodeName,
       String workflowId, String workflowName, String workflowNodeName,
       String workflowAdjacencies) {
       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.jobid = new Utf8(id.toString());
     datum.jobName = new Utf8(jobName);
     datum.jobName = new Utf8(jobName);
     datum.userName = new Utf8(userName);
     datum.userName = new Utf8(userName);
@@ -101,6 +126,9 @@ public class JobSubmittedEvent implements HistoryEvent {
     if (workflowAdjacencies != null) {
     if (workflowAdjacencies != null) {
       datum.workflowAdjacencies = new Utf8(workflowAdjacencies);
       datum.workflowAdjacencies = new Utf8(workflowAdjacencies);
     }
     }
+    if (workflowTags != null) {
+      datum.workflowTags = new Utf8(workflowTags);
+    }
   }
   }
 
 
   JobSubmittedEvent() {}
   JobSubmittedEvent() {}
@@ -168,6 +196,13 @@ public class JobSubmittedEvent implements HistoryEvent {
     }
     }
     return null;
     return null;
   }
   }
+  /** Get the workflow tags */
+  public String getWorkflowTags() {
+    if (datum.workflowTags != null) {
+      return datum.workflowTags.toString();
+    }
+    return null;
+  }
   /** Get the event type */
   /** Get the event type */
   public EventType getEventType() { return EventType.JOB_SUBMITTED; }
   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.Map;
 import java.util.TreeMap;
 import java.util.TreeMap;
 
 
+import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
 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.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
 import org.apache.hadoop.mapreduce.v2.hs.CompletedTask;
 import org.apache.hadoop.mapreduce.v2.hs.CompletedTask;
-import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
-import org.mockito.Mockito;
+import static org.mockito.Mockito.*;
+import static org.junit.Assert.*;
 
 
 public class TestCompletedTask{
 public class TestCompletedTask{
 
 
-  @Test
+  @Test (timeout=5000)
   public void testTaskStartTimes() {
   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
     Map<TaskAttemptID, TaskAttemptInfo> taskAttempts
       = new TreeMap<TaskAttemptID, TaskAttemptInfo>();
       = new TreeMap<TaskAttemptID, TaskAttemptInfo>();
     
     
     TaskAttemptID id = new TaskAttemptID("0", 0, TaskType.MAP, 0, 0);
     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);
     taskAttempts.put(id, info);
     
     
     id = new TaskAttemptID("1", 0, TaskType.MAP, 1, 1);
     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);
     taskAttempts.put(id, info);
     
     
     
     
-    Mockito.when(taskInfo.getAllTaskAttempts()).thenReturn(taskAttempts);
+    when(taskInfo.getAllTaskAttempts()).thenReturn(taskAttempts);
     CompletedTask task = new CompletedTask(taskId, taskInfo);
     CompletedTask task = new CompletedTask(taskId, taskInfo);
     TaskReport report = task.getReport();
     TaskReport report = task.getReport();
 
 
     // Make sure the startTime returned by report is the lesser of the 
     // Make sure the startTime returned by report is the lesser of the 
     // attempy launch times
     // 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.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 import org.junit.runners.Parameterized.Parameters;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
 
 
+import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 import static org.mockito.Mockito.*;
 
 
 @RunWith(value = Parameterized.class)
 @RunWith(value = Parameterized.class)
@@ -79,7 +81,7 @@ public class TestJobHistoryEntities {
   }
   }
 
 
   /* Verify some expected values based on the history file */
   /* Verify some expected values based on the history file */
-  @Test (timeout=10000)
+  @Test (timeout=100000)
   public void testCompletedJob() throws Exception {
   public void testCompletedJob() throws Exception {
     HistoryFileInfo info = mock(HistoryFileInfo.class);
     HistoryFileInfo info = mock(HistoryFileInfo.class);
     when(info.getConfFile()).thenReturn(fullConfPath);
     when(info.getConfFile()).thenReturn(fullConfPath);
@@ -168,4 +170,45 @@ public class TestJobHistoryEntities {
     assertEquals(45454, rta1Report.getNodeManagerPort());
     assertEquals(45454, rta1Report.getNodeManagerPort());
     assertEquals(9999, rta1Report.getNodeManagerHttpPort());
     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;
 package org.apache.hadoop.mapreduce.v2.hs;
 
 
 import java.io.ByteArrayOutputStream;
 import java.io.ByteArrayOutputStream;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.io.PrintStream;
 import java.util.Arrays;
 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.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 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.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.MRApp;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 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.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.FileNameIndexUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
 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.net.DNSToSwitchMapping;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.service.Service;
 import org.apache.hadoop.yarn.service.Service;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.BuilderUtils;
@@ -80,12 +88,12 @@ public class TestJobHistoryParsing {
 
 
   private static final String RACK_NAME = "/MyRackName";
   private static final String RACK_NAME = "/MyRackName";
 
 
-  private  ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+  private ByteArrayOutputStream outContent = new ByteArrayOutputStream();
 
 
   public static class MyResolver implements DNSToSwitchMapping {
   public static class MyResolver implements DNSToSwitchMapping {
     @Override
     @Override
     public List<String> resolve(List<String> names) {
     public List<String> resolve(List<String> names) {
-      return Arrays.asList(new String[]{RACK_NAME});
+      return Arrays.asList(new String[] { RACK_NAME });
     }
     }
 
 
     @Override
     @Override
@@ -93,14 +101,14 @@ public class TestJobHistoryParsing {
     }
     }
   }
   }
 
 
-  @Test (timeout=50000)
+  @Test(timeout = 50000)
   public void testJobInfo() throws Exception {
   public void testJobInfo() throws Exception {
     JobInfo info = new JobInfo();
     JobInfo info = new JobInfo();
     Assert.assertEquals("NORMAL", info.getPriority());
     Assert.assertEquals("NORMAL", info.getPriority());
     info.printAll();
     info.printAll();
   }
   }
 
 
-  @Test (timeout=50000)
+  @Test(timeout = 300000)
   public void testHistoryParsing() throws Exception {
   public void testHistoryParsing() throws Exception {
     LOG.info("STARTING testHistoryParsing()");
     LOG.info("STARTING testHistoryParsing()");
     try {
     try {
@@ -109,8 +117,8 @@ public class TestJobHistoryParsing {
       LOG.info("FINISHED testHistoryParsing()");
       LOG.info("FINISHED testHistoryParsing()");
     }
     }
   }
   }
-  
-  @Test (timeout=50000)
+
+  @Test(timeout = 50000)
   public void testHistoryParsingWithParseErrors() throws Exception {
   public void testHistoryParsingWithParseErrors() throws Exception {
     LOG.info("STARTING testHistoryParsingWithParseErrors()");
     LOG.info("STARTING testHistoryParsingWithParseErrors()");
     try {
     try {
@@ -119,18 +127,18 @@ public class TestJobHistoryParsing {
       LOG.info("FINISHED testHistoryParsingWithParseErrors()");
       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);
     Path qPath = fc.makeQualified(path);
     FSDataInputStream in = fc.open(qPath);
     FSDataInputStream in = fc.open(qPath);
     String jobSummaryString = in.readUTF();
     String jobSummaryString = in.readUTF();
     in.close();
     in.close();
     return jobSummaryString;
     return jobSummaryString;
   }
   }
-  
+
   private void checkHistoryParsing(final int numMaps, final int numReduces,
   private void checkHistoryParsing(final int numMaps, final int numReduces,
-      final int numSuccessfulMaps) 
-  throws Exception {
+      final int numSuccessfulMaps) throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name"));
     conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name"));
     long amStartTimeEst = System.currentTimeMillis();
     long amStartTimeEst = System.currentTimeMillis();
@@ -138,9 +146,8 @@ public class TestJobHistoryParsing {
         CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
         CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
         MyResolver.class, DNSToSwitchMapping.class);
         MyResolver.class, DNSToSwitchMapping.class);
     RackResolver.init(conf);
     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);
     app.submit(conf);
     Job job = app.getContext().getAllJobs().values().iterator().next();
     Job job = app.getContext().getAllJobs().values().iterator().next();
     JobId jobId = job.getID();
     JobId jobId = job.getID();
@@ -152,7 +159,7 @@ public class TestJobHistoryParsing {
 
 
     String jobhistoryDir = JobHistoryUtils
     String jobhistoryDir = JobHistoryUtils
         .getHistoryIntermediateDoneDirForUser(conf);
         .getHistoryIntermediateDoneDirForUser(conf);
-    
+
     FileContext fc = null;
     FileContext fc = null;
     try {
     try {
       fc = FileContext.getFileContext(conf);
       fc = FileContext.getFileContext(conf);
@@ -160,7 +167,7 @@ public class TestJobHistoryParsing {
       LOG.info("Can not get FileContext", ioe);
       LOG.info("Can not get FileContext", ioe);
       throw (new Exception("Can not get File Context"));
       throw (new Exception("Can not get File Context"));
     }
     }
-    
+
     if (numMaps == numSuccessfulMaps) {
     if (numMaps == numSuccessfulMaps) {
       String summaryFileName = JobHistoryUtils
       String summaryFileName = JobHistoryUtils
           .getIntermediateSummaryFileName(jobId);
           .getIntermediateSummaryFileName(jobId);
@@ -185,20 +192,22 @@ public class TestJobHistoryParsing {
           Long.parseLong(jobSummaryElements.get("submitTime")) != 0);
           Long.parseLong(jobSummaryElements.get("submitTime")) != 0);
       Assert.assertTrue("launchTime should not be 0",
       Assert.assertTrue("launchTime should not be 0",
           Long.parseLong(jobSummaryElements.get("launchTime")) != 0);
           Long.parseLong(jobSummaryElements.get("launchTime")) != 0);
-      Assert.assertTrue("firstMapTaskLaunchTime should not be 0",
-          Long.parseLong(jobSummaryElements.get("firstMapTaskLaunchTime")) != 0);
       Assert
       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",
       Assert.assertTrue("finishTime should not be 0",
           Long.parseLong(jobSummaryElements.get("finishTime")) != 0);
           Long.parseLong(jobSummaryElements.get("finishTime")) != 0);
       Assert.assertEquals("Mismatch in num map slots", numSuccessfulMaps,
       Assert.assertEquals("Mismatch in num map slots", numSuccessfulMaps,
           Integer.parseInt(jobSummaryElements.get("numMaps")));
           Integer.parseInt(jobSummaryElements.get("numMaps")));
       Assert.assertEquals("Mismatch in num reduce slots", numReduces,
       Assert.assertEquals("Mismatch in num reduce slots", numReduces,
           Integer.parseInt(jobSummaryElements.get("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",
       Assert.assertEquals("Queue does not match", "default",
           jobSummaryElements.get("queue"));
           jobSummaryElements.get("queue"));
       Assert.assertEquals("Status does not match", "SUCCEEDED",
       Assert.assertEquals("Status does not match", "SUCCEEDED",
@@ -210,8 +219,8 @@ public class TestJobHistoryParsing {
     HistoryFileInfo fileInfo = jobHistory.getJobFileInfo(jobId);
     HistoryFileInfo fileInfo = jobHistory.getJobFileInfo(jobId);
     JobInfo jobInfo;
     JobInfo jobInfo;
     long numFinishedMaps;
     long numFinishedMaps;
-    
-    synchronized(fileInfo) {
+
+    synchronized (fileInfo) {
       Path historyFilePath = fileInfo.getHistoryFile();
       Path historyFilePath = fileInfo.getHistoryFile();
       FSDataInputStream in = null;
       FSDataInputStream in = null;
       LOG.info("JobHistoryFile is: " + historyFilePath);
       LOG.info("JobHistoryFile is: " + historyFilePath);
@@ -228,11 +237,11 @@ public class TestJobHistoryParsing {
       if (numMaps == numSuccessfulMaps) {
       if (numMaps == numSuccessfulMaps) {
         reader = realReader;
         reader = realReader;
       } else {
       } else {
-        final AtomicInteger numFinishedEvents = new AtomicInteger(0);  // Hack!
+        final AtomicInteger numFinishedEvents = new AtomicInteger(0); // Hack!
         Mockito.when(reader.getNextEvent()).thenAnswer(
         Mockito.when(reader.getNextEvent()).thenAnswer(
             new Answer<HistoryEvent>() {
             new Answer<HistoryEvent>() {
-              public HistoryEvent answer(InvocationOnMock invocation) 
-              throws IOException {
+              public HistoryEvent answer(InvocationOnMock invocation)
+                  throws IOException {
                 HistoryEvent event = realReader.getNextEvent();
                 HistoryEvent event = realReader.getNextEvent();
                 if (event instanceof TaskFinishedEvent) {
                 if (event instanceof TaskFinishedEvent) {
                   numFinishedEvents.incrementAndGet();
                   numFinishedEvents.incrementAndGet();
@@ -244,22 +253,20 @@ public class TestJobHistoryParsing {
                   throw new IOException("test");
                   throw new IOException("test");
                 }
                 }
               }
               }
-            }
-        );
+            });
       }
       }
 
 
       jobInfo = parser.parse(reader);
       jobInfo = parser.parse(reader);
 
 
-      numFinishedMaps = 
-        computeFinishedMaps(jobInfo, numMaps, numSuccessfulMaps);
+      numFinishedMaps = computeFinishedMaps(jobInfo, numMaps, numSuccessfulMaps);
 
 
       if (numFinishedMaps != numMaps) {
       if (numFinishedMaps != numMaps) {
         Exception parseException = parser.getParseException();
         Exception parseException = parser.getParseException();
-        Assert.assertNotNull("Didn't get expected parse exception", 
+        Assert.assertNotNull("Didn't get expected parse exception",
             parseException);
             parseException);
       }
       }
     }
     }
-    
+
     Assert.assertEquals("Incorrect username ", System.getProperty("user.name"),
     Assert.assertEquals("Incorrect username ", System.getProperty("user.name"),
         jobInfo.getUsername());
         jobInfo.getUsername());
     Assert.assertEquals("Incorrect jobName ", "test", jobInfo.getJobname());
     Assert.assertEquals("Incorrect jobName ", "test", jobInfo.getJobname());
@@ -267,7 +274,7 @@ public class TestJobHistoryParsing {
         jobInfo.getJobQueueName());
         jobInfo.getJobQueueName());
     Assert
     Assert
         .assertEquals("incorrect conf path", "test", jobInfo.getJobConfPath());
         .assertEquals("incorrect conf path", "test", jobInfo.getJobConfPath());
-    Assert.assertEquals("incorrect finishedMap ", numSuccessfulMaps, 
+    Assert.assertEquals("incorrect finishedMap ", numSuccessfulMaps,
         numFinishedMaps);
         numFinishedMaps);
     Assert.assertEquals("incorrect finishedReduces ", numReduces,
     Assert.assertEquals("incorrect finishedReduces ", numReduces,
         jobInfo.getFinishedReduces());
         jobInfo.getFinishedReduces());
@@ -275,8 +282,8 @@ public class TestJobHistoryParsing {
         jobInfo.getUberized());
         jobInfo.getUberized());
     Map<TaskID, TaskInfo> allTasks = jobInfo.getAllTasks();
     Map<TaskID, TaskInfo> allTasks = jobInfo.getAllTasks();
     int totalTasks = allTasks.size();
     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
     // Verify aminfo
     Assert.assertEquals(1, jobInfo.getAMInfos().size());
     Assert.assertEquals(1, jobInfo.getAMInfos().size());
@@ -306,8 +313,7 @@ public class TestJobHistoryParsing {
 
 
     // Deep compare Job and JobInfo
     // Deep compare Job and JobInfo
     for (Task task : job.getTasks().values()) {
     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);
       Assert.assertNotNull("TaskInfo not found", taskInfo);
       for (TaskAttempt taskAttempt : task.getAttempts().values()) {
       for (TaskAttempt taskAttempt : task.getAttempts().values()) {
         TaskAttemptInfo taskAttemptInfo = taskInfo.getAllTaskAttempts().get(
         TaskAttemptInfo taskAttemptInfo = taskInfo.getAllTaskAttempts().get(
@@ -318,27 +324,32 @@ public class TestJobHistoryParsing {
         if (numMaps == numSuccessfulMaps) {
         if (numMaps == numSuccessfulMaps) {
           Assert.assertEquals(MRApp.NM_HOST, taskAttemptInfo.getHostname());
           Assert.assertEquals(MRApp.NM_HOST, taskAttemptInfo.getHostname());
           Assert.assertEquals(MRApp.NM_PORT, taskAttemptInfo.getPort());
           Assert.assertEquals(MRApp.NM_PORT, taskAttemptInfo.getPort());
-          
+
           // Verify rack-name
           // 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
     // test output for HistoryViewer
-    PrintStream stdps=System.out;
+    PrintStream stdps = System.out;
     try {
     try {
       System.setOut(new PrintStream(outContent));
       System.setOut(new PrintStream(outContent));
       HistoryViewer viewer = new HistoryViewer(fc.makeQualified(
       HistoryViewer viewer = new HistoryViewer(fc.makeQualified(
           fileInfo.getHistoryFile()).toString(), conf, true);
           fileInfo.getHistoryFile()).toString(), conf, true);
       viewer.print();
       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 {
     } finally {
       System.setOut(stdps);
       System.setOut(stdps);
@@ -363,186 +374,180 @@ public class TestJobHistoryParsing {
     }
     }
     return numFinishedMaps;
     return numFinishedMaps;
   }
   }
-  
-  @Test (timeout=50000)
+
+  @Test(timeout = 30000)
   public void testHistoryParsingForFailedAttempts() throws Exception {
   public void testHistoryParsingForFailedAttempts() throws Exception {
     LOG.info("STARTING testHistoryParsingForFailedAttempts");
     LOG.info("STARTING testHistoryParsingForFailedAttempts");
     try {
     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 {
     } finally {
       LOG.info("FINISHED testHistoryParsingForFailedAttempts");
       LOG.info("FINISHED testHistoryParsingForFailedAttempts");
     }
     }
   }
   }
-  
-  @Test (timeout=5000)
+
+  @Test(timeout = 60000)
   public void testCountersForFailedTask() throws Exception {
   public void testCountersForFailedTask() throws Exception {
     LOG.info("STARTING testCountersForFailedTask");
     LOG.info("STARTING testCountersForFailedTask");
     try {
     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 {
     } finally {
       LOG.info("FINISHED testCountersForFailedTask");
       LOG.info("FINISHED testCountersForFailedTask");
     }
     }
   }
   }
 
 
-  @Test (timeout=50000)
+  @Test(timeout = 50000)
   public void testScanningOldDirs() throws Exception {
   public void testScanningOldDirs() throws Exception {
     LOG.info("STARTING testScanningOldDirs");
     LOG.info("STARTING testScanningOldDirs");
     try {
     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");
       LOG.info("FINISHED testScanningOldDirs");
     }
     }
   }
   }
 
 
   static class MRAppWithHistoryWithFailedAttempt extends MRAppWithHistory {
   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);
       super(maps, reduces, autoComplete, testName, cleanOnStart);
     }
     }
-    
+
     @SuppressWarnings("unchecked")
     @SuppressWarnings("unchecked")
     @Override
     @Override
     protected void attemptLaunched(TaskAttemptId attemptID) {
     protected void attemptLaunched(TaskAttemptId attemptID) {
@@ -558,8 +563,8 @@ public class TestJobHistoryParsing {
 
 
   static class MRAppWithHistoryWithFailedTask extends MRAppWithHistory {
   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);
       super(maps, reduces, autoComplete, testName, cleanOnStart);
     }
     }
 
 
@@ -587,4 +592,133 @@ public class TestJobHistoryParsing {
     t.testHistoryParsing();
     t.testHistoryParsing();
     t.testHistoryParsingForFailedAttempts();
     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) {
       if (workflowAdjacencies == null) {
         workflowAdjacencies = "";
         workflowAdjacencies = "";
       }
       }
+      String workflowTags = line.get("WORKFLOW_TAGS");
+      if (workflowTags == null) {
+        workflowTags = "";
+      }
       
       
 
 
       if (submitTime != null) {
       if (submitTime != null) {
@@ -104,7 +108,8 @@ public class Job20LineHistoryEventEmitter extends HistoryEventEmitter {
           new HashMap<JobACL, AccessControlList>();
           new HashMap<JobACL, AccessControlList>();
         return new JobSubmittedEvent(jobID, jobName, user,
         return new JobSubmittedEvent(jobID, jobName, user,
             that.originalSubmitTime, jobConf, jobACLs, jobQueueName,
             that.originalSubmitTime, jobConf, jobACLs, jobQueueName,
-            workflowId, workflowName, workflowNodeName, workflowAdjacencies);
+            workflowId, workflowName, workflowNodeName, workflowAdjacencies,
+            workflowTags);
       }
       }
 
 
       return null;
       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-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
   BREAKDOWN OF HADOOP-8562 SUBTASKS
 
 
     YARN-158. Yarn creating package-info.java must not depend on sh.
     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
     YARN-359. Fixing commands for container signalling in Windows. (Chris Nauroth
     via vinodkv)
     via vinodkv)
     
     
-    YARN-524 TestYarnVersionInfo failing if generated properties doesn't
-    include an SVN URL(stevel)
-
 Release 2.0.5-beta - UNRELEASED
 Release 2.0.5-beta - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -123,6 +126,12 @@ Release 2.0.5-beta - UNRELEASED
     YARN-458. YARN daemon addresses must be placed in many different configs. 
     YARN-458. YARN daemon addresses must be placed in many different configs. 
     (sandyr via tucu)
     (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
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -195,6 +204,16 @@ Release 2.0.5-beta - UNRELEASED
     to implement closeable so that they can be stopped when needed via
     to implement closeable so that they can be stopped when needed via
     RPC.stopProxy(). (Siddharth Seth via vinodkv)
     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
 Release 2.0.4-alpha - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   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
     YARN-200. yarn log does not output all needed information, and is in a
     binary format (Ravi Prakash via jlowe)
     binary format (Ravi Prakash via jlowe)
 
 
+    YARN-525. make CS node-locality-delay refreshable (Thomas Graves via jlowe)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     YARN-357. App submission should not be synchronized (daryn)
     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
      * $USER
      * Final, non-modifiable.
      * Final, non-modifiable.
      */
      */
-    USER("USER"),
+    USER(Shell.WINDOWS ? "USERNAME": "USER"),
     
     
     /**
     /**
      * $LOGNAME
      * $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());
       ctx.setResource(container.getResource());
 
 
       String jobUserName = System.getenv(ApplicationConstants.Environment.USER
       String jobUserName = System.getenv(ApplicationConstants.Environment.USER
-          .name());
+          .key());
       ctx.setUser(jobUserName);
       ctx.setUser(jobUserName);
       LOG.info("Setting user in ContainerLaunchContext to: " + 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.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.junit.AfterClass;
 import org.junit.AfterClass;
@@ -50,7 +51,7 @@ public class TestUnmanagedAMLauncher {
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 128);
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 128);
     if (yarnCluster == null) {
     if (yarnCluster == null) {
       yarnCluster = new MiniYARNCluster(
       yarnCluster = new MiniYARNCluster(
-          TestUnmanagedAMLauncher.class.getName(), 1, 1, 1);
+          TestUnmanagedAMLauncher.class.getSimpleName(), 1, 1, 1);
       yarnCluster.init(conf);
       yarnCluster.init(conf);
       yarnCluster.start();
       yarnCluster.start();
       URL url = Thread.currentThread().getContextClassLoader()
       URL url = Thread.currentThread().getContextClassLoader()
@@ -93,7 +94,7 @@ public class TestUnmanagedAMLauncher {
     return envClassPath;
     return envClassPath;
   }
   }
 
 
-  @Test(timeout=10000)
+  @Test(timeout=30000)
   public void testDSShell() throws Exception {
   public void testDSShell() throws Exception {
     String classpath = getTestRuntimeClasspath();
     String classpath = getTestRuntimeClasspath();
     String javaHome = System.getenv("JAVA_HOME");
     String javaHome = System.getenv("JAVA_HOME");
@@ -112,7 +113,8 @@ public class TestUnmanagedAMLauncher {
         javaHome
         javaHome
             + "/bin/java -Xmx512m "
             + "/bin/java -Xmx512m "
             + "org.apache.hadoop.yarn.applications.distributedshell.ApplicationMaster "
             + "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");
     LOG.info("Initializing Launcher");
     UnmanagedAMLauncher launcher = new UnmanagedAMLauncher(new Configuration(
     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 =
   public static final String RM_SCHEDULER_MAXIMUM_ALLOCATION_MB =
     YARN_PREFIX + "scheduler.maximum-allocation-mb";
     YARN_PREFIX + "scheduler.maximum-allocation-mb";
   public static final int DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB = 8192;
   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";
       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.*/
   /** Number of threads to handle scheduler interface.*/
   public static final String RM_SCHEDULER_CLIENT_THREAD_COUNT =
   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.io.IOException;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
-import java.util.Random;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Callable;
 import java.util.regex.Pattern;
 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.FileUtil;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Path;
 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.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.RunJar;
 import org.apache.hadoop.util.RunJar;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 
 
 /**
 /**
  * Download a single URL to the local disk.
  * 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> {
 public class FSDownload implements Callable<Path> {
 
 
   private static final Log LOG = LogFactory.getLog(FSDownload.class);
   private static final Log LOG = LogFactory.getLog(FSDownload.class);
-  
-  private Random rand;
+
   private FileContext files;
   private FileContext files;
   private final UserGroupInformation userUgi;
   private final UserGroupInformation userUgi;
   private Configuration conf;
   private Configuration conf;
@@ -71,13 +68,12 @@ public class FSDownload implements Callable<Path> {
 
 
 
 
   public FSDownload(FileContext files, UserGroupInformation ugi, Configuration conf,
   public FSDownload(FileContext files, UserGroupInformation ugi, Configuration conf,
-      Path destDirPath, LocalResource resource, Random rand) {
+      Path destDirPath, LocalResource resource) {
     this.conf = conf;
     this.conf = conf;
     this.destDirPath = destDirPath;
     this.destDirPath = destDirPath;
     this.files = files;
     this.files = files;
     this.userUgi = ugi;
     this.userUgi = ugi;
     this.resource = resource;
     this.resource = resource;
-    this.rand = rand;
   }
   }
 
 
   LocalResource getResource() {
   LocalResource getResource() {
@@ -270,11 +266,6 @@ public class FSDownload implements Callable<Path> {
     } catch (URISyntaxException e) {
     } catch (URISyntaxException e) {
       throw new IOException("Invalid resource", 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);
     createDir(destDirPath, cachePerms);
     final Path dst_work = new Path(destDirPath + "_tmp");
     final Path dst_work = new Path(destDirPath + "_tmp");
     createDir(dst_work, cachePerms);
     createDir(dst_work, cachePerms);
@@ -305,8 +296,6 @@ public class FSDownload implements Callable<Path> {
         files.delete(dst_work, true);
         files.delete(dst_work, true);
       } catch (FileNotFoundException ignore) {
       } catch (FileNotFoundException ignore) {
       }
       }
-      // clear ref to internal var
-      rand = null;
       conf = null;
       conf = null;
       resource = 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.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.jar.JarOutputStream;
 import java.util.jar.JarOutputStream;
 import java.util.jar.Manifest;
 import java.util.jar.Manifest;
 
 
@@ -66,6 +67,8 @@ import org.junit.Test;
 public class TestFSDownload {
 public class TestFSDownload {
 
 
   private static final Log LOG = LogFactory.getLog(TestFSDownload.class);
   private static final Log LOG = LogFactory.getLog(TestFSDownload.class);
+  private static AtomicLong uniqueNumberGenerator =
+    new AtomicLong(System.currentTimeMillis());
   
   
   @AfterClass
   @AfterClass
   public static void deleteTestDir() throws IOException {
   public static void deleteTestDir() throws IOException {
@@ -267,9 +270,11 @@ public class TestFSDownload {
     rsrcVis.put(rsrc, vis);
     rsrcVis.put(rsrc, vis);
     Path destPath = dirs.getLocalPathForWrite(
     Path destPath = dirs.getLocalPathForWrite(
         basedir.toString(), size, conf);
         basedir.toString(), size, conf);
+    destPath = new Path (destPath,
+      Long.toString(uniqueNumberGenerator.incrementAndGet()));
     FSDownload fsd =
     FSDownload fsd =
       new FSDownload(files, UserGroupInformation.getCurrentUser(), conf,
       new FSDownload(files, UserGroupInformation.getCurrentUser(), conf,
-          destPath, rsrc, new Random(sharedSeed));
+          destPath, rsrc);
     pending.put(rsrc, exec.submit(fsd));
     pending.put(rsrc, exec.submit(fsd));
 
 
     try {
     try {
@@ -320,9 +325,11 @@ public class TestFSDownload {
       rsrcVis.put(rsrc, vis);
       rsrcVis.put(rsrc, vis);
       Path destPath = dirs.getLocalPathForWrite(
       Path destPath = dirs.getLocalPathForWrite(
           basedir.toString(), sizes[i], conf);
           basedir.toString(), sizes[i], conf);
+      destPath = new Path (destPath,
+          Long.toString(uniqueNumberGenerator.incrementAndGet()));
       FSDownload fsd =
       FSDownload fsd =
           new FSDownload(files, UserGroupInformation.getCurrentUser(), conf,
           new FSDownload(files, UserGroupInformation.getCurrentUser(), conf,
-              destPath, rsrc, new Random(sharedSeed));
+              destPath, rsrc);
       pending.put(rsrc, exec.submit(fsd));
       pending.put(rsrc, exec.submit(fsd));
     }
     }
 
 
@@ -380,9 +387,10 @@ public class TestFSDownload {
     Path p = new Path(basedir, "" + 1);
     Path p = new Path(basedir, "" + 1);
     LocalResource rsrc = createTarFile(files, p, size, rand, vis);
     LocalResource rsrc = createTarFile(files, p, size, rand, vis);
     Path destPath = dirs.getLocalPathForWrite(basedir.toString(), size, conf);
     Path destPath = dirs.getLocalPathForWrite(basedir.toString(), size, conf);
+    destPath = new Path (destPath,
+        Long.toString(uniqueNumberGenerator.incrementAndGet()));
     FSDownload fsd = new FSDownload(files,
     FSDownload fsd = new FSDownload(files,
-        UserGroupInformation.getCurrentUser(), conf, destPath, rsrc,
-        new Random(sharedSeed));
+        UserGroupInformation.getCurrentUser(), conf, destPath, rsrc);
     pending.put(rsrc, exec.submit(fsd));
     pending.put(rsrc, exec.submit(fsd));
     
     
     try {
     try {
@@ -437,9 +445,10 @@ public class TestFSDownload {
     LocalResource rsrcjar = createJarFile(files, p, size, rand, vis);
     LocalResource rsrcjar = createJarFile(files, p, size, rand, vis);
     rsrcjar.setType(LocalResourceType.PATTERN);
     rsrcjar.setType(LocalResourceType.PATTERN);
     Path destPathjar = dirs.getLocalPathForWrite(basedir.toString(), size, conf);
     Path destPathjar = dirs.getLocalPathForWrite(basedir.toString(), size, conf);
+    destPathjar = new Path (destPathjar,
+        Long.toString(uniqueNumberGenerator.incrementAndGet()));
     FSDownload fsdjar = new FSDownload(files,
     FSDownload fsdjar = new FSDownload(files,
-        UserGroupInformation.getCurrentUser(), conf, destPathjar, rsrcjar,
-        new Random(sharedSeed));
+        UserGroupInformation.getCurrentUser(), conf, destPathjar, rsrcjar);
     pending.put(rsrcjar, exec.submit(fsdjar));
     pending.put(rsrcjar, exec.submit(fsdjar));
 
 
     try {
     try {
@@ -493,9 +502,10 @@ public class TestFSDownload {
     Path p = new Path(basedir, "" + 1);
     Path p = new Path(basedir, "" + 1);
     LocalResource rsrczip = createZipFile(files, p, size, rand, vis);
     LocalResource rsrczip = createZipFile(files, p, size, rand, vis);
     Path destPathjar = dirs.getLocalPathForWrite(basedir.toString(), size, conf);
     Path destPathjar = dirs.getLocalPathForWrite(basedir.toString(), size, conf);
+    destPathjar = new Path (destPathjar,
+        Long.toString(uniqueNumberGenerator.incrementAndGet()));
     FSDownload fsdzip = new FSDownload(files,
     FSDownload fsdzip = new FSDownload(files,
-        UserGroupInformation.getCurrentUser(), conf, destPathjar, rsrczip,
-        new Random(sharedSeed));
+        UserGroupInformation.getCurrentUser(), conf, destPathjar, rsrczip);
     pending.put(rsrczip, exec.submit(fsdzip));
     pending.put(rsrczip, exec.submit(fsdzip));
 
 
     try {
     try {
@@ -586,9 +596,11 @@ public class TestFSDownload {
       rsrcVis.put(rsrc, vis);
       rsrcVis.put(rsrc, vis);
       Path destPath = dirs.getLocalPathForWrite(
       Path destPath = dirs.getLocalPathForWrite(
           basedir.toString(), conf);
           basedir.toString(), conf);
+      destPath = new Path (destPath,
+          Long.toString(uniqueNumberGenerator.incrementAndGet()));
       FSDownload fsd =
       FSDownload fsd =
           new FSDownload(files, UserGroupInformation.getCurrentUser(), conf,
           new FSDownload(files, UserGroupInformation.getCurrentUser(), conf,
-              destPath, rsrc, new Random(sharedSeed));
+              destPath, rsrc);
       pending.put(rsrc, exec.submit(fsd));
       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 NodeHealthCheckerService healthChecker;
   private final NodeManagerMetrics metrics;
   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,
   public NodeStatusUpdaterImpl(Context context, Dispatcher dispatcher,
       NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
       NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
@@ -99,7 +98,6 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     this.context = context;
     this.context = context;
     this.dispatcher = dispatcher;
     this.dispatcher = dispatcher;
     this.metrics = metrics;
     this.metrics = metrics;
-    this.previousHeartBeatSucceeded = true;
   }
   }
 
 
   @Override
   @Override
@@ -137,8 +135,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
             YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS);
             YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS);
     
     
     LOG.info("Initialized nodemanager for " + nodeId + ":" +
     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);
     super.init(conf);
   }
   }
@@ -192,12 +190,12 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
 
   private void registerWithRM() throws YarnRemoteException {
   private void registerWithRM() throws YarnRemoteException {
     Configuration conf = getConfig();
     Configuration conf = getConfig();
-    long rmConnectWaitMS =
+    rmConnectWaitMS =
         conf.getInt(
         conf.getInt(
             YarnConfiguration.RESOURCEMANAGER_CONNECT_WAIT_SECS,
             YarnConfiguration.RESOURCEMANAGER_CONNECT_WAIT_SECS,
             YarnConfiguration.DEFAULT_RESOURCEMANAGER_CONNECT_WAIT_SECS)
             YarnConfiguration.DEFAULT_RESOURCEMANAGER_CONNECT_WAIT_SECS)
         * 1000;
         * 1000;
-    long rmConnectionRetryIntervalMS =
+    rmConnectionRetryIntervalMS =
         conf.getLong(
         conf.getLong(
             YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_SECS,
             YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_SECS,
             YarnConfiguration
             YarnConfiguration
@@ -210,7 +208,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
           " should not be negative.");
           " should not be negative.");
     }
     }
 
 
-    boolean waitForEver = (rmConnectWaitMS == -1000);
+    waitForEver = (rmConnectWaitMS == -1000);
 
 
     if(! waitForEver) {
     if(! waitForEver) {
       if(rmConnectWaitMS < 0) {
       if(rmConnectWaitMS < 0) {
@@ -319,14 +317,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     NodeStatus nodeStatus = recordFactory.newRecordInstance(NodeStatus.class);
     NodeStatus nodeStatus = recordFactory.newRecordInstance(NodeStatus.class);
     nodeStatus.setNodeId(this.nodeId);
     nodeStatus.setNodeId(this.nodeId);
 
 
-    List<ContainerStatus> containersStatuses = new ArrayList<ContainerStatus>();
-    if(previousHeartBeatSucceeded) {
-      previousContainersStatuses.clear();
-    } else {
-      containersStatuses.addAll(previousContainersStatuses);
-    }
-
     int numActiveContainers = 0;
     int numActiveContainers = 0;
+    List<ContainerStatus> containersStatuses = new ArrayList<ContainerStatus>();
     for (Iterator<Entry<ContainerId, Container>> i =
     for (Iterator<Entry<ContainerId, Container>> i =
         this.context.getContainers().entrySet().iterator(); i.hasNext();) {
         this.context.getContainers().entrySet().iterator(); i.hasNext();) {
       Entry<ContainerId, Container> e = i.next();
       Entry<ContainerId, Container> e = i.next();
@@ -341,7 +333,6 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
       LOG.info("Sending out status for container: " + containerStatus);
       LOG.info("Sending out status for container: " + containerStatus);
 
 
       if (containerStatus.getState() == ContainerState.COMPLETE) {
       if (containerStatus.getState() == ContainerState.COMPLETE) {
-        previousContainersStatuses.add(containerStatus);
         // Remove
         // Remove
         i.remove();
         i.remove();
 
 
@@ -404,6 +395,9 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         while (!isStopped) {
         while (!isStopped) {
           // Send heartbeat
           // Send heartbeat
           try {
           try {
+            NodeHeartbeatResponse response = null;
+            int rmRetryCount = 0;
+            long waitStartTime = System.currentTimeMillis();
             NodeStatus nodeStatus = getNodeStatus();
             NodeStatus nodeStatus = getNodeStatus();
             nodeStatus.setResponseId(lastHeartBeatID);
             nodeStatus.setResponseId(lastHeartBeatID);
             
             
@@ -414,9 +408,31 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
               request.setLastKnownMasterKey(NodeStatusUpdaterImpl.this.context
               request.setLastKnownMasterKey(NodeStatusUpdaterImpl.this.context
                 .getContainerTokenSecretManager().getCurrentKey());
                 .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
             //get next heartbeat interval from response
             nextHeartBeatInterval = response.getNextHeartBeatInterval();
             nextHeartBeatInterval = response.getNextHeartBeatInterval();
             // See if the master-key has rolled over
             // See if the master-key has rolled over
@@ -432,7 +448,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
             if (response.getNodeAction() == NodeAction.SHUTDOWN) {
             if (response.getNodeAction() == NodeAction.SHUTDOWN) {
               LOG
               LOG
                   .info("Recieved SHUTDOWN signal from Resourcemanager as part of heartbeat," +
                   .info("Recieved SHUTDOWN signal from Resourcemanager as part of heartbeat," +
-                  		" hence shutting down.");
+                      " hence shutting down.");
               dispatcher.getEventHandler().handle(
               dispatcher.getEventHandler().handle(
                   new NodeManagerEvent(NodeManagerEventType.SHUTDOWN));
                   new NodeManagerEvent(NodeManagerEventType.SHUTDOWN));
               break;
               break;
@@ -461,8 +477,12 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
               dispatcher.getEventHandler().handle(
               dispatcher.getEventHandler().handle(
                   new CMgrCompletedAppsEvent(appsToCleanup));
                   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) {
           } catch (Throwable e) {
-            previousHeartBeatSucceeded = false;
             // TODO Better error handling. Thread can die with the rest of the
             // TODO Better error handling. Thread can die with the rest of the
             // NM still running.
             // NM still running.
             LOG.error("Caught exception in status-updater", e);
             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;
 package org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords;
 
 
 import java.util.List;
 import java.util.List;
-
-import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.server.nodemanager.api.*;
 
 
 public interface LocalizerHeartbeatResponse {
 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 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.Iterator;
 import java.util.List;
 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.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.LocalizerActionProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerHeartbeatResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerHeartbeatResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerHeartbeatResponseProtoOrBuilder;
 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.LocalizerAction;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
 
 
@@ -40,13 +41,14 @@ public class LocalizerHeartbeatResponsePBImpl
   LocalizerHeartbeatResponseProto.Builder builder = null;
   LocalizerHeartbeatResponseProto.Builder builder = null;
   boolean viaProto = false;
   boolean viaProto = false;
 
 
-  private List<LocalResource> resources;
+  private List<ResourceLocalizationSpec> resourceSpecs;
 
 
   public LocalizerHeartbeatResponsePBImpl() {
   public LocalizerHeartbeatResponsePBImpl() {
     builder = LocalizerHeartbeatResponseProto.newBuilder();
     builder = LocalizerHeartbeatResponseProto.newBuilder();
   }
   }
 
 
-  public LocalizerHeartbeatResponsePBImpl(LocalizerHeartbeatResponseProto proto) {
+  public LocalizerHeartbeatResponsePBImpl(
+      LocalizerHeartbeatResponseProto proto) {
     this.proto = proto;
     this.proto = proto;
     viaProto = true;
     viaProto = true;
   }
   }
@@ -59,7 +61,7 @@ public class LocalizerHeartbeatResponsePBImpl
   }
   }
 
 
   private void mergeLocalToBuilder() {
   private void mergeLocalToBuilder() {
-    if (resources != null) {
+    if (resourceSpecs != null) {
       addResourcesToProto();
       addResourcesToProto();
     }
     }
   }
   }
@@ -79,6 +81,7 @@ public class LocalizerHeartbeatResponsePBImpl
     viaProto = false;
     viaProto = false;
   }
   }
 
 
+  @Override
   public LocalizerAction getLocalizerAction() {
   public LocalizerAction getLocalizerAction() {
     LocalizerHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
     LocalizerHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
     if (!p.hasAction()) {
     if (!p.hasAction()) {
@@ -87,14 +90,10 @@ public class LocalizerHeartbeatResponsePBImpl
     return convertFromProtoFormat(p.getAction());
     return convertFromProtoFormat(p.getAction());
   }
   }
 
 
-  public List<LocalResource> getAllResources() {
-    initResources();
-    return this.resources;
-  }
-
-  public LocalResource getLocalResource(int i) {
+  @Override
+  public List<ResourceLocalizationSpec> getResourceSpecs() {
     initResources();
     initResources();
-    return this.resources.get(i);
+    return this.resourceSpecs;
   }
   }
 
 
   public void setLocalizerAction(LocalizerAction action) {
   public void setLocalizerAction(LocalizerAction action) {
@@ -106,31 +105,39 @@ public class LocalizerHeartbeatResponsePBImpl
     builder.setAction(convertToProtoFormat(action));
     builder.setAction(convertToProtoFormat(action));
   }
   }
 
 
+  public void setResourceSpecs(List<ResourceLocalizationSpec> rsrcs) {
+    maybeInitBuilder();
+    if (rsrcs == null) {
+      builder.clearResources();
+      return;
+    }
+    this.resourceSpecs = rsrcs;
+  }
+
   private void initResources() {
   private void initResources() {
-    if (this.resources != null) {
+    if (this.resourceSpecs != null) {
       return;
       return;
     }
     }
     LocalizerHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
     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() {
   private void addResourcesToProto() {
     maybeInitBuilder();
     maybeInitBuilder();
     builder.clearResources();
     builder.clearResources();
-    if (this.resources == null) 
+    if (this.resourceSpecs == null) 
       return;
       return;
-    Iterable<LocalResourceProto> iterable =
-        new Iterable<LocalResourceProto>() {
+    Iterable<ResourceLocalizationSpecProto> iterable =
+        new Iterable<ResourceLocalizationSpecProto>() {
       @Override
       @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
           @Override
           public boolean hasNext() {
           public boolean hasNext() {
@@ -138,8 +145,10 @@ public class LocalizerHeartbeatResponsePBImpl
           }
           }
 
 
           @Override
           @Override
-          public LocalResourceProto next() {
-            return convertToProtoFormat(iter.next());
+          public ResourceLocalizationSpecProto next() {
+            ResourceLocalizationSpec resource = iter.next();
+            
+            return ((ResourceLocalizationSpecPBImpl)resource).getProto();
           }
           }
 
 
           @Override
           @Override
@@ -154,34 +163,10 @@ public class LocalizerHeartbeatResponsePBImpl
     builder.addAllResources(iterable);
     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) {
   private LocalizerActionProto convertToProtoFormat(LocalizerAction a) {
@@ -191,5 +176,4 @@ public class LocalizerHeartbeatResponsePBImpl
   private LocalizerAction convertFromProtoFormat(LocalizerActionProto a) {
   private LocalizerAction convertFromProtoFormat(LocalizerActionProto a) {
     return LocalizerAction.valueOf(a.name());
     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.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 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.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocol;
 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.LocalResourceStatus;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
 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.LocalizerStatus;
@@ -89,8 +91,6 @@ public class ContainerLocalizer {
   private final String localizerId;
   private final String localizerId;
   private final FileContext lfs;
   private final FileContext lfs;
   private final Configuration conf;
   private final Configuration conf;
-  private final LocalDirAllocator appDirs;
-  private final LocalDirAllocator userDirs;
   private final RecordFactory recordFactory;
   private final RecordFactory recordFactory;
   private final Map<LocalResource,Future<Path>> pendingResources;
   private final Map<LocalResource,Future<Path>> pendingResources;
   private final String appCacheDirContextName;
   private final String appCacheDirContextName;
@@ -112,8 +112,6 @@ public class ContainerLocalizer {
     this.recordFactory = recordFactory;
     this.recordFactory = recordFactory;
     this.conf = new Configuration();
     this.conf = new Configuration();
     this.appCacheDirContextName = String.format(APPCACHE_CTXT_FMT, appId);
     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>>();
     this.pendingResources = new HashMap<LocalResource,Future<Path>>();
   }
   }
 
 
@@ -197,10 +195,10 @@ public class ContainerLocalizer {
     return new ExecutorCompletionService<Path>(exec);
     return new ExecutorCompletionService<Path>(exec);
   }
   }
 
 
-  Callable<Path> download(LocalDirAllocator lda, LocalResource rsrc,
+  Callable<Path> download(Path path, LocalResource rsrc,
       UserGroupInformation ugi) throws IOException {
       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) {
   static long getEstimatedSize(LocalResource rsrc) {
@@ -238,25 +236,12 @@ public class ContainerLocalizer {
         LocalizerHeartbeatResponse response = nodemanager.heartbeat(status);
         LocalizerHeartbeatResponse response = nodemanager.heartbeat(status);
         switch (response.getLocalizerAction()) {
         switch (response.getLocalizerAction()) {
         case LIVE:
         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;
           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.LinkedList;
 import java.util.Queue;
 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.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 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
   // TODO: Remove this in favour of EventHandler.handle
   void localizationCompleted(LocalResourceRequest req, boolean success);
   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.Iterator;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Matcher;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
 
 
@@ -66,6 +67,12 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
    */
    */
   private ConcurrentHashMap<LocalResourceRequest, Path>
   private ConcurrentHashMap<LocalResourceRequest, Path>
     inProgressLocalResourcesMap;
     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,
   public LocalResourcesTrackerImpl(String user, Dispatcher dispatcher,
       boolean useLocalCacheDirectoryManager, Configuration conf) {
       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.LinkedList;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
-import java.util.Random;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ConcurrentHashMap;
 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.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
 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.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.LocalResourceStatus;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAction;
 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.LocalizerHeartbeatResponse;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus;
 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.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
 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.event.ResourceRequestEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security.LocalizerTokenSecretManager;
 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.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.AbstractService;
 import org.apache.hadoop.yarn.service.CompositeService;
 import org.apache.hadoop.yarn.service.CompositeService;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
@@ -326,7 +328,7 @@ public class ResourceLocalizationService extends CompositeService
     // 0) Create application tracking structs
     // 0) Create application tracking structs
     String userName = app.getUser();
     String userName = app.getUser();
     privateRsrc.putIfAbsent(userName, new LocalResourcesTrackerImpl(userName,
     privateRsrc.putIfAbsent(userName, new LocalResourcesTrackerImpl(userName,
-      dispatcher, false, super.getConfig()));
+      dispatcher, true, super.getConfig()));
     if (null != appRsrc.putIfAbsent(
     if (null != appRsrc.putIfAbsent(
       ConverterUtils.toString(app.getAppId()),
       ConverterUtils.toString(app.getAppId()),
       new LocalResourcesTrackerImpl(app.getUser(), dispatcher, false, super
       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
    * Sub-component handling the spawning of {@link ContainerLocalizer}s
    */
    */
@@ -648,8 +665,11 @@ public class ResourceLocalizationService extends CompositeService
               DiskChecker.checkDir(
               DiskChecker.checkDir(
                 new File(publicDirDestPath.toUri().getPath()));
                 new File(publicDirDestPath.toUri().getPath()));
             }
             }
+            publicDirDestPath =
+                new Path(publicDirDestPath, Long.toString(publicRsrc
+                  .nextUniqueNumber()));
             pending.put(queue.submit(new FSDownload(
             pending.put(queue.submit(new FSDownload(
-                lfs, null, conf, publicDirDestPath, resource, new Random())),
+                lfs, null, conf, publicDirDestPath, resource)),
                 request);
                 request);
             attempts.put(key, new LinkedList<LocalizerResourceRequestEvent>());
             attempts.put(key, new LinkedList<LocalizerResourceRequestEvent>());
           } catch (IOException e) {
           } catch (IOException e) {
@@ -803,7 +823,20 @@ public class ResourceLocalizationService extends CompositeService
         LocalResource next = findNextResource();
         LocalResource next = findNextResource();
         if (next != null) {
         if (next != null) {
           response.setLocalizerAction(LocalizerAction.LIVE);
           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()) {
         } else if (pending.isEmpty()) {
           // TODO: Synchronization
           // TODO: Synchronization
           response.setLocalizerAction(LocalizerAction.DIE);
           response.setLocalizerAction(LocalizerAction.DIE);
@@ -812,7 +845,8 @@ public class ResourceLocalizationService extends CompositeService
         }
         }
         return response;
         return response;
       }
       }
-
+      ArrayList<ResourceLocalizationSpec> rsrcs =
+          new ArrayList<ResourceLocalizationSpec>();
       for (LocalResourceStatus stat : remoteResourceStatuses) {
       for (LocalResourceStatus stat : remoteResourceStatuses) {
         LocalResource rsrc = stat.getResource();
         LocalResource rsrc = stat.getResource();
         LocalResourceRequest req = null;
         LocalResourceRequest req = null;
@@ -835,6 +869,7 @@ public class ResourceLocalizationService extends CompositeService
                   new ResourceLocalizedEvent(req,
                   new ResourceLocalizedEvent(req,
                     ConverterUtils.getPathFromYarnURL(stat.getLocalPath()),
                     ConverterUtils.getPathFromYarnURL(stat.getLocalPath()),
                     stat.getLocalSize()));
                     stat.getLocalSize()));
+              localizationCompleted(stat);
             } catch (URISyntaxException e) { }
             } catch (URISyntaxException e) { }
             if (pending.isEmpty()) {
             if (pending.isEmpty()) {
               // TODO: Synchronization
               // TODO: Synchronization
@@ -844,7 +879,17 @@ public class ResourceLocalizationService extends CompositeService
             response.setLocalizerAction(LocalizerAction.LIVE);
             response.setLocalizerAction(LocalizerAction.LIVE);
             LocalResource next = findNextResource();
             LocalResource next = findNextResource();
             if (next != null) {
             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;
             break;
           case FETCH_PENDING:
           case FETCH_PENDING:
@@ -854,6 +899,7 @@ public class ResourceLocalizationService extends CompositeService
             LOG.info("DEBUG: FAILED " + req, stat.getException());
             LOG.info("DEBUG: FAILED " + req, stat.getException());
             assoc.getResource().unlock();
             assoc.getResource().unlock();
             response.setLocalizerAction(LocalizerAction.DIE);
             response.setLocalizerAction(LocalizerAction.DIE);
+            localizationCompleted(stat);
             // TODO: Why is this event going directly to the container. Why not
             // TODO: Why is this event going directly to the container. Why not
             // the resource itself? What happens to the resource? Is it removed?
             // the resource itself? What happens to the resource? Is it removed?
             dispatcher.getEventHandler().handle(
             dispatcher.getEventHandler().handle(
@@ -869,9 +915,53 @@ public class ResourceLocalizationService extends CompositeService
             break;
             break;
         }
         }
       }
       }
+      response.setResourceSpecs(rsrcs);
       return response;
       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
     @Override
     @SuppressWarnings("unchecked") // dispatcher not typed
     @SuppressWarnings("unchecked") // dispatcher not typed
     public void run() {
     public void run() {
@@ -1033,4 +1123,4 @@ public class ResourceLocalizationService extends CompositeService
     del.delete(null, dirPath, new Path[] {});
     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;
   DIE = 2;
 }
 }
 
 
+message ResourceLocalizationSpecProto {
+  optional LocalResourceProto resource = 1;
+  optional URLProto destination_directory = 2;
+}
+
 message LocalizerHeartbeatResponseProto {
 message LocalizerHeartbeatResponseProto {
   optional LocalizerActionProto action = 1;
   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.Map;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicInteger;
 
 
 import org.apache.commons.logging.Log;
 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.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 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.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 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.ResourceTracker;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
@@ -167,6 +170,10 @@ public class TestNodeStatusUpdater {
         throws YarnRemoteException {
         throws YarnRemoteException {
       NodeStatus nodeStatus = request.getNodeStatus();
       NodeStatus nodeStatus = request.getNodeStatus();
       LOG.info("Got heartbeat number " + heartBeatID);
       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++);
       nodeStatus.setResponseId(heartBeatID++);
       Map<ApplicationId, List<ContainerStatus>> appToContainers =
       Map<ApplicationId, List<ContainerStatus>> appToContainers =
           getAppToContainerStatusMap(nodeStatus.getContainersStatuses());
           getAppToContainerStatusMap(nodeStatus.getContainersStatuses());
@@ -183,7 +190,8 @@ public class TestNodeStatusUpdater {
         launchContext.setContainerId(firstContainerID);
         launchContext.setContainerId(firstContainerID);
         launchContext.setResource(recordFactory.newRecordInstance(Resource.class));
         launchContext.setResource(recordFactory.newRecordInstance(Resource.class));
         launchContext.getResource().setMemory(2);
         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);
         this.context.getContainers().put(firstContainerID, container);
       } else if (heartBeatID == 2) {
       } else if (heartBeatID == 2) {
         // Checks on the RM end
         // Checks on the RM end
@@ -207,7 +215,8 @@ public class TestNodeStatusUpdater {
         launchContext.setContainerId(secondContainerID);
         launchContext.setContainerId(secondContainerID);
         launchContext.setResource(recordFactory.newRecordInstance(Resource.class));
         launchContext.setResource(recordFactory.newRecordInstance(Resource.class));
         launchContext.getResource().setMemory(3);
         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);
         this.context.getContainers().put(secondContainerID, container);
       } else if (heartBeatID == 3) {
       } else if (heartBeatID == 3) {
         // Checks on the RM end
         // Checks on the RM end
@@ -229,13 +238,14 @@ public class TestNodeStatusUpdater {
   }
   }
 
 
   private class MyNodeStatusUpdater extends NodeStatusUpdaterImpl {
   private class MyNodeStatusUpdater extends NodeStatusUpdaterImpl {
-    public ResourceTracker resourceTracker = new MyResourceTracker(this.context);
+    public ResourceTracker resourceTracker;
     private Context context;
     private Context context;
 
 
     public MyNodeStatusUpdater(Context context, Dispatcher dispatcher,
     public MyNodeStatusUpdater(Context context, Dispatcher dispatcher,
         NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
         NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
       super(context, dispatcher, healthChecker, metrics);
       super(context, dispatcher, healthChecker, metrics);
       this.context = context;
       this.context = context;
+      resourceTracker = new MyResourceTracker(this.context);
     }
     }
 
 
     @Override
     @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 class MyNodeManager extends NodeManager {
     
     
     private MyNodeStatusUpdater3 nodeStatusUpdater;
     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 {
   private class MyResourceTracker2 implements ResourceTracker {
     public NodeAction heartBeatNodeAction = NodeAction.NORMAL;
     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
   @Before
   public void clearError() {
   public void clearError() {
     nmStartError = null;
     nmStartError = null;
@@ -883,6 +954,30 @@ public class TestNodeStatusUpdater {
     nm.stop();
     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 {
   private class MyNMContext extends NMContext {
     ConcurrentMap<ContainerId, Container> containers =
     ConcurrentMap<ContainerId, Container> containers =
         new ConcurrentSkipListMap<ContainerId, Container>();
         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;
 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.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataInputBuffer;
 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.LocalResourceStatusProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerHeartbeatResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerHeartbeatResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalizerStatusProto;
 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.LocalResourceStatus;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAction;
 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.LocalizerHeartbeatResponse;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus;
 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.api.protocolrecords.ResourceStatusType;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
-
 import org.junit.Test;
 import org.junit.Test;
-import static org.junit.Assert.*;
 
 
 public class TestPBRecordImpl {
 public class TestPBRecordImpl {
 
 
@@ -54,9 +60,8 @@ public class TestPBRecordImpl {
   static LocalResource createResource() {
   static LocalResource createResource() {
     LocalResource ret = recordFactory.newRecordInstance(LocalResource.class);
     LocalResource ret = recordFactory.newRecordInstance(LocalResource.class);
     assertTrue(ret instanceof LocalResourcePBImpl);
     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.setSize(4344L);
     ret.setTimestamp(3141592653589793L);
     ret.setTimestamp(3141592653589793L);
     ret.setVisibility(LocalResourceVisibility.PUBLIC);
     ret.setVisibility(LocalResourceVisibility.PUBLIC);
@@ -90,16 +95,27 @@ public class TestPBRecordImpl {
     return ret;
     return ret;
   }
   }
 
 
-  static LocalizerHeartbeatResponse createLocalizerHeartbeatResponse() {
+  static LocalizerHeartbeatResponse createLocalizerHeartbeatResponse() 
+      throws URISyntaxException {
     LocalizerHeartbeatResponse ret =
     LocalizerHeartbeatResponse ret =
       recordFactory.newRecordInstance(LocalizerHeartbeatResponse.class);
       recordFactory.newRecordInstance(LocalizerHeartbeatResponse.class);
     assertTrue(ret instanceof LocalizerHeartbeatResponsePBImpl);
     assertTrue(ret instanceof LocalizerHeartbeatResponsePBImpl);
     ret.setLocalizerAction(LocalizerAction.LIVE);
     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;
     return ret;
   }
   }
 
 
-  @Test
+  @Test(timeout=10000)
   public void testLocalResourceStatusSerDe() throws Exception {
   public void testLocalResourceStatusSerDe() throws Exception {
     LocalResourceStatus rsrcS = createLocalResourceStatus();
     LocalResourceStatus rsrcS = createLocalResourceStatus();
     assertTrue(rsrcS instanceof LocalResourceStatusPBImpl);
     assertTrue(rsrcS instanceof LocalResourceStatusPBImpl);
@@ -119,7 +135,7 @@ public class TestPBRecordImpl {
     assertEquals(createResource(), rsrcD.getResource());
     assertEquals(createResource(), rsrcD.getResource());
   }
   }
 
 
-  @Test
+  @Test(timeout=10000)
   public void testLocalizerStatusSerDe() throws Exception {
   public void testLocalizerStatusSerDe() throws Exception {
     LocalizerStatus rsrcS = createLocalizerStatus();
     LocalizerStatus rsrcS = createLocalizerStatus();
     assertTrue(rsrcS instanceof LocalizerStatusPBImpl);
     assertTrue(rsrcS instanceof LocalizerStatusPBImpl);
@@ -141,7 +157,7 @@ public class TestPBRecordImpl {
     assertEquals(createLocalResourceStatus(), rsrcD.getResourceStatus(0));
     assertEquals(createLocalResourceStatus(), rsrcD.getResourceStatus(0));
   }
   }
 
 
-  @Test
+  @Test(timeout=10000)
   public void testLocalizerHeartbeatResponseSerDe() throws Exception {
   public void testLocalizerHeartbeatResponseSerDe() throws Exception {
     LocalizerHeartbeatResponse rsrcS = createLocalizerHeartbeatResponse();
     LocalizerHeartbeatResponse rsrcS = createLocalizerHeartbeatResponse();
     assertTrue(rsrcS instanceof LocalizerHeartbeatResponsePBImpl);
     assertTrue(rsrcS instanceof LocalizerHeartbeatResponsePBImpl);
@@ -158,8 +174,8 @@ public class TestPBRecordImpl {
       new LocalizerHeartbeatResponsePBImpl(rsrcPbD);
       new LocalizerHeartbeatResponsePBImpl(rsrcPbD);
 
 
     assertEquals(rsrcS, rsrcD);
     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.ArrayList;
 import java.util.List;
 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.LocalizerAction;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
 
 
@@ -28,28 +28,30 @@ public class MockLocalizerHeartbeatResponse
     implements LocalizerHeartbeatResponse {
     implements LocalizerHeartbeatResponse {
 
 
   LocalizerAction action;
   LocalizerAction action;
-  List<LocalResource> rsrc;
+  List<ResourceLocalizationSpec> resourceSpecs;
 
 
   MockLocalizerHeartbeatResponse() {
   MockLocalizerHeartbeatResponse() {
-    rsrc = new ArrayList<LocalResource>();
+    resourceSpecs = new ArrayList<ResourceLocalizationSpec>();
   }
   }
 
 
   MockLocalizerHeartbeatResponse(
   MockLocalizerHeartbeatResponse(
-      LocalizerAction action, List<LocalResource> rsrc) {
+      LocalizerAction action, List<ResourceLocalizationSpec> resources) {
     this.action = action;
     this.action = action;
-    this.rsrc = rsrc;
+    this.resourceSpecs = resources;
   }
   }
 
 
   public LocalizerAction getLocalizerAction() { return action; }
   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) {
   public void setLocalizerAction(LocalizerAction action) {
     this.action = 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.AbstractFileSystem;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileContext;
-import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.DataInputBuffer;
 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.api.records.URL;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocol;
 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.LocalResourceStatus;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAction;
 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.server.nodemanager.api.protocolrecords.LocalizerStatus;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.ArgumentMatcher;
 import org.mockito.ArgumentMatcher;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.invocation.InvocationOnMock;
@@ -95,12 +96,33 @@ public class TestContainerLocalizer {
   public void testContainerLocalizerMain() throws Exception {
   public void testContainerLocalizerMain() throws Exception {
     ContainerLocalizer localizer = setupContainerLocalizerForTest();
     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
     // 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)))
     when(nmProxy.heartbeat(isA(LocalizerStatus.class)))
       .thenReturn(new MockLocalizerHeartbeatResponse(LocalizerAction.LIVE,
       .thenReturn(new MockLocalizerHeartbeatResponse(LocalizerAction.LIVE,
             Collections.singletonList(rsrcA)))
             Collections.singletonList(rsrcA)))
@@ -111,27 +133,33 @@ public class TestContainerLocalizer {
       .thenReturn(new MockLocalizerHeartbeatResponse(LocalizerAction.LIVE,
       .thenReturn(new MockLocalizerHeartbeatResponse(LocalizerAction.LIVE,
             Collections.singletonList(rsrcD)))
             Collections.singletonList(rsrcD)))
       .thenReturn(new MockLocalizerHeartbeatResponse(LocalizerAction.LIVE,
       .thenReturn(new MockLocalizerHeartbeatResponse(LocalizerAction.LIVE,
-            Collections.<LocalResource>emptyList()))
+            Collections.<ResourceLocalizationSpec>emptyList()))
       .thenReturn(new MockLocalizerHeartbeatResponse(LocalizerAction.DIE,
       .thenReturn(new MockLocalizerHeartbeatResponse(LocalizerAction.DIE,
             null));
             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));
         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));
         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));
         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));
         isA(UserGroupInformation.class));
 
 
     // run localization
     // run localization
     assertEquals(0, localizer.runLocalization(nmAddr));
     assertEquals(0, localizer.runLocalization(nmAddr));
-
-    // verify created cache
     for (Path p : localDirs) {
     for (Path p : localDirs) {
       Path base = new Path(new Path(p, ContainerLocalizer.USERCACHE), appUser);
       Path base = new Path(new Path(p, ContainerLocalizer.USERCACHE), appUser);
       Path privcache = new Path(base, ContainerLocalizer.FILECACHE);
       Path privcache = new Path(base, ContainerLocalizer.FILECACHE);
@@ -143,15 +171,14 @@ public class TestContainerLocalizer {
       Path appcache = new Path(appDir, ContainerLocalizer.FILECACHE);
       Path appcache = new Path(appDir, ContainerLocalizer.FILECACHE);
       verify(spylfs).mkdir(eq(appcache), isA(FsPermission.class), eq(false));
       verify(spylfs).mkdir(eq(appcache), isA(FsPermission.class), eq(false));
     }
     }
-
     // verify tokens read at expected location
     // verify tokens read at expected location
     verify(spylfs).open(tokenPath);
     verify(spylfs).open(tokenPath);
 
 
     // verify downloaded resources reported to NM
     // 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 all HB use localizerID provided
     verify(nmProxy, never()).heartbeat(argThat(
     verify(nmProxy, never()).heartbeat(argThat(
@@ -306,10 +333,12 @@ public class TestContainerLocalizer {
     return mockRF;
     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());
     String name = Long.toHexString(r.nextLong());
     URL uri = mock(org.apache.hadoop.yarn.api.records.URL.class);
     URL uri = mock(org.apache.hadoop.yarn.api.records.URL.class);
     when(uri.getScheme()).thenReturn("file");
     when(uri.getScheme()).thenReturn("file");
@@ -322,7 +351,10 @@ public class TestContainerLocalizer {
     when(rsrc.getType()).thenReturn(LocalResourceType.FILE);
     when(rsrc.getType()).thenReturn(LocalResourceType.FILE);
     when(rsrc.getVisibility()).thenReturn(vis);
     when(rsrc.getVisibility()).thenReturn(vis);
 
 
-    return rsrc;
+    when(resourceLocalizationSpec.getResource()).thenReturn(rsrc);
+    when(resourceLocalizationSpec.getDestinationDirectory()).
+      thenReturn(ConverterUtils.getYarnUrlFromPath(p));
+    return resourceLocalizationSpec;
   }
   }
 
 
   @SuppressWarnings({ "rawtypes", "unchecked" })
   @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;
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Matchers.anyLong;
 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.timeout;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.times;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
@@ -375,7 +377,7 @@ public class TestResourceLocalizationService {
     }
     }
   }
   }
   
   
-  @Test
+  @Test( timeout = 10000)
   @SuppressWarnings("unchecked") // mocked generics
   @SuppressWarnings("unchecked") // mocked generics
   public void testLocalizationHeartbeat() throws Exception {
   public void testLocalizationHeartbeat() throws Exception {
     Configuration conf = new YarnConfiguration();
     Configuration conf = new YarnConfiguration();
@@ -386,12 +388,17 @@ public class TestResourceLocalizationService {
         isA(Path.class), isA(FsPermission.class), anyBoolean());
         isA(Path.class), isA(FsPermission.class), anyBoolean());
 
 
     List<Path> localDirs = new ArrayList<Path>();
     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);
     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();
     String logDir = lfs.makeQualified(new Path(basedir, "logdir " )).toString();
     conf.set(YarnConfiguration.NM_LOG_DIRS, logDir);
     conf.set(YarnConfiguration.NM_LOG_DIRS, logDir);
     DrainDispatcher dispatcher = new DrainDispatcher();
     DrainDispatcher dispatcher = new DrainDispatcher();
@@ -452,12 +459,23 @@ public class TestResourceLocalizationService {
       doReturn(out).when(spylfs).createInternal(isA(Path.class),
       doReturn(out).when(spylfs).createInternal(isA(Path.class),
           isA(EnumSet.class), isA(FsPermission.class), anyInt(), anyShort(),
           isA(EnumSet.class), isA(FsPermission.class), anyInt(), anyShort(),
           anyLong(), isA(Progressable.class), isA(ChecksumOpt.class), anyBoolean());
           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 =
       Map<LocalResourceVisibility, Collection<LocalResourceRequest>> rsrcs =
         new HashMap<LocalResourceVisibility, 
         new HashMap<LocalResourceVisibility, 
                     Collection<LocalResourceRequest>>();
                     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));
       spyService.handle(new ContainerLocalizationRequestEvent(c, rsrcs));
       // Sigh. Thread init of private localizer not accessible
       // Sigh. Thread init of private localizer not accessible
       Thread.sleep(1000);
       Thread.sleep(1000);
@@ -471,33 +489,67 @@ public class TestResourceLocalizationService {
       Path localizationTokenPath = tokenPathCaptor.getValue();
       Path localizationTokenPath = tokenPathCaptor.getValue();
 
 
       // heartbeat from localizer
       // heartbeat from localizer
-      LocalResourceStatus rsrcStat = mock(LocalResourceStatus.class);
+      LocalResourceStatus rsrcStat1 = mock(LocalResourceStatus.class);
+      LocalResourceStatus rsrcStat2 = mock(LocalResourceStatus.class);
       LocalizerStatus stat = mock(LocalizerStatus.class);
       LocalizerStatus stat = mock(LocalizerStatus.class);
       when(stat.getLocalizerId()).thenReturn(ctnrStr);
       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");
       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())
       when(stat.getResources())
         .thenReturn(Collections.<LocalResourceStatus>emptyList())
         .thenReturn(Collections.<LocalResourceStatus>emptyList())
-        .thenReturn(Collections.singletonList(rsrcStat))
+        .thenReturn(Collections.singletonList(rsrcStat1))
+        .thenReturn(Collections.singletonList(rsrcStat2))
         .thenReturn(Collections.<LocalResourceStatus>emptyList());
         .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);
       LocalizerHeartbeatResponse response = spyService.heartbeat(stat);
       assertEquals(LocalizerAction.LIVE, response.getLocalizerAction());
       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
       // empty rsrc
       response = spyService.heartbeat(stat);
       response = spyService.heartbeat(stat);
       assertEquals(LocalizerAction.LIVE, response.getLocalizerAction());
       assertEquals(LocalizerAction.LIVE, response.getLocalizerAction());
-      assertEquals(0, response.getAllResources().size());
+      assertEquals(0, response.getResourceSpecs().size());
 
 
       // get shutdown
       // get shutdown
       response = spyService.heartbeat(stat);
       response = spyService.heartbeat(stat);
       assertEquals(LocalizerAction.DIE, response.getLocalizerAction());
       assertEquals(LocalizerAction.DIE, response.getLocalizerAction());
 
 
+
+      dispatcher.await();
       // verify container notification
       // verify container notification
       ArgumentMatcher<ContainerEvent> matchesContainerLoc =
       ArgumentMatcher<ContainerEvent> matchesContainerLoc =
         new ArgumentMatcher<ContainerEvent>() {
         new ArgumentMatcher<ContainerEvent>() {
@@ -508,9 +560,9 @@ public class TestResourceLocalizationService {
               && c.getContainerID() == evt.getContainerID();
               && 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 deletion of localization token.
       verify(delService).delete((String)isNull(), eq(localizationTokenPath));
       verify(delService).delete((String)isNull(), eq(localizationTokenPath));
     } finally {
     } 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.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 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.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
 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.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
 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.RMAppAttemptStatusupdateEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
 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.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.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.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
 import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.service.AbstractService;
@@ -276,6 +278,14 @@ public class ApplicationMasterService extends AbstractService implements
       List<ResourceRequest> ask = request.getAskList();
       List<ResourceRequest> ask = request.getAskList();
       List<ContainerId> release = request.getReleaseList();
       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.
       // Send new requests to appAttempt.
       Allocation allocation =
       Allocation allocation =
           this.rScheduler.allocate(appAttemptId, ask, release);
           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.NodeReport;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 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.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 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.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 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.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.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.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.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
@@ -273,6 +277,21 @@ public class ClientRMService extends AbstractService implements
       // Safety 
       // Safety 
       submissionContext.setUser(user);
       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 
       // This needs to be synchronous as the client can query 
       // immediately following the submission to get the application status.
       // immediately following the submission to get the application status.
       // So call handle directly and do not send an event.
       // 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 static final Log LOG = LogFactory.getLog(RMAppManager.class);
 
 
   private int completedAppsMax = YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS;
   private int completedAppsMax = YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS;
+  private int globalMaxAppAttempts;
   private LinkedList<ApplicationId> completedApps = new LinkedList<ApplicationId>();
   private LinkedList<ApplicationId> completedApps = new LinkedList<ApplicationId>();
 
 
   private final RMContext rmContext;
   private final RMContext rmContext;
@@ -76,6 +77,8 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     setCompletedAppsMax(conf.getInt(
     setCompletedAppsMax(conf.getInt(
         YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS,
         YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS,
         YarnConfiguration.DEFAULT_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();
     Map<ApplicationId, ApplicationState> appStates = state.getApplicationState();
     LOG.info("Recovering " + appStates.size() + " applications");
     LOG.info("Recovering " + appStates.size() + " applications");
     for(ApplicationState appState : appStates.values()) {
     for(ApplicationState appState : appStates.values()) {
+      boolean shouldRecover = true;
       // re-submit the application
       // re-submit the application
       // this is going to send an app start event but since the async dispatcher 
       // 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
       // 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 
         // 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
         // RM will re-connect with the running AM's instead of restarting them
         LOG.info("Not recovering unmanaged application " + appState.getAppId());
         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 {
       } 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());
         LOG.info("Recovering application " + appState.getAppId());
         submitApplication(appState.getApplicationSubmissionContext(), 
         submitApplication(appState.getApplicationSubmissionContext(), 
-                          appState.getSubmitTime());
+                        appState.getSubmitTime());
         // re-populate attempt information in application
         // re-populate attempt information in application
         RMAppImpl appImpl = (RMAppImpl) rmContext.getRMApps().get(
         RMAppImpl appImpl = (RMAppImpl) rmContext.getRMApps().get(
-                                                          appState.getAppId());
+                                                        appState.getAppId());
         appImpl.recover(state);
         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);
       this.applicationACLsManager, this.conf);
   }
   }
 
 
+  // sanity check for configurations
   protected static void validateConfigs(Configuration conf) {
   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);
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
     if (globalMaxAppAttempts <= 0) {
     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
   @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(
         roundUp(
             Math.max(r.getMemory(), minimumResource.getMemory()),
             Math.max(r.getMemory(), minimumResource.getMemory()),
-            minimumResource.getMemory())
-        );
+            minimumResource.getMemory()),
+            maximumResource.getMemory());
+    return Resources.createResource(normalizedMemory);
   }
   }
 
 
   @Override
   @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
   @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(
         roundUp(
-            Math.max(r.getMemory(), minimumResource.getMemory()), 
+            Math.max(r.getMemory(), minimumResource.getMemory()),
             minimumResource.getMemory()),
             minimumResource.getMemory()),
+            maximumResource.getMemory());
+    int normalizedCores = Math.min(
         roundUp(
         roundUp(
             Math.max(r.getVirtualCores(), minimumResource.getVirtualCores()),
             Math.max(r.getVirtualCores(), minimumResource.getVirtualCores()),
-            minimumResource.getVirtualCores())
-        );
+            minimumResource.getVirtualCores()),
+            maximumResource.getVirtualCores());
+    return Resources.createResource(normalizedMemory,
+        normalizedCores);
   }
   }
 
 
   @Override
   @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 
    * 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 r resource
    * @param minimumResource step-factor
    * @param minimumResource step-factor
+   * @param maximumResource the upper bound of the resource to be allocated
    * @return normalized resource
    * @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>.
    * 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(
   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(
   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
             AM_CONTAINER_PRIORITY, ResourceRequest.ANY, appAttempt.submissionContext
                 .getAMContainerSpec().getResource(), 1);
                 .getAMContainerSpec().getResource(), 1);
 
 
+        // SchedulerUtils.validateResourceRequests is not necessary because
+        // AM resource has been checked when submission
         Allocation amContainerAllocation = appAttempt.scheduler.allocate(
         Allocation amContainerAllocation = appAttempt.scheduler.allocate(
             appAttempt.applicationAttemptId,
             appAttempt.applicationAttemptId,
             Collections.singletonList(request), EMPTY_CONTAINER_RELEASE_LIST);
             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,
       List<ResourceRequest> asks,
       ResourceCalculator resourceCalculator, 
       ResourceCalculator resourceCalculator, 
       Resource clusterResource,
       Resource clusterResource,
-      Resource minimumResource) {
+      Resource minimumResource,
+      Resource maximumResource) {
     for (ResourceRequest ask : asks) {
     for (ResourceRequest ask : asks) {
       normalizeRequest(
       normalizeRequest(
-          ask, resourceCalculator, clusterResource, minimumResource);
+          ask, resourceCalculator, clusterResource, minimumResource,
+          maximumResource);
     }
     }
   }
   }
 
 
@@ -104,11 +106,50 @@ public class SchedulerUtils {
       ResourceRequest ask, 
       ResourceRequest ask, 
       ResourceCalculator resourceCalculator, 
       ResourceCalculator resourceCalculator, 
       Resource clusterResource,
       Resource clusterResource,
-      Resource minimumResource) {
+      Resource minimumResource,
+      Resource maximumResource) {
     Resource normalized = 
     Resource normalized = 
         Resources.normalize(
         Resources.normalize(
-            resourceCalculator, ask.getCapability(), minimumResource);
+            resourceCalculator, ask.getCapability(), minimumResource,
+            maximumResource);
     ask.setCapability(normalized);
     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
     // Sanity check
     SchedulerUtils.normalizeRequests(
     SchedulerUtils.normalizeRequests(
-        ask, calculator, getClusterResources(), minimumAllocation);
+        ask, calculator, getClusterResources(), minimumAllocation,
+        maximumAllocation);
 
 
     // Release containers
     // Release containers
     for (ContainerId releasedContainerId : release) {
     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.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
     int maximumCores = getInt(
     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);
     return Resources.createResource(maximumMemory, maximumCores);
   }
   }
 
 

Some files were not shown because too many files changed in this diff