Przeglądaj źródła

Made a number of miscellaneous fixes for javac, javadoc, and checstyle warnings.

Sangjin Lee 9 lat temu
rodzic
commit
6cf6ab7b78
78 zmienionych plików z 634 dodań i 543 usunięć
  1. 0 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
  2. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
  3. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
  4. 10 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java
  5. 15 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
  6. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
  7. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/EntityWriterV2.java
  8. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV2.java
  9. 7 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterConstants.java
  10. 4 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV1.java
  11. 6 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV1.java
  12. 3 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV2.java
  13. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineServicePerformance.java
  14. 3 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
  15. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/package-info.java
  16. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  17. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/TimelineServiceHelper.java
  18. 16 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  19. 14 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
  20. 0 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShellWithNodeLabels.java
  21. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
  22. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
  23. 19 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
  24. 26 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
  25. 10 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineServiceRecords.java
  26. 20 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java
  27. 14 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java
  28. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/CollectorNodemanagerProtocol.java
  29. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/CollectorNodemanagerProtocolPB.java
  30. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/CollectorNodemanagerProtocolPBClientImpl.java
  31. 4 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/CollectorNodemanagerProtocolPBServiceImpl.java
  32. 15 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetTimelineCollectorContextRequestPBImpl.java
  33. 15 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetTimelineCollectorContextResponsePBImpl.java
  34. 5 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
  35. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
  36. 9 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewCollectorInfoRequestPBImpl.java
  37. 7 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewCollectorInfoResponsePBImpl.java
  38. 9 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/AppCollectorsMapPBImpl.java
  39. 34 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
  40. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
  41. 2 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
  42. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  43. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationContainerFinishedEvent.java
  44. 0 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
  45. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
  46. 9 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
  47. 2 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
  48. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java
  49. 11 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
  50. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java
  51. 3 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockApp.java
  52. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
  53. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java
  54. 15 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
  55. 6 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
  56. 27 24
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
  57. 96 96
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
  58. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixOfflineAggregationWriterImpl.java
  59. 21 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java
  60. 7 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
  61. 6 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
  62. 18 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
  63. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java
  64. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java
  65. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
  66. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java
  67. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java
  68. 12 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestNMTimelineCollectorManager.java
  69. 8 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java
  70. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderUtils.java
  71. 7 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
  72. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesUtils.java
  73. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineUIDConverter.java
  74. 22 22
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java
  75. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java
  76. 12 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
  77. 14 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java
  78. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md

+ 0 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java

@@ -46,7 +46,6 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.TaskStatus;
 import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.CounterGroup;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.JobCounter;
 import org.apache.hadoop.mapreduce.MRJobConfig;
@@ -76,11 +75,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.node.ArrayNode;
 import org.codehaus.jackson.node.JsonNodeFactory;
-import org.codehaus.jackson.node.ObjectNode;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.sun.jersey.api.client.ClientHandlerException;

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java

@@ -829,7 +829,7 @@ public class RMContainerAllocator extends RMContainerRequestor
     if (collectorAddr != null && !collectorAddr.isEmpty()
         && appContext.getTimelineClient() != null) {
       appContext.getTimelineClient().setTimelineServiceAddress(
-        response.getCollectorAddr());
+          response.getCollectorAddr());
     }
 
     for (ContainerStatus cont : finishedContainers) {

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java

@@ -786,7 +786,8 @@ public class TestJobHistoryEventHandler {
     public TestParams(boolean isLastAMRetry) {
       this(AppContext.class, isLastAMRetry);
     }
-    public TestParams(Class<? extends AppContext> contextClass, boolean isLastAMRetry) {
+    public TestParams(Class<? extends AppContext> contextClass,
+        boolean isLastAMRetry) {
       this.isLastAMRetry = isLastAMRetry;
       mockAppContext = mockAppContext(contextClass, appId, this.isLastAMRetry);
     }

+ 10 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java

@@ -42,9 +42,17 @@ public interface HistoryEvent {
   /** Set the Avro datum wrapped by this. */
   void setDatum(Object datum);
 
-  /** Map HistoryEvent to TimelineEvent */
+  /**
+   * Map HistoryEvent to TimelineEvent.
+   *
+   * @return the timeline event
+   */
   TimelineEvent toTimelineEvent();
 
-  /** Counters or Metrics if any else return null. */
+  /**
+   * Counters or Metrics if any else return null.
+   *
+   * @return the set of timeline metrics
+   */
   Set<TimelineMetric> getTimelineMetrics();
 }

+ 15 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java

@@ -64,7 +64,7 @@ public class TestMRTimelineEventHandling {
 
   private static final String TIMELINE_AUX_SERVICE_NAME = "timeline_collector";
   private static final Log LOG =
-    LogFactory.getLog(TestMRTimelineEventHandling.class);
+      LogFactory.getLog(TestMRTimelineEventHandling.class);
 
   @Test
   public void testTimelineServiceStartInMiniCluster() throws Exception {
@@ -169,6 +169,7 @@ public class TestMRTimelineEventHandling {
     }
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void testMRNewTimelineServiceEventHandling() throws Exception {
     LOG.info("testMRNewTimelineServiceEventHandling start.");
@@ -183,7 +184,7 @@ public class TestMRTimelineEventHandling {
     // enable aux-service based timeline collectors
     conf.set(YarnConfiguration.NM_AUX_SERVICES, TIMELINE_AUX_SERVICE_NAME);
     conf.set(YarnConfiguration.NM_AUX_SERVICES + "." + TIMELINE_AUX_SERVICE_NAME
-      + ".class", PerNodeTimelineCollectorsAuxService.class.getName());
+        + ".class", PerNodeTimelineCollectorsAuxService.class.getName());
 
     conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, true);
 
@@ -245,7 +246,8 @@ public class TestMRTimelineEventHandling {
       }
       // Cleanup test file
       String testRoot =
-          FileSystemTimelineWriterImpl.DEFAULT_TIMELINE_SERVICE_STORAGE_DIR_ROOT;
+          FileSystemTimelineWriterImpl.
+              DEFAULT_TIMELINE_SERVICE_STORAGE_DIR_ROOT;
       File testRootFolder = new File(testRoot);
       if(testRootFolder.isDirectory()) {
         FileUtils.deleteDirectory(testRootFolder);
@@ -320,8 +322,10 @@ public class TestMRTimelineEventHandling {
         " does not exist.",
         taskFolder.isDirectory());
 
-    String taskEventFileName = appId.toString().replaceAll("application", "task")
-        + "_m_000000" + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
+    String taskEventFileName =
+        appId.toString().replaceAll("application", "task") +
+        "_m_000000" +
+        FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
 
     String taskEventFilePath = outputDirTask + taskEventFileName;
     File taskEventFile = new File(taskEventFilePath);
@@ -372,10 +376,12 @@ public class TestMRTimelineEventHandling {
       reader = new BufferedReader(new FileReader(entityFile));
       while ((strLine = reader.readLine()) != null) {
         if (strLine.trim().length() > 0) {
-          org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
-              FileSystemTimelineReaderImpl.getTimelineRecordFromJSON(
-                  strLine.trim(),
-                  org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity.class);
+          org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+              entity =
+                  FileSystemTimelineReaderImpl.getTimelineRecordFromJSON(
+                      strLine.trim(),
+                      org.apache.hadoop.yarn.api.records.timelineservice.
+                          TimelineEntity.class);
           if (eventId == null) {
             // Job metrics are published without any events for
             // ApplicationEntity. There is also possibility that some other

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java

@@ -621,8 +621,8 @@ public class UtilsForTests {
     return job;
   }
 
-  public static void waitForAppFinished(RunningJob job, MiniMRYarnCluster cluster)
- throws IOException {
+  public static void waitForAppFinished(RunningJob job,
+      MiniMRYarnCluster cluster) throws IOException {
     ApplicationId appId = ApplicationId.newInstance(
         Long.parseLong(job.getID().getJtIdentifier()), job.getID().getId());
     ConcurrentMap<ApplicationId, RMApp> rmApps =

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

@@ -33,7 +33,8 @@ import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollector
  * to the timeline service.
  */
 abstract class EntityWriterV2
-    extends org.apache.hadoop.mapreduce.Mapper<IntWritable,IntWritable,Writable,Writable> {
+    extends org.apache.hadoop.mapreduce.Mapper
+        <IntWritable, IntWritable, Writable, Writable> {
   @Override
   public void map(IntWritable key, IntWritable val, Context context)
       throws IOException {

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

@@ -89,8 +89,8 @@ class JobHistoryFileReplayMapperV2 extends EntityWriterV2 {
             parser.parseHistoryFile(job.getJobHistoryFilePath());
         Configuration jobConf =
             parser.parseConfiguration(job.getJobConfFilePath());
-        LOG.info("parsed the job history file and the configuration file for job"
-            + jobIdStr);
+        LOG.info("parsed the job history file and the configuration file " +
+            "for job " + jobIdStr);
 
         // set the context
         // flow id: job name, flow run id: timestamp, user id

+ 7 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterConstants.java

@@ -32,12 +32,12 @@ interface SimpleEntityWriterConstants {
 
   /**
    *  To ensure that the compression really gets exercised, generate a
-   *  random alphanumeric fixed length payload
+   *  random alphanumeric fixed length payload.
    */
-  char[] ALPHA_NUMS = new char[] { 'a', 'b', 'c', 'd', 'e', 'f',
-    'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o', 'p', 'q', 'r',
-    's', 't', 'u', 'v', 'w', 'x', 'y', 'z', 'A', 'B', 'C', 'D',
-    'E', 'F', 'G', 'H', 'I', 'J', 'K', 'L', 'M', 'N', 'O', 'P',
-    'Q', 'R', 'S', 'T', 'U', 'V', 'W', 'X', 'Y', 'Z', '1', '2',
-    '3', '4', '5', '6', '7', '8', '9', '0', ' ' };
+  char[] ALPHA_NUMS = new char[] {'a', 'b', 'c', 'd', 'e', 'f',
+      'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o', 'p', 'q', 'r',
+      's', 't', 'u', 'v', 'w', 'x', 'y', 'z', 'A', 'B', 'C', 'D',
+      'E', 'F', 'G', 'H', 'I', 'J', 'K', 'L', 'M', 'N', 'O', 'P',
+      'Q', 'R', 'S', 'T', 'U', 'V', 'W', 'X', 'Y', 'Z', '1', '2',
+      '3', '4', '5', '6', '7', '8', '9', '0', ' '};
 }

+ 4 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV1.java

@@ -39,11 +39,13 @@ import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
    * configuration.
    */
 class SimpleEntityWriterV1
-    extends org.apache.hadoop.mapreduce.Mapper<IntWritable,IntWritable,Writable,Writable>
+    extends org.apache.hadoop.mapreduce.Mapper
+        <IntWritable, IntWritable, Writable, Writable>
     implements SimpleEntityWriterConstants {
   private static final Log LOG = LogFactory.getLog(SimpleEntityWriterV1.class);
 
-  public void map(IntWritable key, IntWritable val, Context context) throws IOException {
+  public void map(IntWritable key, IntWritable val, Context context)
+      throws IOException {
     TimelineClient tlc = new TimelineClientImpl();
     Configuration conf = context.getConfiguration();
 

+ 6 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV1.java

@@ -90,9 +90,10 @@ class TimelineEntityConverterV1 {
     return job;
   }
 
-  private Set<TimelineEntity> createTaskAndTaskAttemptEntities(JobInfo jobInfo) {
+  private Set<TimelineEntity>
+      createTaskAndTaskAttemptEntities(JobInfo jobInfo) {
     Set<TimelineEntity> entities = new HashSet<>();
-    Map<TaskID,TaskInfo> taskInfoMap = jobInfo.getAllTasks();
+    Map<TaskID, TaskInfo> taskInfoMap = jobInfo.getAllTasks();
     LOG.info("job " + jobInfo.getJobId()+ " has " + taskInfoMap.size() +
         " tasks");
     for (TaskInfo taskInfo: taskInfoMap.values()) {
@@ -124,7 +125,7 @@ class TimelineEntityConverterV1 {
 
   private Set<TimelineEntity> createTaskAttemptEntities(TaskInfo taskInfo) {
     Set<TimelineEntity> taskAttempts = new HashSet<TimelineEntity>();
-    Map<TaskAttemptID,TaskAttemptInfo> taskAttemptInfoMap =
+    Map<TaskAttemptID, TaskAttemptInfo> taskAttemptInfoMap =
         taskInfo.getAllTaskAttempts();
     LOG.info("task " + taskInfo.getTaskId() + " has " +
         taskAttemptInfoMap.size() + " task attempts");
@@ -135,7 +136,8 @@ class TimelineEntityConverterV1 {
     return taskAttempts;
   }
 
-  private TimelineEntity createTaskAttemptEntity(TaskAttemptInfo taskAttemptInfo) {
+  private TimelineEntity
+      createTaskAttemptEntity(TaskAttemptInfo taskAttemptInfo) {
     TimelineEntity taskAttempt = new TimelineEntity();
     taskAttempt.setEntityType(TASK_ATTEMPT);
     taskAttempt.setEntityId(taskAttemptInfo.getAttemptId().toString());

+ 3 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV2.java

@@ -27,11 +27,6 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.CounterGroup;
-import org.apache.hadoop.mapreduce.Counters;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
@@ -109,7 +104,7 @@ class TimelineEntityConverterV2 {
   }
 
   private void addConfiguration(TimelineEntity job, Configuration conf) {
-    for (Map.Entry<String,String> e: conf) {
+    for (Map.Entry<String, String> e: conf) {
       job.addConfig(e.getKey(), e.getValue());
     }
   }
@@ -130,7 +125,7 @@ class TimelineEntityConverterV2 {
   private List<TimelineEntity> createTaskAndTaskAttemptEntities(
       JobInfo jobInfo) {
     List<TimelineEntity> entities = new ArrayList<>();
-    Map<TaskID,TaskInfo> taskInfoMap = jobInfo.getAllTasks();
+    Map<TaskID, TaskInfo> taskInfoMap = jobInfo.getAllTasks();
     LOG.info("job " + jobInfo.getJobId()+ " has " + taskInfoMap.size() +
         " tasks");
     for (TaskInfo taskInfo: taskInfoMap.values()) {
@@ -167,7 +162,7 @@ class TimelineEntityConverterV2 {
 
   private Set<TimelineEntity> createTaskAttemptEntities(TaskInfo taskInfo) {
     Set<TimelineEntity> taskAttempts = new HashSet<TimelineEntity>();
-    Map<TaskAttemptID,TaskAttemptInfo> taskAttemptInfoMap =
+    Map<TaskAttemptID, TaskAttemptInfo> taskAttemptInfoMap =
         taskInfo.getAllTaskAttempts();
     LOG.info("task " + taskInfo.getTaskId() + " has " +
         taskAttemptInfoMap.size() + " task attempts");

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

@@ -137,7 +137,8 @@ public class TimelineServicePerformance extends Configured implements Tool {
     default:
       // use the current timestamp as the "run id" of the test: this will
       // be used as simulating the cluster timestamp for apps
-      conf.setLong(SimpleEntityWriterConstants.TIMELINE_SERVICE_PERFORMANCE_RUN_ID,
+      conf.setLong(
+          SimpleEntityWriterConstants.TIMELINE_SERVICE_PERFORMANCE_RUN_ID,
           System.currentTimeMillis());
       switch (timeline_service_version) {
       case TIMELINE_SERVICE_VERSION_2:

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

@@ -181,10 +181,11 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
     }
     if (enableTimelineAuxService) {
       conf.setStrings(YarnConfiguration.NM_AUX_SERVICES,
-          new String[] { ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID, TIMELINE_AUX_SERVICE_NAME });
+          new String[] {ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID,
+              TIMELINE_AUX_SERVICE_NAME});
     } else {
       conf.setStrings(YarnConfiguration.NM_AUX_SERVICES,
-          new String[] { ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID });
+          new String[] {ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID});
     }
     conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT,
         ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID), ShuffleHandler.class,

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/package-info.java

@@ -24,4 +24,3 @@
 package org.apache.hadoop.yarn.api.records.timelineservice;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -500,7 +500,8 @@ public class YarnConfiguration extends Configuration {
 
   /**
    *  The setting that controls whether yarn system metrics is published on the
-   *  timeline server or not by RM and NM. This configuration setting is for ATS V2
+   *  timeline server or not by RM and NM. This configuration setting is for
+   *  ATS v2.
    */
   public static final String SYSTEM_METRICS_PUBLISHER_ENABLED = YARN_PREFIX
       + "system-metrics-publisher.enabled";
@@ -840,7 +841,7 @@ public class YarnConfiguration extends Configuration {
   
   /** Number of threads container manager uses.*/
   public static final String NM_COLLECTOR_SERVICE_THREAD_COUNT =
-    NM_PREFIX + "collector-service.thread-count";
+      NM_PREFIX + "collector-service.thread-count";
   public static final int DEFAULT_NM_COLLECTOR_SERVICE_THREAD_COUNT = 5;
 
   /** Number of threads used in cleanup.*/
@@ -872,7 +873,7 @@ public class YarnConfiguration extends Configuration {
   
   /** Address where the collector service IPC is.*/
   public static final String NM_COLLECTOR_SERVICE_ADDRESS =
-    NM_PREFIX + "collector-service.address";
+      NM_PREFIX + "collector-service.address";
   public static final int DEFAULT_NM_COLLECTOR_SERVICE_PORT = 8048;
   public static final String DEFAULT_NM_COLLECTOR_SERVICE_ADDRESS =
       "0.0.0.0:" + DEFAULT_NM_LOCALIZER_PORT;

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/TimelineServiceHelper.java

@@ -36,6 +36,8 @@ public final class TimelineServiceHelper {
   /**
    * Cast map to HashMap for generic type.
    * @param originalMap the map need to be casted
+   * @param <E> key type
+   * @param <V> value type
    * @return casted HashMap object
    */
   public static <E, V> HashMap<E, V> mapCastToHashMap(

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

@@ -102,7 +102,6 @@ import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.log4j.LogManager;
 
@@ -747,7 +746,7 @@ public class ApplicationMaster {
             DSEvent.DS_APP_ATTEMPT_END);
       } else {
         publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
-          DSEvent.DS_APP_ATTEMPT_END, domainId, appSubmitterUgi);
+            DSEvent.DS_APP_ATTEMPT_END, domainId, appSubmitterUgi);
       }
     }
 
@@ -858,7 +857,7 @@ public class ApplicationMaster {
             publishContainerEndEventOnTimelineServiceV2(containerStatus);
           } else {
             publishContainerEndEvent(
-              timelineClient, containerStatus, domainId, appSubmitterUgi);
+                timelineClient, containerStatus, domainId, appSubmitterUgi);
           }
         }
       }
@@ -988,8 +987,8 @@ public class ApplicationMaster {
               container);
         } else {
           applicationMaster.publishContainerStartEvent(
-            applicationMaster.timelineClient, container,
-            applicationMaster.domainId, applicationMaster.appSubmitterUgi);
+              applicationMaster.timelineClient, container,
+              applicationMaster.domainId, applicationMaster.appSubmitterUgi);
         }
       }
     }
@@ -1348,8 +1347,10 @@ public class ApplicationMaster {
 
   private void publishContainerStartEventOnTimelineServiceV2(
       Container container) {
-    final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
-        new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
+    final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+        entity =
+            new org.apache.hadoop.yarn.api.records.timelineservice.
+            TimelineEntity();
     entity.setId(container.getId().toString());
     entity.setType(DSEntity.DS_CONTAINER.toString());
     long ts = System.currentTimeMillis();
@@ -1381,8 +1382,10 @@ public class ApplicationMaster {
 
   private void publishContainerEndEventOnTimelineServiceV2(
       final ContainerStatus container) {
-    final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
-        new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
+    final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+        entity =
+            new org.apache.hadoop.yarn.api.records.timelineservice.
+            TimelineEntity();
     entity.setId(container.getContainerId().toString());
     entity.setType(DSEntity.DS_CONTAINER.toString());
     //entity.setDomainId(domainId);
@@ -1412,8 +1415,10 @@ public class ApplicationMaster {
 
   private void publishApplicationAttemptEventOnTimelineServiceV2(
       DSEvent appEvent) {
-    final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
-        new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
+    final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+        entity =
+            new org.apache.hadoop.yarn.api.records.timelineservice.
+            TimelineEntity();
     entity.setId(appAttemptID.toString());
     entity.setType(DSEntity.DS_APP_ATTEMPT.toString());
     long ts = System.currentTimeMillis();

+ 14 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java

@@ -153,7 +153,8 @@ public class TestDistributedShell {
         ProcfsBasedProcessTree.class.getName());
     conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, true);
     conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.YARN_MINICLUSTER_CONTROL_RESOURCE_MONITORING,
+    conf.setBoolean(
+        YarnConfiguration.YARN_MINICLUSTER_CONTROL_RESOURCE_MONITORING,
         true);
     conf.setBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
           true);
@@ -180,11 +181,13 @@ public class TestDistributedShell {
       // disable v1 timeline server since we no longer have a server here
       // enable aux-service based timeline aggregators
       conf.set(YarnConfiguration.NM_AUX_SERVICES, TIMELINE_AUX_SERVICE_NAME);
-      conf.set(YarnConfiguration.NM_AUX_SERVICES + "." + TIMELINE_AUX_SERVICE_NAME
-        + ".class", PerNodeTimelineCollectorsAuxService.class.getName());
+      conf.set(YarnConfiguration.NM_AUX_SERVICES + "." +
+          TIMELINE_AUX_SERVICE_NAME + ".class",
+          PerNodeTimelineCollectorsAuxService.class.getName());
       conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
           FileSystemTimelineWriterImpl.class,
-          org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter.class);
+          org.apache.hadoop.yarn.server.timelineservice.storage.
+              TimelineWriter.class);
     } else {
       Assert.fail("Wrong timeline version number: " + timelineVersion);
     }
@@ -395,7 +398,8 @@ public class TestDistributedShell {
       }
 
       if (appReport.getYarnApplicationState() == YarnApplicationState.FINISHED
-          && appReport.getFinalApplicationStatus() != FinalApplicationStatus.UNDEFINED) {
+          && appReport.getFinalApplicationStatus() !=
+              FinalApplicationStatus.UNDEFINED) {
         break;
       }
     }
@@ -431,7 +435,7 @@ public class TestDistributedShell {
   }
 
   private void checkTimelineV1(boolean haveDomain) throws Exception {
-        TimelineDomain domain = null;
+    TimelineDomain domain = null;
     if (haveDomain) {
       domain = yarnCluster.getApplicationHistoryServer()
           .getTimelineStore().getDomain("TEST_DOMAIN");
@@ -545,7 +549,7 @@ public class TestDistributedShell {
         if (numOfContainerFinishedOccurences > 0) {
           break;
         } else {
-          Thread.sleep(500l);
+          Thread.sleep(500L);
         }
       }
       Assert.assertEquals(
@@ -577,7 +581,7 @@ public class TestDistributedShell {
         if (numOfStringOccurences > 0) {
           break;
         } else {
-          Thread.sleep(500l);
+          Thread.sleep(500L);
         }
       }
       Assert.assertEquals(
@@ -631,8 +635,9 @@ public class TestDistributedShell {
     try {
       reader = new BufferedReader(new FileReader(entityFile));
       while ((strLine = reader.readLine()) != null) {
-        if (strLine.trim().contains(searchString))
+        if (strLine.trim().contains(searchString)) {
           actualCount++;
+        }
       }
     } finally {
       reader.close();

+ 0 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShellWithNodeLabels.java

@@ -30,9 +30,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TestName;
 
 import com.google.common.collect.ImmutableMap;
 

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java

@@ -463,15 +463,15 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
 
   /**
    * Register TimelineClient to AMRMClient.
-   * @param timelineClient
+   * @param client the timeline client to register
    */
-  public void registerTimelineClient(TimelineClient timelineClient) {
-    this.timelineClient = timelineClient;
+  public void registerTimelineClient(TimelineClient client) {
+    this.timelineClient = client;
   }
 
   /**
    * Get registered timeline client.
-   * @return
+   * @return the registered timeline client
    */
   public TimelineClient getRegisteredTimeineClient() {
     return this.timelineClient;
@@ -481,7 +481,7 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
    * Wait for <code>check</code> to return true for each 1000 ms.
    * See also {@link #waitFor(com.google.common.base.Supplier, int)}
    * and {@link #waitFor(com.google.common.base.Supplier, int, int)}
-   * @param check
+   * @param check the condition for which it should wait
    */
   public void waitFor(Supplier<Boolean> check) throws InterruptedException {
     waitFor(check, 1000);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java

@@ -304,7 +304,7 @@ extends AbstractService {
 
   /**
    * Get registered timeline client.
-   * @return
+   * @return the registered timeline client
    */
   public TimelineClient getRegisteredTimeineClient() {
     return client.getRegisteredTimeineClient();
@@ -325,7 +325,7 @@ extends AbstractService {
    * Wait for <code>check</code> to return true for each 1000 ms.
    * See also {@link #waitFor(com.google.common.base.Supplier, int)}
    * and {@link #waitFor(com.google.common.base.Supplier, int, int)}
-   * @param check
+   * @param check the condition for which it should wait
    */
   public void waitFor(Supplier<Boolean> check) throws InterruptedException {
     waitFor(check, 1000);

+ 19 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java

@@ -30,8 +30,6 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
@@ -55,10 +53,12 @@ public abstract class TimelineClient extends AbstractService implements
    * construct and initialize a timeline client if the following operations are
    * supposed to be conducted by that user.
    */
-  protected ApplicationId contextAppId;
+  private ApplicationId contextAppId;
 
   /**
    * Creates an instance of the timeline v.1.x client.
+   *
+   * @return the created timeline client instance
    */
   @Public
   public static TimelineClient createTimelineClient() {
@@ -68,6 +68,10 @@ public abstract class TimelineClient extends AbstractService implements
 
   /**
    * Creates an instance of the timeline v.2 client.
+   *
+   * @param appId the application id with which the timeline client is
+   * associated
+   * @return the created timeline client instance
    */
   @Public
   public static TimelineClient createTimelineClient(ApplicationId appId) {
@@ -91,8 +95,8 @@ public abstract class TimelineClient extends AbstractService implements
    * @param entities
    *          the collection of {@link TimelineEntity}
    * @return the error information if the sent entities are not correctly stored
-   * @throws IOException
-   * @throws YarnException
+   * @throws IOException if there are I/O errors
+   * @throws YarnException if entities are incomplete/invalid
    */
   @Public
   public abstract TimelinePutResponse putEntities(
@@ -112,8 +116,8 @@ public abstract class TimelineClient extends AbstractService implements
    * @param entities
    *          the collection of {@link TimelineEntity}
    * @return the error information if the sent entities are not correctly stored
-   * @throws IOException
-   * @throws YarnException
+   * @throws IOException if there are I/O errors
+   * @throws YarnException if entities are incomplete/invalid
    */
   @Public
   public abstract TimelinePutResponse putEntities(
@@ -212,15 +216,15 @@ public abstract class TimelineClient extends AbstractService implements
    * for a non-v.2 timeline client instance, a YarnException is thrown.
    * </p>
    *
-   * @param entities
-   *          the collection of {@link org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity}
+   * @param entities the collection of {@link
+   * org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity}
    * @throws IOException
    * @throws YarnException
    */
   @Public
   public abstract void putEntities(
-      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity... entities)
-      throws IOException, YarnException;
+      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
+          entities) throws IOException, YarnException;
 
   /**
    * <p>
@@ -230,15 +234,15 @@ public abstract class TimelineClient extends AbstractService implements
    * non-v.2 timeline client instance, a YarnException is thrown.
    * </p>
    *
-   * @param entities
-   *          the collection of {@link org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity}
+   * @param entities the collection of {@link
+   * org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity}
    * @throws IOException
    * @throws YarnException
    */
   @Public
   public abstract void putEntitiesAsync(
-      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity... entities)
-      throws IOException, YarnException;
+      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
+          entities) throws IOException, YarnException;
 
   /**
    * <p>

+ 26 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java

@@ -391,8 +391,8 @@ public class TimelineClientImpl extends TimelineClient {
 
   @Override
   public void putEntities(
-      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity... entities)
-          throws IOException, YarnException {
+      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
+          entities) throws IOException, YarnException {
     if (!timelineServiceV2) {
       throw new YarnException("v.2 method is invoked on a v.1.x client");
     }
@@ -401,8 +401,8 @@ public class TimelineClientImpl extends TimelineClient {
 
   @Override
   public void putEntitiesAsync(
-      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity... entities)
-      throws IOException, YarnException {
+      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
+          entities) throws IOException, YarnException {
     if (!timelineServiceV2) {
       throw new YarnException("v.2 method is invoked on a v.1.x client");
     }
@@ -494,7 +494,8 @@ public class TimelineClientImpl extends TimelineClient {
       throw new IOException(re);
     }
     if (resp == null ||
-        resp.getClientResponseStatus() != ClientResponse.Status.OK) {
+        resp.getStatusInfo().getStatusCode() !=
+            ClientResponse.Status.OK.getStatusCode()) {
       String msg = "Response from the timeline server is " +
           ((resp == null) ? "null":
           "not successful," + " HTTP error code: " + resp.getStatus()
@@ -530,7 +531,8 @@ public class TimelineClientImpl extends TimelineClient {
             // TODO we should add retry logic here if timelineServiceAddress is
             // not available immediately.
             return (Token) authUrl.getDelegationToken(
-                constructResURI(getConfig(), getTimelineServiceAddress(), false).toURL(),
+                constructResURI(getConfig(),
+                    getTimelineServiceAddress(), false).toURL(),
                 token, renewer, doAsUser);
           }
         };
@@ -911,17 +913,21 @@ public class TimelineClientImpl extends TimelineClient {
   }
 
   private final class EntitiesHolder extends FutureTask<Void> {
-    private final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities entities;
+    private final
+        org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
+            entities;
     private final boolean isSync;
 
     EntitiesHolder(
-        final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities entities,
+        final
+            org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
+                entities,
         final boolean isSync) {
       super(new Callable<Void>() {
         // publishEntities()
         public Void call() throws Exception {
           MultivaluedMap<String, String> params = new MultivaluedMapImpl();
-          params.add("appid", contextAppId.toString());
+          params.add("appid", getContextAppId().toString());
           params.add("async", Boolean.toString(!isSync));
           putObjects("entities", params, entities);
           return null;
@@ -935,7 +941,8 @@ public class TimelineClientImpl extends TimelineClient {
       return isSync;
     }
 
-    public org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities getEntities() {
+    public org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
+        getEntities() {
       return entities;
     }
   }
@@ -947,7 +954,7 @@ public class TimelineClientImpl extends TimelineClient {
   private class TimelineEntityDispatcher {
     /**
      * Time period for which the timelineclient will wait for draining after
-     * stop
+     * stop.
      */
     private static final long DRAIN_TIME_PERIOD = 2000L;
 
@@ -1063,17 +1070,20 @@ public class TimelineClientImpl extends TimelineClient {
     }
 
     public void dispatchEntities(boolean sync,
-        org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity[] entitiesTobePublished)
-            throws YarnException {
+        org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity[]
+            entitiesTobePublished) throws YarnException {
       if (executor.isShutdown()) {
         throw new YarnException("Timeline client is in the process of stopping,"
             + " not accepting any more TimelineEntities");
       }
 
       // wrap all TimelineEntity into TimelineEntities object
-      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities entities =
-          new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities();
-      for (org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity : entitiesTobePublished) {
+      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
+          entities =
+              new org.apache.hadoop.yarn.api.records.timelineservice.
+                  TimelineEntities();
+      for (org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+               entity : entitiesTobePublished) {
         entities.addEntity(entity);
       }
 

+ 10 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineServiceRecords.java

@@ -100,7 +100,8 @@ public class TestTimelineServiceRecords {
     }
     entity.addMetric(metric2);
 
-    TimelineMetric metric3 = new TimelineMetric(TimelineMetric.Type.SINGLE_VALUE);
+    TimelineMetric metric3 =
+        new TimelineMetric(TimelineMetric.Type.SINGLE_VALUE);
     metric3.setId("test metric id 1");
     metric3.addValue(4L, (short) 4);
     Assert.assertEquals("metric3 should equal to metric2! ", metric3, metric2);
@@ -212,18 +213,22 @@ public class TestTimelineServiceRecords {
         ApplicationAttemptId.newInstance(
             ApplicationId.newInstance(0, 1), 1), 1).toString());
 
-    cluster.addChild(TimelineEntityType.YARN_FLOW_RUN.toString(), flow1.getId());
+    cluster.addChild(TimelineEntityType.YARN_FLOW_RUN.toString(),
+        flow1.getId());
     flow1
         .setParent(TimelineEntityType.YARN_CLUSTER.toString(), cluster.getId());
     flow1.addChild(TimelineEntityType.YARN_FLOW_RUN.toString(), flow2.getId());
     flow2.setParent(TimelineEntityType.YARN_FLOW_RUN.toString(), flow1.getId());
-    flow2.addChild(TimelineEntityType.YARN_APPLICATION.toString(), app1.getId());
-    flow2.addChild(TimelineEntityType.YARN_APPLICATION.toString(), app2.getId());
+    flow2.addChild(TimelineEntityType.YARN_APPLICATION.toString(),
+        app1.getId());
+    flow2.addChild(TimelineEntityType.YARN_APPLICATION.toString(),
+        app2.getId());
     app1.setParent(TimelineEntityType.YARN_FLOW_RUN.toString(), flow2.getId());
     app1.addChild(TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(),
         appAttempt.getId());
     appAttempt
-        .setParent(TimelineEntityType.YARN_APPLICATION.toString(), app1.getId());
+        .setParent(TimelineEntityType.YARN_APPLICATION.toString(),
+            app1.getId());
     app2.setParent(TimelineEntityType.YARN_FLOW_RUN.toString(), flow2.getId());
     appAttempt.addChild(TimelineEntityType.YARN_CONTAINER.toString(),
         container.getId());

+ 20 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java

@@ -43,7 +43,7 @@ public class TestTimelineClientV2Impl {
   private static final Log LOG =
       LogFactory.getLog(TestTimelineClientV2Impl.class);
   private TestV2TimelineClient client;
-  private static long TIME_TO_SLEEP = 150;
+  private static final long TIME_TO_SLEEP = 150L;
   private static final String EXCEPTION_MSG = "Exception in the content";
 
   @Before
@@ -62,12 +62,12 @@ public class TestTimelineClientV2Impl {
   public TestName currTestName = new TestName();
   private YarnConfiguration conf;
 
-  private TestV2TimelineClient createTimelineClient(YarnConfiguration conf) {
+  private TestV2TimelineClient createTimelineClient(YarnConfiguration config) {
     ApplicationId id = ApplicationId.newInstance(0, 0);
-    TestV2TimelineClient client = new TestV2TimelineClient(id);
-    client.init(conf);
-    client.start();
-    return client;
+    TestV2TimelineClient tc = new TestV2TimelineClient(id);
+    tc.init(config);
+    tc.start();
+    return tc;
   }
 
   private class TestV2TimelineClientForExceptionHandling
@@ -76,12 +76,16 @@ public class TestTimelineClientV2Impl {
       super(id);
     }
 
-    protected boolean throwYarnException;
+    private boolean throwYarnException;
 
     public void setThrowYarnException(boolean throwYarnException) {
       this.throwYarnException = throwYarnException;
     }
 
+    public boolean isThrowYarnException() {
+      return throwYarnException;
+    }
+
     @Override
     protected void putObjects(URI base, String path,
         MultivaluedMap<String, String> params, Object obj)
@@ -123,7 +127,7 @@ public class TestTimelineClientV2Impl {
     protected void putObjects(String path,
         MultivaluedMap<String, String> params, Object obj)
             throws IOException, YarnException {
-      if (throwYarnException) {
+      if (isThrowYarnException()) {
         throw new YarnException("ActualException");
       }
       publishedEntities.add((TimelineEntities) obj);
@@ -139,17 +143,17 @@ public class TestTimelineClientV2Impl {
 
   @Test
   public void testExceptionMultipleRetry() {
-    TestV2TimelineClientForExceptionHandling client =
+    TestV2TimelineClientForExceptionHandling c =
         new TestV2TimelineClientForExceptionHandling(
             ApplicationId.newInstance(0, 0));
     int maxRetries = 2;
     conf.setInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
         maxRetries);
-    client.init(conf);
-    client.start();
-    client.setTimelineServiceAddress("localhost:12345");
+    c.init(conf);
+    c.start();
+    c.setTimelineServiceAddress("localhost:12345");
     try {
-      client.putEntities(new TimelineEntity());
+      c.putEntities(new TimelineEntity());
     } catch (IOException e) {
       Assert.fail("YARN exception is expected");
     } catch (YarnException e) {
@@ -161,9 +165,9 @@ public class TestTimelineClientV2Impl {
               "TimelineClient has reached to max retry times : " + maxRetries));
     }
 
-    client.setThrowYarnException(true);
+    c.setThrowYarnException(true);
     try {
-      client.putEntities(new TimelineEntity());
+      c.putEntities(new TimelineEntity());
     } catch (IOException e) {
       Assert.fail("YARN exception is expected");
     } catch (YarnException e) {
@@ -173,7 +177,7 @@ public class TestTimelineClientV2Impl {
       Assert.assertTrue("YARN exception is expected",
           cause.getMessage().contains(EXCEPTION_MSG));
     }
-    client.stop();
+    c.stop();
   }
 
   @Test
@@ -348,7 +352,6 @@ public class TestTimelineClientV2Impl {
     for (int i = 0; i < client.getNumOfTimelineEntitiesPublished(); i++) {
       TimelineEntities publishedEntities = client.getPublishedEntities(i);
       StringBuilder entitiesPerPublish = new StringBuilder();
-      ;
       for (TimelineEntity entity : publishedEntities.getEntities()) {
         entitiesPerPublish.append(entity.getId());
         entitiesPerPublish.append(",");

+ 14 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java

@@ -37,18 +37,21 @@ public class TestTimelineServiceHelper {
 
     // Test empty hashmap be casted to a empty hashmap
     Map<String, String> emptyHashMap = new HashMap<String, String>();
-    Assert.assertEquals(TimelineServiceHelper.mapCastToHashMap(emptyHashMap).size(), 0);
+    Assert.assertEquals(
+        TimelineServiceHelper.mapCastToHashMap(emptyHashMap).size(), 0);
 
     // Test empty non-hashmap be casted to a empty hashmap
     Map<String, String> emptyTreeMap = new TreeMap<String, String>();
-    Assert.assertEquals(TimelineServiceHelper.mapCastToHashMap(emptyTreeMap).size(), 0);
+    Assert.assertEquals(
+        TimelineServiceHelper.mapCastToHashMap(emptyTreeMap).size(), 0);
 
     // Test non-empty hashmap be casted to hashmap correctly
     Map<String, String> firstHashMap = new HashMap<String, String>();
     String key = "KEY";
     String value = "VALUE";
     firstHashMap.put(key, value);
-    Assert.assertEquals(TimelineServiceHelper.mapCastToHashMap(firstHashMap), firstHashMap);
+    Assert.assertEquals(
+        TimelineServiceHelper.mapCastToHashMap(firstHashMap), firstHashMap);
 
     // Test non-empty non-hashmap is casted correctly.
     Map<String, String> firstTreeMap = new TreeMap<String, String>();
@@ -59,17 +62,21 @@ public class TestTimelineServiceHelper {
     Assert.assertEquals(alternateHashMap.get(key), value);
 
     // Test complicated hashmap be casted correctly
-    Map<String, Set<String>> complicatedHashMap = new HashMap<String, Set<String>>();
+    Map<String, Set<String>> complicatedHashMap =
+        new HashMap<String, Set<String>>();
     Set<String> hashSet = new HashSet<String>();
     hashSet.add(value);
     complicatedHashMap.put(key, hashSet);
-    Assert.assertEquals(TimelineServiceHelper.mapCastToHashMap(complicatedHashMap),
+    Assert.assertEquals(
+        TimelineServiceHelper.mapCastToHashMap(complicatedHashMap),
         complicatedHashMap);
 
     // Test complicated non-hashmap get casted correctly
-    Map<String, Set<String>> complicatedTreeMap = new TreeMap<String, Set<String>>();
+    Map<String, Set<String>> complicatedTreeMap =
+        new TreeMap<String, Set<String>>();
     complicatedTreeMap.put(key, hashSet);
-    Assert.assertEquals(TimelineServiceHelper.mapCastToHashMap(complicatedTreeMap).get(key),
+    Assert.assertEquals(
+        TimelineServiceHelper.mapCastToHashMap(complicatedTreeMap).get(key),
         hashSet);
   }
 

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

@@ -48,9 +48,9 @@ public interface CollectorNodemanagerProtocol {
    *
    * @param request the request of registering a new collector or a list of
    *                collectors
-   * @return
-   * @throws YarnException
-   * @throws IOException
+   * @return the response for registering the new collector
+   * @throws YarnException if the request is invalid
+   * @throws IOException if there are I/O errors
    */
   ReportNewCollectorInfoResponse reportNewCollectorInfo(
       ReportNewCollectorInfoRequest request)
@@ -63,9 +63,9 @@ public interface CollectorNodemanagerProtocol {
    * </p>
    * @param request the request of getting the aggregator context information of
    *                the given application
-   * @return
-   * @throws YarnException
-   * @throws IOException
+   * @return the response for registering the new collector
+   * @throws YarnException if the request is invalid
+   * @throws IOException if there are I/O errors
    */
   GetTimelineCollectorContextResponse getTimelineCollectorContext(
       GetTimelineCollectorContextRequest request)

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

@@ -25,7 +25,8 @@ import org.apache.hadoop.yarn.proto.CollectorNodemanagerProtocol.CollectorNodema
 @Private
 @Unstable
 @ProtocolInfo(
-    protocolName = "org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocolPB",
+    protocolName =
+        "org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocolPB",
     protocolVersion = 1)
 public interface CollectorNodemanagerProtocolPB extends
     CollectorNodemanagerProtocolService.BlockingInterface {

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

@@ -53,7 +53,7 @@ public class CollectorNodemanagerProtocolPBClientImpl implements
       + "rpc.nm-command-timeout";
 
   /**
-   * Maximum of 1 minute timeout for a Node to react to the command
+   * Maximum of 1 minute timeout for a Node to react to the command.
    */
   static final int DEFAULT_COMMAND_TIMEOUT = 60000;
 
@@ -63,7 +63,7 @@ public class CollectorNodemanagerProtocolPBClientImpl implements
   public CollectorNodemanagerProtocolPBClientImpl(long clientVersion,
       InetSocketAddress addr, Configuration conf) throws IOException {
     RPC.setProtocolEngine(conf, CollectorNodemanagerProtocolPB.class,
-      ProtobufRpcEngine.class);
+        ProtobufRpcEngine.class);
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
 
     int expireIntvl = conf.getInt(NM_COMMAND_TIMEOUT, DEFAULT_COMMAND_TIMEOUT);

+ 4 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/CollectorNodemanagerProtocolPBServiceImpl.java

@@ -41,7 +41,8 @@ public class CollectorNodemanagerProtocolPBServiceImpl implements
 
   private CollectorNodemanagerProtocol real;
 
-  public CollectorNodemanagerProtocolPBServiceImpl(CollectorNodemanagerProtocol impl) {
+  public CollectorNodemanagerProtocolPBServiceImpl(
+      CollectorNodemanagerProtocol impl) {
     this.real = impl;
   }
 
@@ -52,7 +53,8 @@ public class CollectorNodemanagerProtocolPBServiceImpl implements
     ReportNewCollectorInfoRequestPBImpl request =
         new ReportNewCollectorInfoRequestPBImpl(proto);
     try {
-      ReportNewCollectorInfoResponse response = real.reportNewCollectorInfo(request);
+      ReportNewCollectorInfoResponse response =
+          real.reportNewCollectorInfo(request);
       return ((ReportNewCollectorInfoResponsePBImpl)response).getProto();
     } catch (YarnException e) {
       throw new ServiceException(e);

+ 15 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetTimelineCollectorContextRequestPBImpl.java

@@ -29,10 +29,10 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorCon
 public class GetTimelineCollectorContextRequestPBImpl extends
     GetTimelineCollectorContextRequest {
 
-  GetTimelineCollectorContextRequestProto
+  private GetTimelineCollectorContextRequestProto
       proto = GetTimelineCollectorContextRequestProto.getDefaultInstance();
-  GetTimelineCollectorContextRequestProto.Builder builder = null;
-  boolean viaProto = false;
+  private GetTimelineCollectorContextRequestProto.Builder builder = null;
+  private boolean viaProto = false;
 
   private ApplicationId appId = null;
 
@@ -60,8 +60,9 @@ public class GetTimelineCollectorContextRequestPBImpl extends
 
   @Override
   public boolean equals(Object other) {
-    if (other == null)
+    if (other == null) {
       return false;
+    }
     if (other.getClass().isAssignableFrom(this.getClass())) {
       return this.getProto().equals(this.getClass().cast(other).getProto());
     }
@@ -80,8 +81,9 @@ public class GetTimelineCollectorContextRequestPBImpl extends
   }
 
   private void mergeLocalToProto() {
-    if (viaProto)
+    if (viaProto) {
       maybeInitBuilder();
+    }
     mergeLocalToBuilder();
     proto = builder.build();
     viaProto = true;
@@ -100,7 +102,8 @@ public class GetTimelineCollectorContextRequestPBImpl extends
       return this.appId;
     }
 
-    GetTimelineCollectorContextRequestProtoOrBuilder p = viaProto ? proto : builder;
+    GetTimelineCollectorContextRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
     if (!p.hasAppId()) {
       return null;
     }
@@ -110,14 +113,16 @@ public class GetTimelineCollectorContextRequestPBImpl extends
   }
 
   @Override
-  public void setApplicationId(ApplicationId appId) {
+  public void setApplicationId(ApplicationId id) {
     maybeInitBuilder();
-    if (appId == null)
+    if (id == null) {
       builder.clearAppId();
-    this.appId = appId;
+    }
+    this.appId = id;
   }
 
-  private ApplicationIdPBImpl convertFromProtoFormat(YarnProtos.ApplicationIdProto p) {
+  private ApplicationIdPBImpl convertFromProtoFormat(
+      YarnProtos.ApplicationIdProto p) {
     return new ApplicationIdPBImpl(p);
   }
 

+ 15 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetTimelineCollectorContextResponsePBImpl.java

@@ -26,10 +26,10 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorCon
 public class GetTimelineCollectorContextResponsePBImpl extends
     GetTimelineCollectorContextResponse {
 
-  GetTimelineCollectorContextResponseProto proto =
+  private GetTimelineCollectorContextResponseProto proto =
       GetTimelineCollectorContextResponseProto.getDefaultInstance();
-  GetTimelineCollectorContextResponseProto.Builder builder = null;
-  boolean viaProto = false;
+  private GetTimelineCollectorContextResponseProto.Builder builder = null;
+  private boolean viaProto = false;
 
   public GetTimelineCollectorContextResponsePBImpl() {
     builder = GetTimelineCollectorContextResponseProto.newBuilder();
@@ -55,8 +55,9 @@ public class GetTimelineCollectorContextResponsePBImpl extends
 
   @Override
   public boolean equals(Object other) {
-    if (other == null)
+    if (other == null) {
       return false;
+    }
     if (other.getClass().isAssignableFrom(this.getClass())) {
       return this.getProto().equals(this.getClass().cast(other).getProto());
     }
@@ -69,8 +70,9 @@ public class GetTimelineCollectorContextResponsePBImpl extends
   }
 
   private void mergeLocalToProto() {
-    if (viaProto)
+    if (viaProto) {
       maybeInitBuilder();
+    }
     proto = builder.build();
     viaProto = true;
   }
@@ -84,7 +86,8 @@ public class GetTimelineCollectorContextResponsePBImpl extends
 
   @Override
   public String getUserId() {
-    GetTimelineCollectorContextResponseProtoOrBuilder p = viaProto ? proto : builder;
+    GetTimelineCollectorContextResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
     if (!p.hasUserId()) {
       return null;
     }
@@ -103,7 +106,8 @@ public class GetTimelineCollectorContextResponsePBImpl extends
 
   @Override
   public String getFlowName() {
-    GetTimelineCollectorContextResponseProtoOrBuilder p = viaProto ? proto : builder;
+    GetTimelineCollectorContextResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
     if (!p.hasFlowName()) {
       return null;
     }
@@ -122,7 +126,8 @@ public class GetTimelineCollectorContextResponsePBImpl extends
 
   @Override
   public String getFlowVersion() {
-    GetTimelineCollectorContextResponseProtoOrBuilder p = viaProto ? proto : builder;
+    GetTimelineCollectorContextResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
     if (!p.hasFlowVersion()) {
       return null;
     }
@@ -141,7 +146,8 @@ public class GetTimelineCollectorContextResponsePBImpl extends
 
   @Override
   public long getFlowRunId() {
-    GetTimelineCollectorContextResponseProtoOrBuilder p = viaProto ? proto : builder;
+    GetTimelineCollectorContextResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
     return p.getFlowRunId();
   }
 

+ 5 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java

@@ -58,7 +58,7 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
   private Set<NodeLabel> labels = null;
   private List<LogAggregationReport> logAggregationReportsForApps = null;
 
-  Map<ApplicationId, String> registeredCollectors = null;
+  private Map<ApplicationId, String> registeredCollectors = null;
 
   public NodeHeartbeatRequestPBImpl() {
     builder = NodeHeartbeatRequestProto.newBuilder();
@@ -161,10 +161,11 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
   private void addRegisteredCollectorsToProto() {
     maybeInitBuilder();
     builder.clearRegisteredCollectors();
-    for (Map.Entry<ApplicationId, String> entry : registeredCollectors.entrySet()) {
+    for (Map.Entry<ApplicationId, String> entry :
+        registeredCollectors.entrySet()) {
       builder.addRegisteredCollectors(AppCollectorsMapProto.newBuilder()
-        .setAppId(convertToProtoFormat(entry.getKey()))
-        .setAppCollectorAddr(entry.getValue()));
+          .setAppId(convertToProtoFormat(entry.getKey()))
+          .setAppCollectorAddr(entry.getValue()));
     }
   }
 

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

@@ -69,7 +69,7 @@ public class NodeHeartbeatResponsePBImpl extends
   private List<ApplicationId> applicationsToCleanup = null;
   private Map<ApplicationId, ByteBuffer> systemCredentials = null;
   private Resource resource = null;
-  Map<ApplicationId, String> appCollectorsMap = null;
+  private Map<ApplicationId, String> appCollectorsMap = null;
 
   private MasterKey containerTokenMasterKey = null;
   private MasterKey nmTokenMasterKey = null;
@@ -148,8 +148,8 @@ public class NodeHeartbeatResponsePBImpl extends
     builder.clearAppCollectorsMap();
     for (Map.Entry<ApplicationId, String> entry : appCollectorsMap.entrySet()) {
       builder.addAppCollectorsMap(AppCollectorsMapProto.newBuilder()
-        .setAppId(convertToProtoFormat(entry.getKey()))
-        .setAppCollectorAddr(entry.getValue()));
+          .setAppId(convertToProtoFormat(entry.getKey()))
+          .setAppCollectorAddr(entry.getValue()));
     }
   }
 

+ 9 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewCollectorInfoRequestPBImpl.java

@@ -30,11 +30,11 @@ import org.apache.hadoop.yarn.server.api.records.impl.pb.AppCollectorsMapPBImpl;
 public class ReportNewCollectorInfoRequestPBImpl extends
     ReportNewCollectorInfoRequest {
 
-  ReportNewCollectorInfoRequestProto proto =
+  private ReportNewCollectorInfoRequestProto proto =
       ReportNewCollectorInfoRequestProto.getDefaultInstance();
 
-  ReportNewCollectorInfoRequestProto.Builder builder = null;
-  boolean viaProto = false;
+  private ReportNewCollectorInfoRequestProto.Builder builder = null;
+  private boolean viaProto = false;
 
   private List<AppCollectorsMap> collectorsList = null;
 
@@ -62,8 +62,9 @@ public class ReportNewCollectorInfoRequestPBImpl extends
 
   @Override
   public boolean equals(Object other) {
-    if (other == null)
+    if (other == null) {
       return false;
+    }
     if (other.getClass().isAssignableFrom(this.getClass())) {
       return this.getProto().equals(this.getClass().cast(other).getProto());
     }
@@ -71,8 +72,9 @@ public class ReportNewCollectorInfoRequestPBImpl extends
   }
 
   private void mergeLocalToProto() {
-    if (viaProto)
+    if (viaProto) {
       maybeInitBuilder();
+    }
     mergeLocalToBuilder();
     proto = builder.build();
     viaProto = true;
@@ -104,10 +106,10 @@ public class ReportNewCollectorInfoRequestPBImpl extends
 
   private void initLocalCollectorsList() {
     ReportNewCollectorInfoRequestProtoOrBuilder p = viaProto ? proto : builder;
-    List<AppCollectorsMapProto> collectorsList =
+    List<AppCollectorsMapProto> list =
         p.getAppCollectorsList();
     this.collectorsList = new ArrayList<AppCollectorsMap>();
-    for (AppCollectorsMapProto m : collectorsList) {
+    for (AppCollectorsMapProto m : list) {
       this.collectorsList.add(convertFromProtoFormat(m));
     }
   }

+ 7 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewCollectorInfoResponsePBImpl.java

@@ -29,18 +29,19 @@ import com.google.protobuf.TextFormat;
 public class ReportNewCollectorInfoResponsePBImpl extends
     ReportNewCollectorInfoResponse {
 
-  ReportNewCollectorInfoResponseProto proto =
+  private ReportNewCollectorInfoResponseProto proto =
       ReportNewCollectorInfoResponseProto.getDefaultInstance();
 
-  ReportNewCollectorInfoResponseProto.Builder builder = null;
+  private ReportNewCollectorInfoResponseProto.Builder builder = null;
 
-  boolean viaProto = false;
+  private boolean viaProto = false;
 
   public ReportNewCollectorInfoResponsePBImpl() {
     builder = ReportNewCollectorInfoResponseProto.newBuilder();
   }
 
-  public ReportNewCollectorInfoResponsePBImpl(ReportNewCollectorInfoResponseProto proto) {
+  public ReportNewCollectorInfoResponsePBImpl(
+      ReportNewCollectorInfoResponseProto proto) {
     this.proto = proto;
     viaProto = true;
   }
@@ -58,8 +59,9 @@ public class ReportNewCollectorInfoResponsePBImpl extends
 
   @Override
   public boolean equals(Object other) {
-    if (other == null)
+    if (other == null) {
       return false;
+    }
     if (other.getClass().isAssignableFrom(this.getClass())) {
       return this.getProto().equals(this.getClass().cast(other).getProto());
     }

+ 9 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/AppCollectorsMapPBImpl.java

@@ -33,11 +33,11 @@ import com.google.protobuf.TextFormat;
 @Unstable
 public class AppCollectorsMapPBImpl extends AppCollectorsMap {
 
-  AppCollectorsMapProto proto =
+  private AppCollectorsMapProto proto =
       AppCollectorsMapProto.getDefaultInstance();
 
-  AppCollectorsMapProto.Builder builder = null;
-  boolean viaProto = false;
+  private AppCollectorsMapProto.Builder builder = null;
+  private boolean viaProto = false;
 
   private ApplicationId appId = null;
   private String collectorAddr = null;
@@ -52,7 +52,7 @@ public class AppCollectorsMapPBImpl extends AppCollectorsMap {
   }
 
   public AppCollectorsMapProto getProto() {
-      mergeLocalToProto();
+    mergeLocalToProto();
     proto = viaProto ? proto : builder.build();
     viaProto = true;
     return proto;
@@ -65,8 +65,9 @@ public class AppCollectorsMapPBImpl extends AppCollectorsMap {
 
   @Override
   public boolean equals(Object other) {
-    if (other == null)
+    if (other == null) {
       return false;
+    }
     if (other.getClass().isAssignableFrom(this.getClass())) {
       return this.getProto().equals(this.getClass().cast(other).getProto());
     }
@@ -98,12 +99,12 @@ public class AppCollectorsMapPBImpl extends AppCollectorsMap {
   }
 
   @Override
-  public void setApplicationId(ApplicationId appId) {
+  public void setApplicationId(ApplicationId id) {
     maybeInitBuilder();
-    if (appId == null) {
+    if (id == null) {
       builder.clearAppId();
     }
-    this.appId = appId;
+    this.appId = id;
   }
 
   @Override

+ 34 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java

@@ -77,7 +77,8 @@ public class TestRPC {
 
   private static final String EXCEPTION_MSG = "test error";
   private static final String EXCEPTION_CAUSE = "exception cause";
-  private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
+  private static final RecordFactory RECORD_FACTORY =
+      RecordFactoryProvider.getRecordFactory(null);
 
   public static final String ILLEGAL_NUMBER_MESSAGE =
       "collectors' number in ReportNewCollectorInfoRequest is not ONE.";
@@ -101,7 +102,8 @@ public class TestRPC {
 
     // Any unrelated protocol would do
     ApplicationClientProtocol proxy = (ApplicationClientProtocol) rpc.getProxy(
-        ApplicationClientProtocol.class, NetUtils.getConnectAddress(server), conf);
+        ApplicationClientProtocol.class, NetUtils.getConnectAddress(server),
+        conf);
 
     try {
       proxy.getNewApplication(Records
@@ -111,7 +113,8 @@ public class TestRPC {
       Assert.assertTrue(e.getMessage().matches(
           "Unknown method getNewApplication called on.*"
               + "org.apache.hadoop.yarn.proto.ApplicationClientProtocol"
-              + "\\$ApplicationClientProtocolService\\$BlockingInterface protocol."));
+              + "\\$ApplicationClientProtocolService\\$BlockingInterface "
+              + "protocol."));
     } catch (Exception e) {
       e.printStackTrace();
     } finally {
@@ -132,8 +135,10 @@ public class TestRPC {
     server.start();
 
     // Test unrelated protocol wouldn't get response
-    ApplicationClientProtocol unknownProxy = (ApplicationClientProtocol) rpc.getProxy(
-        ApplicationClientProtocol.class, NetUtils.getConnectAddress(server), conf);
+    ApplicationClientProtocol unknownProxy =
+        (ApplicationClientProtocol) rpc.getProxy(
+        ApplicationClientProtocol.class, NetUtils.getConnectAddress(server),
+        conf);
 
     try {
       unknownProxy.getNewApplication(Records
@@ -143,14 +148,17 @@ public class TestRPC {
       Assert.assertTrue(e.getMessage().matches(
           "Unknown method getNewApplication called on.*"
               + "org.apache.hadoop.yarn.proto.ApplicationClientProtocol"
-              + "\\$ApplicationClientProtocolService\\$BlockingInterface protocol."));
+              + "\\$ApplicationClientProtocolService\\$BlockingInterface "
+              + "protocol."));
     } catch (Exception e) {
       e.printStackTrace();
     }
 
     // Test CollectorNodemanagerProtocol get proper response
-    CollectorNodemanagerProtocol proxy = (CollectorNodemanagerProtocol)rpc.getProxy(
-        CollectorNodemanagerProtocol.class, NetUtils.getConnectAddress(server), conf);
+    CollectorNodemanagerProtocol proxy =
+        (CollectorNodemanagerProtocol)rpc.getProxy(
+        CollectorNodemanagerProtocol.class, NetUtils.getConnectAddress(server),
+        conf);
     // Verify request with DEFAULT_APP_ID and DEFAULT_COLLECTOR_ADDR get
     // normally response.
     try {
@@ -196,7 +204,8 @@ public class TestRPC {
       Assert.fail("RPC call failured is expected here.");
     } catch (YarnException | IOException e) {
       Assert.assertTrue(e instanceof  YarnException);
-      Assert.assertTrue(e.getMessage().contains("The application is not found."));
+      Assert.assertTrue(e.getMessage().contains(
+          "The application is not found."));
     }
     server.stop();
   }
@@ -215,12 +224,13 @@ public class TestRPC {
     Server server = rpc.getServer(ContainerManagementProtocol.class,
             new DummyContainerManager(), addr, conf, null, 1);
     server.start();
-    RPC.setProtocolEngine(conf, ContainerManagementProtocolPB.class, ProtobufRpcEngine.class);
+    RPC.setProtocolEngine(conf, ContainerManagementProtocolPB.class,
+        ProtobufRpcEngine.class);
     ContainerManagementProtocol proxy = (ContainerManagementProtocol)
         rpc.getProxy(ContainerManagementProtocol.class,
             NetUtils.getConnectAddress(server), conf);
     ContainerLaunchContext containerLaunchContext =
-        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+        RECORD_FACTORY.newRecordInstance(ContainerLaunchContext.class);
 
     ApplicationId applicationId = ApplicationId.newInstance(0, 0);
     ApplicationAttemptId applicationAttemptId =
@@ -257,10 +267,10 @@ public class TestRPC {
     boolean exception = false;
     try {
       StopContainersRequest stopRequest =
-          recordFactory.newRecordInstance(StopContainersRequest.class);
+          RECORD_FACTORY.newRecordInstance(StopContainersRequest.class);
       stopRequest.setContainerIds(containerIds);
       proxy.stopContainers(stopRequest);
-      } catch (YarnException e) {
+    } catch (YarnException e) {
       exception = true;
       Assert.assertTrue(e.getMessage().contains(EXCEPTION_MSG));
       Assert.assertTrue(e.getMessage().contains(EXCEPTION_CAUSE));
@@ -284,7 +294,7 @@ public class TestRPC {
         GetContainerStatusesRequest request)
     throws YarnException {
       GetContainerStatusesResponse response =
-          recordFactory.newRecordInstance(GetContainerStatusesResponse.class);
+          RECORD_FACTORY.newRecordInstance(GetContainerStatusesResponse.class);
       response.setContainerStatuses(statuses);
       return response;
     }
@@ -293,8 +303,9 @@ public class TestRPC {
     public StartContainersResponse startContainers(
         StartContainersRequest requests) throws YarnException {
       StartContainersResponse response =
-          recordFactory.newRecordInstance(StartContainersResponse.class);
-      for (StartContainerRequest request : requests.getStartContainerRequests()) {
+          RECORD_FACTORY.newRecordInstance(StartContainersResponse.class);
+      for (StartContainerRequest request :
+          requests.getStartContainerRequests()) {
         Token containerToken = request.getContainerToken();
         ContainerTokenIdentifier tokenId = null;
 
@@ -304,7 +315,7 @@ public class TestRPC {
           throw RPCUtil.getRemoteException(e);
         }
         ContainerStatus status =
-            recordFactory.newRecordInstance(ContainerStatus.class);
+            RECORD_FACTORY.newRecordInstance(ContainerStatus.class);
         status.setState(ContainerState.RUNNING);
         status.setContainerId(tokenId.getContainerID());
         status.setExitStatus(0);
@@ -324,7 +335,8 @@ public class TestRPC {
 
     @Override
     public IncreaseContainersResourceResponse increaseContainersResource(
-        IncreaseContainersResourceRequest request) throws YarnException, IOException {
+        IncreaseContainersResourceRequest request)
+            throws YarnException, IOException {
       return null;
     }
 
@@ -383,7 +395,8 @@ public class TestRPC {
       }
 
       ReportNewCollectorInfoResponse response =
-          recordFactory.newRecordInstance(ReportNewCollectorInfoResponse.class);
+          RECORD_FACTORY.newRecordInstance(
+              ReportNewCollectorInfoResponse.class);
       return response;
     }
 
@@ -392,8 +405,8 @@ public class TestRPC {
         GetTimelineCollectorContextRequest request)
         throws  YarnException, IOException {
       if (request.getApplicationId().getId() == 1) {
-         return GetTimelineCollectorContextResponse.newInstance(
-                "test_user_id", "test_flow_name", "test_flow_version", 12345678L);
+        return GetTimelineCollectorContextResponse.newInstance(
+            "test_user_id", "test_flow_name", "test_flow_version", 12345678L);
       } else {
         throw new YarnException("The application is not found.");
       }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java

@@ -187,8 +187,8 @@ public class NodeManager extends CompositeService
     }
   }
 
-  protected NMCollectorService createNMCollectorService(Context context) {
-    return new NMCollectorService(context);
+  protected NMCollectorService createNMCollectorService(Context ctxt) {
+    return new NMCollectorService(ctxt);
   }
 
   protected WebServer createWebServer(Context nmContext,

+ 2 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java

@@ -816,7 +816,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
                     NodeStatusUpdaterImpl.this.context
                         .getNMTokenSecretManager().getCurrentKey(),
                     nodeLabelsForHeartbeat,
-                    NodeStatusUpdaterImpl.this.context.getRegisteredCollectors());
+                    NodeStatusUpdaterImpl.this.context
+                        .getRegisteredCollectors());
 
             if (logAggregationEnabled) {
               // pull log aggregation status for application running in this NM
@@ -939,23 +940,6 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         }
       }
 
-      /**
-       * Caller should take care of sending non null nodelabels for both
-       * arguments
-       *
-       * @param nodeLabelsNew
-       * @param nodeLabelsOld
-       * @return if the New node labels are diff from the older one.
-       */
-      private boolean areNodeLabelsUpdated(Set<NodeLabel> nodeLabelsNew,
-          Set<NodeLabel> nodeLabelsOld) {
-        if (nodeLabelsNew.size() != nodeLabelsOld.size()
-            || !nodeLabelsOld.containsAll(nodeLabelsNew)) {
-          return true;
-        }
-        return false;
-      }
-
       private void updateTimelineClientsAddress(
           NodeHeartbeatResponse response) {
         Map<ApplicationId, String> knownCollectorsMap =

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

@@ -230,8 +230,7 @@ public class ContainerManagerImpl extends CompositeService implements
       nmMetricsPublisher = createNMTimelinePublisher(context);
       context.setNMTimelinePublisher(nmMetricsPublisher);
     }
-    this.containersMonitor =
-        new ContainersMonitorImpl(exec, dispatcher, this.context);
+    this.containersMonitor = createContainersMonitor(exec);
     addService(this.containersMonitor);
 
     dispatcher.register(ContainerEventType.class,
@@ -447,8 +446,9 @@ public class ContainerManagerImpl extends CompositeService implements
   }
 
   @VisibleForTesting
-  protected NMTimelinePublisher createNMTimelinePublisher(Context context) {
-    NMTimelinePublisher nmTimelinePublisherLocal = new NMTimelinePublisher(context);
+  protected NMTimelinePublisher createNMTimelinePublisher(Context ctxt) {
+    NMTimelinePublisher nmTimelinePublisherLocal =
+        new NMTimelinePublisher(ctxt);
     addIfService(nmTimelinePublisherLocal);
     return nmTimelinePublisherLocal;
   }

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationContainerFinishedEvent.java

@@ -25,7 +25,8 @@ public class ApplicationContainerFinishedEvent extends ApplicationEvent {
   private ContainerStatus containerStatus;
 
   public ApplicationContainerFinishedEvent(ContainerStatus containerStatus) {
-    super(containerStatus.getContainerId().getApplicationAttemptId().getApplicationId(),
+    super(containerStatus.getContainerId().getApplicationAttemptId().
+        getApplicationId(),
         ApplicationEventType.APPLICATION_CONTAINER_FINISHED);
     this.containerStatus = containerStatus;
   }

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

@@ -74,7 +74,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.eve
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainerStartMonitoringEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainerStopMonitoringEvent;
-import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerState;
@@ -87,7 +86,6 @@ import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
 import org.apache.hadoop.yarn.util.Clock;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java

@@ -435,8 +435,9 @@ public class ContainersMonitorImpl extends AbstractService implements
                     + " for the first time");
 
                 ResourceCalculatorProcessTree pt =
-                    ResourceCalculatorProcessTree.getResourceCalculatorProcessTree(
-                        pId, processTreeClass, conf);
+                    ResourceCalculatorProcessTree.
+                        getResourceCalculatorProcessTree(
+                            pId, processTreeClass, conf);
                 ptInfo.setPid(pId);
                 ptInfo.setProcessTree(pt);
 

+ 9 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java

@@ -33,12 +33,12 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.timelineservice.ContainerEntity;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity.Identifier;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -56,6 +56,8 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.Contai
 import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Metrics publisher service that publishes data to the timeline service v.2. It
  * is used only if the timeline service v.2 is enabled and the system publishing
@@ -73,7 +75,7 @@ public class NMTimelinePublisher extends CompositeService {
 
   private String httpAddress;
 
-  protected final Map<ApplicationId, TimelineClient> appToClientMap;
+  private final Map<ApplicationId, TimelineClient> appToClientMap;
 
   public NMTimelinePublisher(Context context) {
     super(NMTimelinePublisher.class.getName());
@@ -99,6 +101,11 @@ public class NMTimelinePublisher extends CompositeService {
     this.httpAddress = nodeId.getHost() + ":" + context.getHttpPort();
   }
 
+  @VisibleForTesting
+  Map<ApplicationId, TimelineClient> getAppToClientMap() {
+    return appToClientMap;
+  }
+
   protected void handleNMTimelineEvent(NMTimelineEvent event) {
     switch (event.getType()) {
     case TIMELINE_ENTITY_PUBLISH:

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

@@ -85,7 +85,6 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProto;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
-import org.apache.hadoop.yarn.server.api.ContainerContext;
 import org.apache.hadoop.yarn.server.api.ResourceTracker;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
@@ -1707,9 +1706,9 @@ public class TestNodeStatusUpdater {
           NMContainerTokenSecretManager containerTokenSecretManager,
           NMTokenSecretManagerInNM nmTokenSecretManager,
           NMStateStoreService store, boolean isDistributedSchedulingEnabled,
-          Configuration conf) {
+          Configuration config) {
         return new MyNMContext(containerTokenSecretManager,
-          nmTokenSecretManager, conf);
+          nmTokenSecretManager, config);
       }
     };
 

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java

@@ -645,8 +645,9 @@ public class TestApplication {
     when(c.getLaunchContext()).thenReturn(launchContext);
     when(launchContext.getApplicationACLs()).thenReturn(
         new HashMap<ApplicationAccessType, String>());
-    when(c.cloneAndGetContainerStatus()).thenReturn(BuilderUtils.newContainerStatus(cId,
-        ContainerState.NEW, "", 0, Resource.newInstance(1024, 1)));
+    when(c.cloneAndGetContainerStatus()).thenReturn(
+        BuilderUtils.newContainerStatus(cId,
+            ContainerState.NEW, "", 0, Resource.newInstance(1024, 1)));
     return c;
   }
 }

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

@@ -99,7 +99,6 @@ import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.LinuxResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 import org.hamcrest.CoreMatchers;
@@ -111,16 +110,17 @@ import org.junit.Test;
 public class TestContainerLaunch extends BaseContainerManagerTest {
 
   private static final String INVALID_JAVA_HOME = "/no/jvm/here";
-  protected Context distContext = new NMContext(new NMContainerTokenSecretManager(
-    conf), new NMTokenSecretManagerInNM(), null,
-    new ApplicationACLsManager(conf), new NMNullStateStoreService(), false,
-      conf) {
-    public int getHttpPort() {
-      return HTTP_PORT;
-    };
-    public NodeId getNodeId() {
-      return NodeId.newInstance("ahost", 1234);
-    };
+  private Context distContext =
+      new NMContext(new NMContainerTokenSecretManager(conf),
+          new NMTokenSecretManagerInNM(), null,
+          new ApplicationACLsManager(conf), new NMNullStateStoreService(),
+          false, conf) {
+        public int getHttpPort() {
+          return HTTP_PORT;
+        };
+        public NodeId getNodeId() {
+          return NodeId.newInstance("ahost", 1234);
+        };
   };
 
   public TestContainerLaunch() throws UnsupportedFileSystemException {

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

@@ -55,8 +55,8 @@ public class TestNMTimelinePublisher {
     when(context.getHttpPort()).thenReturn(0);
     NMTimelinePublisher publisher = new NMTimelinePublisher(context) {
       public void createTimelineClient(ApplicationId appId) {
-        if (!appToClientMap.containsKey(appId)) {
-          appToClientMap.put(appId, timelineClient);
+        if (!getAppToClientMap().containsKey(appId)) {
+          getAppToClientMap().put(appId, timelineClient);
         }
       }
     };

+ 3 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockApp.java

@@ -24,7 +24,6 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
@@ -40,10 +39,9 @@ public class MockApp implements Application {
   Map<ContainerId, Container> containers = new HashMap<ContainerId, Container>();
   ApplicationState appState;
   Application app;
-  String flowName;
-  String flowVersion;
-  long flowRunId;
-  TimelineClient timelineClient = null;
+  private String flowName;
+  private String flowVersion;
+  private long flowRunId;
 
   public MockApp(int uniqId) {
     this("mockUser", 1234, uniqId);

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

@@ -373,7 +373,8 @@ public class RMContextImpl implements RMContext {
   @Override
   public void setRMTimelineCollectorManager(
       RMTimelineCollectorManager timelineCollectorManager) {
-    activeServiceContext.setRMTimelineCollectorManager(timelineCollectorManager);
+    activeServiceContext.setRMTimelineCollectorManager(
+        timelineCollectorManager);
   }
 
   @Override

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

@@ -311,7 +311,7 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
     entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO,
         container.getAllocatedNode().getHost());
     entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO,
-       container.getAllocatedNode().getPort());
+        container.getAllocatedNode().getPort());
     entity.setOtherInfo(entityInfo);
     tEvent.setEventInfo(eventInfo);
 

+ 15 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java

@@ -76,9 +76,9 @@ import org.junit.Test;
 public class TestSystemMetricsPublisherForV2 {
 
   /**
-   * is the folder where the FileSystemTimelineWriterImpl writes the entities
+   * The folder where the FileSystemTimelineWriterImpl writes the entities.
    */
-  protected static File testRootDir = new File("target",
+  private static File testRootDir = new File("target",
       TestSystemMetricsPublisherForV2.class.getName() + "-localDir")
       .getAbsoluteFile();
 
@@ -151,7 +151,8 @@ public class TestSystemMetricsPublisherForV2 {
     } catch (IOException e) {
       e.printStackTrace();
       Assert
-          .fail("Exception while setting the TIMELINE_SERVICE_STORAGE_DIR_ROOT ");
+          .fail("Exception while setting the " +
+              "TIMELINE_SERVICE_STORAGE_DIR_ROOT ");
     }
     return conf;
   }
@@ -159,30 +160,30 @@ public class TestSystemMetricsPublisherForV2 {
   @Test
   public void testSystemMetricPublisherInitialization() {
     @SuppressWarnings("resource")
-    TimelineServiceV2Publisher metricsPublisher =
+    TimelineServiceV2Publisher publisher =
         new TimelineServiceV2Publisher(mock(RMContext.class));
     try {
       Configuration conf = getTimelineV2Conf();
       conf.setBoolean(YarnConfiguration.RM_PUBLISH_CONTAINER_EVENTS_ENABLED,
           YarnConfiguration.DEFAULT_RM_PUBLISH_CONTAINER_EVENTS_ENABLED);
-      metricsPublisher.init(conf);
+      publisher.init(conf);
       assertFalse(
           "Default configuration should not publish container events from RM",
-          metricsPublisher.isPublishContainerEvents());
+          publisher.isPublishContainerEvents());
 
-      metricsPublisher.stop();
+      publisher.stop();
 
-      metricsPublisher = new TimelineServiceV2Publisher(mock(RMContext.class));
+      publisher = new TimelineServiceV2Publisher(mock(RMContext.class));
       conf = getTimelineV2Conf();
-      metricsPublisher.init(conf);
+      publisher.init(conf);
       assertTrue("Expected to have registered event handlers and set ready to "
           + "publish events after init",
-          metricsPublisher.isPublishContainerEvents());
-      metricsPublisher.start();
+          publisher.isPublishContainerEvents());
+      publisher.start();
       assertTrue("Expected to publish container events from RM",
-          metricsPublisher.isPublishContainerEvents());
+          publisher.isPublishContainerEvents());
     } finally {
-      metricsPublisher.stop();
+      publisher.stop();
     }
   }
 
@@ -243,7 +244,7 @@ public class TestSystemMetricsPublisherForV2 {
             + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
     File appFile = new File(outputDirApp, timelineServiceFileName);
     Assert.assertTrue(appFile.exists());
-    verifyEntity(appFile,2, AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE);
+    verifyEntity(appFile, 2, AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE);
   }
 
   @Test(timeout = 10000)

+ 6 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java

@@ -134,7 +134,8 @@ public class TestTimelineServiceClientIntegration {
       ApplicationEntity app = new ApplicationEntity();
       app.setId(appId.toString());
       flow.addChild(app.getType(), app.getId());
-      ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
+      ApplicationAttemptId attemptId =
+          ApplicationAttemptId.newInstance(appId, 1);
       ApplicationAttemptEntity appAttempt = new ApplicationAttemptEntity();
       appAttempt.setId(attemptId.toString());
       ContainerId containerId = ContainerId.newContainerId(attemptId, 1);
@@ -144,8 +145,10 @@ public class TestTimelineServiceClientIntegration {
       user.setId(UserGroupInformation.getCurrentUser().getShortUserName());
       QueueEntity queue = new QueueEntity();
       queue.setId("default_queue");
-      client.putEntities(cluster, flow, app, appAttempt, container, user, queue);
-      client.putEntitiesAsync(cluster, flow, app, appAttempt, container, user, queue);
+      client.putEntities(cluster, flow, app, appAttempt, container, user,
+          queue);
+      client.putEntitiesAsync(cluster, flow, app, appAttempt, container, user,
+          queue);
     } finally {
       client.stop();
     }

+ 27 - 24
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java

@@ -256,17 +256,17 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     entity5.addEvent(event54);
     Map<String, Set<String>> isRelatedTo1 = new HashMap<String, Set<String>>();
     isRelatedTo1.put("type2",
-        Sets.newHashSet("entity21","entity22","entity23","entity24"));
-    isRelatedTo1.put("type4", Sets.newHashSet("entity41","entity42"));
-    isRelatedTo1.put("type1", Sets.newHashSet("entity14","entity15"));
+        Sets.newHashSet("entity21", "entity22", "entity23", "entity24"));
+    isRelatedTo1.put("type4", Sets.newHashSet("entity41", "entity42"));
+    isRelatedTo1.put("type1", Sets.newHashSet("entity14", "entity15"));
     isRelatedTo1.put("type3",
         Sets.newHashSet("entity31", "entity35", "entity32", "entity33"));
     entity5.addIsRelatedToEntities(isRelatedTo1);
     Map<String, Set<String>> relatesTo1 = new HashMap<String, Set<String>>();
     relatesTo1.put("type2",
-        Sets.newHashSet("entity21","entity22","entity23","entity24"));
-    relatesTo1.put("type4", Sets.newHashSet("entity41","entity42"));
-    relatesTo1.put("type1", Sets.newHashSet("entity14","entity15"));
+        Sets.newHashSet("entity21", "entity22", "entity23", "entity24"));
+    relatesTo1.put("type4", Sets.newHashSet("entity41", "entity42"));
+    relatesTo1.put("type1", Sets.newHashSet("entity14", "entity15"));
     relatesTo1.put("type3",
         Sets.newHashSet("entity31", "entity35", "entity32", "entity33"));
     entity5.addRelatesToEntities(relatesTo1);
@@ -317,16 +317,16 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     entity6.addEvent(event64);
     Map<String, Set<String>> isRelatedTo2 = new HashMap<String, Set<String>>();
     isRelatedTo2.put("type2",
-        Sets.newHashSet("entity21","entity22","entity23","entity24"));
-    isRelatedTo2.put("type5", Sets.newHashSet("entity51","entity52"));
-    isRelatedTo2.put("type6", Sets.newHashSet("entity61","entity66"));
+        Sets.newHashSet("entity21", "entity22", "entity23", "entity24"));
+    isRelatedTo2.put("type5", Sets.newHashSet("entity51", "entity52"));
+    isRelatedTo2.put("type6", Sets.newHashSet("entity61", "entity66"));
     isRelatedTo2.put("type3", Sets.newHashSet("entity31"));
     entity6.addIsRelatedToEntities(isRelatedTo2);
     Map<String, Set<String>> relatesTo2 = new HashMap<String, Set<String>>();
     relatesTo2.put("type2",
-        Sets.newHashSet("entity21","entity22","entity23","entity24"));
-    relatesTo2.put("type5", Sets.newHashSet("entity51","entity52"));
-    relatesTo2.put("type6", Sets.newHashSet("entity61","entity66"));
+        Sets.newHashSet("entity21", "entity22", "entity23", "entity24"));
+    relatesTo2.put("type5", Sets.newHashSet("entity51", "entity52"));
+    relatesTo2.put("type6", Sets.newHashSet("entity61", "entity66"));
     relatesTo2.put("type3", Sets.newHashSet("entity31"));
     entity6.addRelatesToEntities(relatesTo2);
     te5.addEntity(entity6);
@@ -391,10 +391,11 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         client.resource(uri).accept(MediaType.APPLICATION_JSON)
         .type(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     if (resp == null ||
-        resp.getClientResponseStatus() != ClientResponse.Status.OK) {
+        resp.getStatusInfo().getStatusCode() !=
+            ClientResponse.Status.OK.getStatusCode()) {
       String msg = "";
       if (resp != null) {
-        msg = resp.getClientResponseStatus().toString();
+        msg = String.valueOf(resp.getStatusInfo().getStatusCode());
       }
       throw new IOException("Incorrect response from timeline reader. " +
           "Status=" + msg);
@@ -406,7 +407,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       implements HttpURLConnectionFactory {
 
     @Override
-    public HttpURLConnection getHttpURLConnection(final URL url) throws IOException {
+    public HttpURLConnection getHttpURLConnection(final URL url)
+        throws IOException {
       try {
         return (HttpURLConnection)url.openConnection();
       } catch (UndeclaredThrowableException e) {
@@ -422,10 +424,10 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   }
 
   private static TimelineMetric newMetric(TimelineMetric.Type type,
-      String id, long ts, Number value) {
+      String id, long t, Number value) {
     TimelineMetric metric = new TimelineMetric(type);
     metric.setId(id);
-    metric.addValue(ts, value);
+    metric.addValue(t, value);
     return metric;
   }
 
@@ -463,7 +465,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         .type(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertNotNull(resp);
     assertTrue("Response from server should have been " + status,
-        resp.getClientResponseStatus().equals(status));
+        resp.getStatusInfo().getStatusCode() == status.getStatusCode());
     System.out.println("Response is: " + resp.getEntity(String.class));
   }
 
@@ -866,7 +868,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       String appUIDWithoutFlowInfo = "cluster1!application_1111111111_1111";
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"+
           "app-uid/" + appUIDWithoutFlowInfo);
-      resp = getResponse(client, uri);;
+      resp = getResponse(client, uri);
       TimelineEntity appEntity2 = resp.getEntity(TimelineEntity.class);
       assertNotNull(appEntity2);
       assertEquals(
@@ -893,7 +895,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       String entityUIDWithFlowInfo = appUIDWithFlowInfo + "!type1!entity1";
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"+
           "entity-uid/" + entityUIDWithFlowInfo);
-      resp = getResponse(client, uri);;
+      resp = getResponse(client, uri);
       TimelineEntity singleEntity1 = resp.getEntity(TimelineEntity.class);
       assertNotNull(singleEntity1);
       assertEquals("type1", singleEntity1.getType());
@@ -903,7 +905,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           appUIDWithoutFlowInfo + "!type1!entity1";
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"+
           "entity-uid/" + entityUIDWithoutFlowInfo);
-      resp = getResponse(client, uri);;
+      resp = getResponse(client, uri);
       TimelineEntity singleEntity2 = resp.getEntity(TimelineEntity.class);
       assertNotNull(singleEntity2);
       assertEquals("type1", singleEntity2.getType());
@@ -1054,12 +1056,12 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertEquals("application_1111111111_2222", entity.getId());
       assertEquals(1, entity.getMetrics().size());
       TimelineMetric m4 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
-         "MAP_SLOT_MILLIS", ts - 80000, 101L);
+          "MAP_SLOT_MILLIS", ts - 80000, 101L);
       for (TimelineMetric metric : entity.getMetrics()) {
         assertTrue(verifyMetrics(metric, m4));
       }
     } finally {
-        client.destroy();
+      client.destroy();
     }
   }
 
@@ -1425,7 +1427,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?infofilters=(info1%20eq%20cluster1%20AND%20info4%20" +
-          "eq%2035000)%20OR%20(info1%20eq%20cluster2%20AND%20info2%20eq%202.0)");
+          "eq%2035000)%20OR%20(info1%20eq%20cluster2%20AND%20info2%20eq%202.0" +
+          ")");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
       assertNotNull(entities);

+ 96 - 96
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java

@@ -444,17 +444,17 @@ public class TestHBaseTimelineStorage {
     te.addEntity(entity2);
     HBaseTimelineWriterImpl hbi = null;
     try {
-        hbi = new HBaseTimelineWriterImpl(util.getConfiguration());
-        hbi.init(util.getConfiguration());
-        hbi.start();
-        String cluster = "cluster1";
-        String user = "user1";
-        String flow = "some_flow_name";
-        String flowVersion = "AB7822C10F1111";
-        long runid = 1002345678919L;
-        String appName = "application_1231111111_1111";
-        hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
-        hbi.stop();
+      hbi = new HBaseTimelineWriterImpl(util.getConfiguration());
+      hbi.init(util.getConfiguration());
+      hbi.start();
+      String cluster = "cluster1";
+      String user = "user1";
+      String flow = "some_flow_name";
+      String flowVersion = "AB7822C10F1111";
+      long runid = 1002345678919L;
+      String appName = "application_1231111111_1111";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.stop();
     } finally {
       if (hbi != null) {
         hbi.stop();
@@ -531,7 +531,7 @@ public class TestHBaseTimelineStorage {
       int count = 0;
       for (Result rr = resultScanner.next(); rr != null;
           rr = resultScanner.next()) {
-         count++;
+        count++;
       }
       // there should be no rows written
       // no exceptions thrown during write
@@ -1173,7 +1173,7 @@ public class TestHBaseTimelineStorage {
       for (TimelineEvent e : events) {
         assertEquals(eventId, e.getId());
         assertEquals(expTs, Long.valueOf(e.getTimestamp()));
-        Map<String,Object> info = e.getInfo();
+        Map<String, Object> info = e.getInfo();
         assertEquals(1, info.size());
         for (Map.Entry<String, Object> infoEntry : info.entrySet()) {
           assertEquals(expKey, infoEntry.getKey());
@@ -1249,7 +1249,7 @@ public class TestHBaseTimelineStorage {
             // the qualifier is a compound key
             // hence match individual values
             assertEquals(eventId, eventColumnName.getId());
-            assertEquals(expTs,eventColumnName.getTimestamp());
+            assertEquals(expTs, eventColumnName.getTimestamp());
             // key must be empty
             assertNull(eventColumnName.getInfoKey());
             Object value = e.getValue();
@@ -1280,7 +1280,7 @@ public class TestHBaseTimelineStorage {
       for (TimelineEvent e : events) {
         assertEquals(eventId, e.getId());
         assertEquals(expTs, Long.valueOf(e.getTimestamp()));
-        Map<String,Object> info = e.getInfo();
+        Map<String, Object> info = e.getInfo();
         assertTrue(info == null || info.isEmpty());
       }
     } finally {
@@ -1337,7 +1337,7 @@ public class TestHBaseTimelineStorage {
       for (TimelineEvent e : events) {
         assertEquals(eventId, e.getId());
         assertEquals(expTs, e.getTimestamp());
-        Map<String,Object> info = e.getInfo();
+        Map<String, Object> info = e.getInfo();
         assertEquals(1, info.size());
         for (Map.Entry<String, Object> infoEntry : info.entrySet()) {
           assertEquals(expKey, infoEntry.getKey());
@@ -1417,14 +1417,14 @@ public class TestHBaseTimelineStorage {
   public void testReadEntities() throws Exception {
     TimelineEntity entity = reader.getEntity(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", "hello"),
+        1002345678919L, "application_1231111111_1111", "world", "hello"),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertNotNull(entity);
     assertEquals(3, entity.getConfigs().size());
     assertEquals(1, entity.getIsRelatedToEntities().size());
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world",
+        1002345678919L, "application_1231111111_1111", "world",
         null), new TimelineEntityFilters(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(3, entities.size());
@@ -1460,7 +1460,7 @@ public class TestHBaseTimelineStorage {
   public void testFilterEntitiesByCreatedTime() throws Exception {
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, 1425016502000L, 1425016502040L, null,
         null, null, null, null, null), new TimelineDataToRetrieve());
     assertEquals(3, entities.size());
@@ -1468,12 +1468,12 @@ public class TestHBaseTimelineStorage {
       if (!entity.getId().equals("hello") && !entity.getId().equals("hello1") &&
           !entity.getId().equals("hello2")) {
         Assert.fail("Entities with ids' hello, hello1 and hello2 should be" +
-           " present");
+            " present");
       }
     }
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, 1425016502015L, null, null, null, null,
         null, null, null), new TimelineDataToRetrieve());
     assertEquals(2, entities.size());
@@ -1485,15 +1485,15 @@ public class TestHBaseTimelineStorage {
     }
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world",  null),
         new TimelineEntityFilters(null, null, 1425016502015L, null, null, null,
         null, null, null), new TimelineDataToRetrieve());
-     assertEquals(1, entities.size());
-     for (TimelineEntity entity : entities) {
-       if (!entity.getId().equals("hello")) {
-         Assert.fail("Entity with id hello should be present");
-       }
-     }
+    assertEquals(1, entities.size());
+    for (TimelineEntity entity : entities) {
+      if (!entity.getId().equals("hello")) {
+        Assert.fail("Entity with id hello should be present");
+      }
+    }
   }
 
   @Test
@@ -1518,7 +1518,7 @@ public class TestHBaseTimelineStorage {
         new HashSet<Object>(Arrays.asList("relatedto5"))));
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, relatesTo, isRelatedTo,
         null, null, null, eventFilter), new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
@@ -1547,7 +1547,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "end_event"));
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         null, ef),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
@@ -1568,7 +1568,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "end_event"));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         null, ef1),
         new TimelineDataToRetrieve());
@@ -1587,7 +1587,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "end_event"));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         null, ef2),
         new TimelineDataToRetrieve());
@@ -1609,7 +1609,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.EQUAL, "dummy_event"));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         null, ef3),
         new TimelineDataToRetrieve());
@@ -1626,7 +1626,7 @@ public class TestHBaseTimelineStorage {
     TimelineFilterList ef4 = new TimelineFilterList(Operator.OR, list1, list2);
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         null, ef4),
         new TimelineDataToRetrieve());
@@ -1647,7 +1647,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "end_event"));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         null, ef5),
         new TimelineDataToRetrieve());
@@ -1656,8 +1656,8 @@ public class TestHBaseTimelineStorage {
     for (TimelineEntity timelineEntity : entities) {
       eventCnt += timelineEntity.getEvents().size();
       if (!timelineEntity.getId().equals("hello")) {
-          Assert.fail("Entity id should have been hello");
-        }
+        Assert.fail("Entity id should have been hello");
+      }
     }
     assertEquals(0, eventCnt);
   }
@@ -1673,7 +1673,7 @@ public class TestHBaseTimelineStorage {
         new HashSet<Object>(Arrays.asList("relatedto4"))));
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, irt, null, null, null,
         null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
@@ -1822,7 +1822,7 @@ public class TestHBaseTimelineStorage {
         new HashSet<Object>(Arrays.asList("relatesto4"))));
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, rt, null, null, null, null,
         null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
@@ -2000,7 +2000,7 @@ public class TestHBaseTimelineStorage {
   public void testReadEntitiesDefaultView() throws Exception {
     TimelineEntity e1 = reader.getEntity(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", "hello"),
+        1002345678919L, "application_1231111111_1111", "world", "hello"),
         new TimelineDataToRetrieve());
     assertNotNull(e1);
     assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() &&
@@ -2008,7 +2008,7 @@ public class TestHBaseTimelineStorage {
         e1.getRelatesToEntities().isEmpty());
     Set<TimelineEntity> es1 = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(),
         new TimelineDataToRetrieve());
     assertEquals(3, es1.size());
@@ -2023,7 +2023,7 @@ public class TestHBaseTimelineStorage {
   public void testReadEntitiesByFields() throws Exception {
     TimelineEntity e1 = reader.getEntity(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", "hello"),
+        1002345678919L, "application_1231111111_1111", "world", "hello"),
         new TimelineDataToRetrieve(
         null, null, EnumSet.of(Field.INFO, Field.CONFIGS), null));
     assertNotNull(e1);
@@ -2031,7 +2031,7 @@ public class TestHBaseTimelineStorage {
     assertEquals(0, e1.getIsRelatedToEntities().size());
     Set<TimelineEntity> es1 = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(),
         new TimelineDataToRetrieve(
         null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS), null));
@@ -2056,13 +2056,13 @@ public class TestHBaseTimelineStorage {
             new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
     TimelineEntity e1 = reader.getEntity(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", "hello"),
+        1002345678919L, "application_1231111111_1111", "world", "hello"),
         new TimelineDataToRetrieve(list, null, null, null));
     assertNotNull(e1);
     assertEquals(1, e1.getConfigs().size());
     Set<TimelineEntity> es1 = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(),
         new TimelineDataToRetrieve(list, null, null, null));
     int cfgCnt = 0;
@@ -2092,7 +2092,7 @@ public class TestHBaseTimelineStorage {
         new TimelineFilterList(Operator.OR, list1, list2);
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList, null, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
@@ -2106,7 +2106,7 @@ public class TestHBaseTimelineStorage {
 
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList, null, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
@@ -2122,7 +2122,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1"));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList1, null, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
@@ -2141,7 +2141,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "config_param2", "value2"));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList2, null, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
@@ -2153,7 +2153,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.EQUAL, "dummy_config", "value1"));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList3, null, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
@@ -2165,7 +2165,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1"));
     entities = reader.getEntities(
             new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-            1002345678919L, "application_1231111111_1111","world", null),
+            1002345678919L, "application_1231111111_1111", "world", null),
             new TimelineEntityFilters(null, null, null, null, null, null,
             confFilterList4, null, null),
             new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
@@ -2177,7 +2177,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1", false));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList5, null, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
@@ -2195,7 +2195,7 @@ public class TestHBaseTimelineStorage {
             new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList, null, null),
         new TimelineDataToRetrieve(list, null, null, null));
@@ -2226,7 +2226,7 @@ public class TestHBaseTimelineStorage {
             new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "config_"));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList1, null, null),
         new TimelineDataToRetrieve(confsToRetrieve, null, null, null));
@@ -2237,7 +2237,7 @@ public class TestHBaseTimelineStorage {
       for (String confKey : entity.getConfigs().keySet()) {
         assertTrue("Config key returned should start with config_",
             confKey.startsWith("config_"));
-       }
+      }
     }
     assertEquals(2, cfgCnt);
   }
@@ -2249,13 +2249,13 @@ public class TestHBaseTimelineStorage {
             new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
     TimelineEntity e1 = reader.getEntity(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", "hello"),
+        1002345678919L, "application_1231111111_1111", "world", "hello"),
         new TimelineDataToRetrieve(null, list, null, null));
     assertNotNull(e1);
     assertEquals(1, e1.getMetrics().size());
     Set<TimelineEntity> es1 = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(),
         new TimelineDataToRetrieve(null, list, null, null));
     int metricCnt = 0;
@@ -2283,7 +2283,7 @@ public class TestHBaseTimelineStorage {
         new TimelineFilterList(Operator.OR, list1, list2);
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
@@ -2297,7 +2297,7 @@ public class TestHBaseTimelineStorage {
 
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
@@ -2315,7 +2315,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "MAP1_BYTES", 30));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList1, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
@@ -2334,7 +2334,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "MAP1_BYTES", 30));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList2, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
@@ -2346,7 +2346,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.EQUAL, "dummy_metric", 5));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList3, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
@@ -2358,7 +2358,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "dummy_metric", 5));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList4, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
@@ -2370,7 +2370,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "dummy_metric", 5, false));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList5, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
@@ -2388,7 +2388,7 @@ public class TestHBaseTimelineStorage {
             new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList, null),
         new TimelineDataToRetrieve(null, list, null, null));
@@ -2417,7 +2417,7 @@ public class TestHBaseTimelineStorage {
         new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList1, null),
         new TimelineDataToRetrieve(
@@ -2436,9 +2436,9 @@ public class TestHBaseTimelineStorage {
     assertEquals(2, metricCnt);
 
     entities = reader.getEntities(new TimelineReaderContext("cluster1", "user1",
-        "some_flow_name", 1002345678919L, "application_1231111111_1111","world",
-        null), new TimelineEntityFilters(null, null, null, null, null, null,
-        null, metricFilterList1, null), new TimelineDataToRetrieve(null,
+        "some_flow_name", 1002345678919L, "application_1231111111_1111",
+        "world", null), new TimelineEntityFilters(null, null, null, null, null,
+        null, null, metricFilterList1, null), new TimelineDataToRetrieve(null,
         metricsToRetrieve, EnumSet.of(Field.METRICS), Integer.MAX_VALUE));
     assertEquals(2, entities.size());
     metricCnt = 0;
@@ -2471,7 +2471,7 @@ public class TestHBaseTimelineStorage {
         new TimelineFilterList(Operator.OR, list1, list2);
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList,
         null, null, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
@@ -2487,7 +2487,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "infoMapKey1", "infoMapValue1"));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList1,
         null, null, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
@@ -2505,7 +2505,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "infoMapKey3", 71.4));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList2,
         null, null, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
@@ -2516,7 +2516,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.EQUAL, "dummy_info", "some_value"));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList3,
         null, null, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
@@ -2527,7 +2527,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "dummy_info", "some_value"));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList4,
         null, null, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
@@ -2538,7 +2538,7 @@ public class TestHBaseTimelineStorage {
         TimelineCompareOp.NOT_EQUAL, "dummy_info", "some_value", false));
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
+        1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList5,
         null, null, null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
@@ -2593,19 +2593,19 @@ public class TestHBaseTimelineStorage {
   @Test
   public void testFilterAppsByCreatedTime() throws Exception {
     Set<TimelineEntity> entities = reader.getEntities(
-       new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-       1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
-       null),
-       new TimelineEntityFilters(null, 1425016502000L, 1425016502040L, null,
-       null, null, null, null, null),
-       new TimelineDataToRetrieve());
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, 1425016502000L, 1425016502040L, null,
+        null, null, null, null, null),
+        new TimelineDataToRetrieve());
     assertEquals(3, entities.size());
     for (TimelineEntity entity : entities) {
       if (!entity.getId().equals("application_1111111111_2222") &&
           !entity.getId().equals("application_1111111111_3333") &&
           !entity.getId().equals("application_1111111111_4444")) {
         Assert.fail("Entities with ids' application_1111111111_2222, " +
-           "application_1111111111_3333 and application_1111111111_4444" +
+            "application_1111111111_3333 and application_1111111111_4444" +
             " should be present");
       }
     }
@@ -2736,8 +2736,8 @@ public class TestHBaseTimelineStorage {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, irt1, null, null, null,
-        null),
+        new TimelineEntityFilters(null, null, null, null, irt1, null, null,
+        null, null),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     isRelatedToCnt = 0;
@@ -2760,8 +2760,8 @@ public class TestHBaseTimelineStorage {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, irt2, null, null, null,
-        null),
+        new TimelineEntityFilters(null, null, null, null, irt2, null, null,
+        null, null),
         new TimelineDataToRetrieve());
     assertEquals(2, entities.size());
     isRelatedToCnt = 0;
@@ -2783,8 +2783,8 @@ public class TestHBaseTimelineStorage {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, irt3, null, null, null,
-        null),
+        new TimelineEntityFilters(null, null, null, null, irt3, null, null,
+        null, null),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     isRelatedToCnt = 0;
@@ -2807,8 +2807,8 @@ public class TestHBaseTimelineStorage {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, irt4, null, null, null,
-        null),
+        new TimelineEntityFilters(null, null, null, null, irt4, null, null,
+        null, null),
         new TimelineDataToRetrieve());
     assertEquals(0, entities.size());
 
@@ -2820,8 +2820,8 @@ public class TestHBaseTimelineStorage {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, irt5, null, null, null,
-        null),
+        new TimelineEntityFilters(null, null, null, null, irt5, null, null,
+        null, null),
         new TimelineDataToRetrieve());
     assertEquals(0, entities.size());
 
@@ -2841,8 +2841,8 @@ public class TestHBaseTimelineStorage {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, irt6, null, null, null,
-        null),
+        new TimelineEntityFilters(null, null, null, null, irt6, null, null,
+        null, null),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     isRelatedToCnt = 0;
@@ -3335,8 +3335,8 @@ public class TestHBaseTimelineStorage {
     for (TimelineEntity timelineEntity : entities) {
       eventCnt += timelineEntity.getEvents().size();
       if (!timelineEntity.getId().equals("application_1111111111_2222")) {
-          Assert.fail("Entity id should have been application_1111111111_2222");
-        }
+        Assert.fail("Entity id should have been application_1111111111_2222");
+      }
     }
     assertEquals(0, eventCnt);
   }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixOfflineAggregationWriterImpl.java

@@ -74,8 +74,8 @@ public class TestPhoenixOfflineAggregationWriterImpl extends BaseTest {
   }
 
   private static PhoenixOfflineAggregationWriterImpl
-    setupPhoenixClusterAndWriterForTest(YarnConfiguration conf)
-      throws Exception{
+      setupPhoenixClusterAndWriterForTest(YarnConfiguration conf)
+      throws Exception {
     Map<String, String> props = new HashMap<>();
     // Must update config before starting server
     props.put(QueryServices.STATS_USE_CURRENT_TIME_ATTRIB,

+ 21 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java

@@ -31,12 +31,14 @@ import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
 import org.apache.hadoop.conf.Configuration;
 
 /**
- * Generates the data/entities for the FlowRun and FlowActivity Tables
+ * Generates the data/entities for the FlowRun and FlowActivity Tables.
  */
-class TestFlowDataGenerator {
+final class TestFlowDataGenerator {
+  private TestFlowDataGenerator() {
+  }
 
-  private static final String metric1 = "MAP_SLOT_MILLIS";
-  private static final String metric2 = "HDFS_BYTES_READ";
+  private static final String METRIC_1 = "MAP_SLOT_MILLIS";
+  private static final String METRIC_2 = "HDFS_BYTES_READ";
   public static final long END_TS_INCR = 10000L;
 
   static TimelineEntity getEntityMetricsApp1(long insertTs, Configuration c1) {
@@ -51,7 +53,7 @@ class TestFlowDataGenerator {
     // add metrics
     Set<TimelineMetric> metrics = new HashSet<>();
     TimelineMetric m1 = new TimelineMetric();
-    m1.setId(metric1);
+    m1.setId(METRIC_1);
     Map<Long, Number> metricValues = new HashMap<Long, Number>();
     long ts = insertTs;
 
@@ -64,10 +66,10 @@ class TestFlowDataGenerator {
     metrics.add(m1);
 
     TimelineMetric m2 = new TimelineMetric();
-    m2.setId(metric2);
+    m2.setId(METRIC_2);
     metricValues = new HashMap<Long, Number>();
     ts = System.currentTimeMillis();
-    for (int k=1; k< 100 ; k++) {
+    for (int k = 1; k < 100; k++) {
       metricValues.put(ts - k*100000L, 31L);
     }
 
@@ -81,7 +83,8 @@ class TestFlowDataGenerator {
   }
 
 
-  static TimelineEntity getEntityMetricsApp1Complete(long insertTs, Configuration c1) {
+  static TimelineEntity getEntityMetricsApp1Complete(long insertTs,
+      Configuration c1) {
     TimelineEntity entity = new TimelineEntity();
     String id = "flowRunMetrics_test";
     String type = TimelineEntityType.YARN_APPLICATION.toString();
@@ -93,7 +96,7 @@ class TestFlowDataGenerator {
     // add metrics
     Set<TimelineMetric> metrics = new HashSet<>();
     TimelineMetric m1 = new TimelineMetric();
-    m1.setId(metric1);
+    m1.setId(METRIC_1);
     Map<Long, Number> metricValues = new HashMap<Long, Number>();
     long ts = insertTs;
 
@@ -103,7 +106,7 @@ class TestFlowDataGenerator {
     metrics.add(m1);
 
     TimelineMetric m2 = new TimelineMetric();
-    m2.setId(metric2);
+    m2.setId(METRIC_2);
     metricValues = new HashMap<Long, Number>();
     ts = insertTs;
     metricValues.put(ts - 80000, 57L);
@@ -134,7 +137,7 @@ class TestFlowDataGenerator {
     // add metrics
     Set<TimelineMetric> metrics = new HashSet<>();
     TimelineMetric m1 = new TimelineMetric();
-    m1.setId(metric1);
+    m1.setId(METRIC_1);
     Map<Long, Number> metricValues = new HashMap<Long, Number>();
     long ts = insertTs;
     metricValues.put(ts - 100000, 2L);
@@ -144,7 +147,7 @@ class TestFlowDataGenerator {
     metrics.add(m1);
 
     TimelineMetric m2 = new TimelineMetric();
-    m2.setId(metric2);
+    m2.setId(METRIC_2);
     metricValues = new HashMap<Long, Number>();
     ts = insertTs;
     metricValues.put(ts - 100000, 31L);
@@ -177,7 +180,7 @@ class TestFlowDataGenerator {
     // add metrics
     Set<TimelineMetric> metrics = new HashSet<>();
     TimelineMetric m1 = new TimelineMetric();
-    m1.setId(metric1);
+    m1.setId(METRIC_1);
     Map<Long, Number> metricValues = new HashMap<Long, Number>();
     long ts = insertTs;
     metricValues.put(ts - 100000, 5L);
@@ -208,7 +211,7 @@ class TestFlowDataGenerator {
     // add metrics
     Set<TimelineMetric> metrics = new HashSet<>();
     TimelineMetric m1 = new TimelineMetric();
-    m1.setId(metric1);
+    m1.setId(METRIC_1);
     Map<Long, Number> metricValues = new HashMap<Long, Number>();
     long ts = System.currentTimeMillis();
     metricValues.put(ts - 120000, 100000000L);
@@ -232,7 +235,7 @@ class TestFlowDataGenerator {
 
     event = new TimelineEvent();
     event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
-    long expTs = cTime + 21600000;// start time + 6hrs
+    long expTs = cTime + 21600000; // start time + 6hrs
     event.setTimestamp(expTs);
     event.addInfo(expKey, expVal);
     entity.addEvent(event);
@@ -250,7 +253,7 @@ class TestFlowDataGenerator {
     // add metrics
     Set<TimelineMetric> metrics = new HashSet<>();
     TimelineMetric m1 = new TimelineMetric();
-    m1.setId(metric1);
+    m1.setId(METRIC_1);
     Map<Long, Number> metricValues = new HashMap<Long, Number>();
     metricValues.put(ts - 120000, 100000000L);
     metricValues.put(ts - 100000, 200000000L);
@@ -262,7 +265,7 @@ class TestFlowDataGenerator {
     m1.setValues(metricValues);
     metrics.add(m1);
     TimelineMetric m2 = new TimelineMetric();
-    m2.setId(metric2);
+    m2.setId(METRIC_2);
     metricValues = new HashMap<Long, Number>();
     metricValues.put(ts - 900000, 31L);
     metricValues.put(ts - 30000, 57L);
@@ -281,7 +284,7 @@ class TestFlowDataGenerator {
 
     event = new TimelineEvent();
     event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
-    long expTs = ts + 21600000;// start time + 6hrs
+    long expTs = ts + 21600000; // start time + 6hrs
     event.setTimestamp(expTs);
     event.addInfo(expKey, expVal);
     entity.addEvent(event);

+ 7 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java

@@ -58,7 +58,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
- * Tests the FlowRun and FlowActivity Tables
+ * Tests the FlowRun and FlowActivity Tables.
  */
 public class TestHBaseStorageFlowActivity {
 
@@ -114,7 +114,7 @@ public class TestHBaseStorageFlowActivity {
     String appName = "application_100000000000_1111";
     long minStartTs = 1424995200300L;
     long greaterStartTs = 1424995200300L + 864000L;
-    long endTs = 1424995200300L + 86000000L;;
+    long endTs = 1424995200300L + 86000000L;
     TimelineEntity entityMinStartTime = TestFlowDataGenerator
         .getEntityMinStartTime(minStartTs);
 
@@ -209,7 +209,7 @@ public class TestHBaseStorageFlowActivity {
 
   /**
    * Write 1 application entity and checks the record for today in the flow
-   * activity table
+   * activity table.
    */
   @Test
   public void testWriteFlowActivityOneFlow() throws Exception {
@@ -313,10 +313,10 @@ public class TestHBaseStorageFlowActivity {
 
   /**
    * Writes 3 applications each with a different run id and version for the same
-   * {cluster, user, flow}
+   * {cluster, user, flow}.
    *
    * They should be getting inserted into one record in the flow activity table
-   * with 3 columns, one per run id
+   * with 3 columns, one per run id.
    */
   @Test
   public void testFlowActivityTableOneFlowMultipleRunIds() throws IOException {
@@ -425,7 +425,8 @@ public class TestHBaseStorageFlowActivity {
     s.setStartRow(startRow);
     String clusterStop = cluster + "1";
     byte[] stopRow =
-        new FlowActivityRowKey(clusterStop, appCreatedTime, user, flow).getRowKey();
+        new FlowActivityRowKey(clusterStop, appCreatedTime, user, flow)
+        .getRowKey();
     s.setStopRow(stopRow);
     Connection conn = ConnectionFactory.createConnection(c1);
     Table table1 = conn.getTable(TableName

+ 6 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java

@@ -69,7 +69,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
- * Tests the FlowRun and FlowActivity Tables
+ * Tests the FlowRun and FlowActivity Tables.
  */
 public class TestHBaseStorageFlowRun {
 
@@ -356,18 +356,20 @@ public class TestHBaseStorageFlowRun {
   /*
    * checks the batch limits on a scan
    */
-   void checkFlowRunTableBatchLimit(String cluster, String user,
+  void checkFlowRunTableBatchLimit(String cluster, String user,
       String flow, long runid, Configuration c1) throws IOException {
 
     Scan s = new Scan();
     s.addFamily(FlowRunColumnFamily.INFO.getBytes());
-    byte[] startRow =  new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
+    byte[] startRow =
+        new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
     s.setStartRow(startRow);
     // set a batch limit
     int batchLimit = 2;
     s.setBatch(batchLimit);
     String clusterStop = cluster + "1";
-    byte[] stopRow = new FlowRunRowKey(clusterStop, user, flow, runid).getRowKey();
+    byte[] stopRow =
+        new FlowRunRowKey(clusterStop, user, flow, runid).getRowKey();
     s.setStopRow(stopRow);
     Connection conn = ConnectionFactory.createConnection(c1);
     Table table1 = conn

+ 18 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java

@@ -63,14 +63,14 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
- * Tests the FlowRun and FlowActivity Tables
+ * Tests the FlowRun and FlowActivity Tables.
  */
 public class TestHBaseStorageFlowRunCompaction {
 
   private static HBaseTestingUtility util;
 
-  private static final String metric1 = "MAP_SLOT_MILLIS";
-  private static final String metric2 = "HDFS_BYTES_READ";
+  private static final String METRIC_1 = "MAP_SLOT_MILLIS";
+  private static final String METRIC_2 = "HDFS_BYTES_READ";
 
   private final byte[] aRowKey = Bytes.toBytes("a");
   private final byte[] aFamily = Bytes.toBytes("family");
@@ -89,8 +89,8 @@ public class TestHBaseStorageFlowRunCompaction {
     TimelineSchemaCreator.createAllTables(util.getConfiguration(), false);
   }
 
-  /** writes non numeric data into flow run table
-   * reads it back
+  /** Writes non numeric data into flow run table
+   * reads it back.
    *
    * @throws Exception
    */
@@ -262,7 +262,7 @@ public class TestHBaseStorageFlowRunCompaction {
           .getFamilyMap(FlowRunColumnFamily.INFO.getBytes());
       // we expect all back in one next call
       assertEquals(4, values.size());
-      System.out.println(" values size " + values.size() +  " " + batchLimit );
+      System.out.println(" values size " + values.size() +  " " + batchLimit);
       rowCount++;
     }
     // should get back 1 row with each invocation
@@ -325,11 +325,12 @@ public class TestHBaseStorageFlowRunCompaction {
         .valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
     List<Region> regions = server.getOnlineRegions(TableName
         .valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
-    assertTrue("Didn't find any regions for primary table!", regions.size() > 0);
+    assertTrue("Didn't find any regions for primary table!",
+        regions.size() > 0);
     // flush and compact all the regions of the primary table
     for (Region region : regions) {
-       region.flush(true);
-       region.compact(true);
+      region.flush(true);
+      region.compact(true);
     }
 
     // check flow run for one flow many apps
@@ -363,13 +364,13 @@ public class TestHBaseStorageFlowRunCompaction {
       rowCount++;
       // check metric1
       byte[] q = ColumnHelper.getColumnQualifier(
-          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), metric1);
+          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), METRIC_1);
       assertTrue(values.containsKey(q));
       assertEquals(141, Bytes.toLong(values.get(q)));
 
       // check metric2
       q = ColumnHelper.getColumnQualifier(
-          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), metric2);
+          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), METRIC_2);
       assertTrue(values.containsKey(q));
       assertEquals(57, Bytes.toLong(values.get(q)));
     }
@@ -385,7 +386,7 @@ public class TestHBaseStorageFlowRunCompaction {
     // okay to pass in nulls for the constructor arguments
     // because all we want to do is invoke the process summation
     FlowScanner fs = new FlowScanner(null, null,
-        (request.isMajor() == true ? FlowScannerOperation.MAJOR_COMPACTION
+        (request.isMajor() ? FlowScannerOperation.MAJOR_COMPACTION
             : FlowScannerOperation.MINOR_COMPACTION));
     assertNotNull(fs);
     return fs;
@@ -404,7 +405,7 @@ public class TestHBaseStorageFlowRunCompaction {
     long currentTimestamp = System.currentTimeMillis();
     long cell1Ts = 1200120L;
     long cell2Ts = TimestampGenerator.getSupplementedTimestamp(
-        System.currentTimeMillis(),"application_123746661110_11202");
+        System.currentTimeMillis(), "application_123746661110_11202");
     long cell3Ts = 1277719L;
     long cell4Ts = currentTimestamp - 10;
 
@@ -571,7 +572,8 @@ public class TestHBaseStorageFlowRunCompaction {
   // of type SUM and SUM_FINAL
   // NOT cells of SUM_FINAL will expire
   @Test
-  public void checkProcessSummationMoreCellsSumFinalVariedTags() throws IOException {
+  public void checkProcessSummationMoreCellsSumFinalVariedTags()
+      throws IOException {
     FlowScanner fs = getFlowScannerForTestingCompaction();
     int countFinal = 20100;
     int countNotFinal = 1000;
@@ -585,7 +587,8 @@ public class TestHBaseStorageFlowRunCompaction {
     long cellTsFinalStart = 10001120L;
     long cellTsFinal = cellTsFinalStart;
 
-    long cellTsFinalStartNotExpire = TimestampGenerator.getSupplementedTimestamp(
+    long cellTsFinalStartNotExpire =
+        TimestampGenerator.getSupplementedTimestamp(
         System.currentTimeMillis(), "application_10266666661166_118821");
     long cellTsFinalNotExpire = cellTsFinalStartNotExpire;
 

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java

@@ -53,7 +53,6 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.api.records.timelineservice.UserEntity;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 
@@ -180,7 +179,7 @@ public class TimelineCollectorWebService {
   private static ApplicationId parseApplicationId(String appId) {
     try {
       if (appId != null) {
-        return ConverterUtils.toApplicationId(appId.trim());
+        return ApplicationId.fromString(appId.trim());
       } else {
         return null;
       }

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java

@@ -26,4 +26,3 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.application;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.common;
 
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 
 /**
  * Encodes and decodes {@link ApplicationId} for row keys.
@@ -50,7 +49,7 @@ public final class AppIdKeyConverter implements KeyConverter<String> {
    */
   @Override
   public byte[] encode(String appIdStr) {
-    ApplicationId appId = ConverterUtils.toApplicationId(appIdStr);
+    ApplicationId appId = ApplicationId.fromString(appIdStr);
     byte[] appIdBytes = new byte[getKeySize()];
     byte[] clusterTs = Bytes.toBytes(
         LongConverter.invertLong(appId.getClusterTimestamp()));

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java

@@ -21,7 +21,6 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.common;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 
 /**
  * Utility class that allows HBase coprocessors to interact with unique
@@ -99,7 +98,7 @@ public class TimestampGenerator {
     if (appIdStr == null) {
       return 0L;
     }
-    ApplicationId appId = ConverterUtils.toApplicationId(appIdStr);
+    ApplicationId appId = ApplicationId.fromString(appIdStr);
     long id = appId.getId() % TS_MULTIPLIER;
     return id;
   }

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java

@@ -26,4 +26,3 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.entity;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-

+ 12 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestNMTimelineCollectorManager.java

@@ -88,9 +88,9 @@ public class TestNMTimelineCollectorManager {
 
   @Test(timeout=60000)
   public void testMultithreadedAdd() throws Exception {
-    final int NUM_APPS = 5;
+    final int numApps = 5;
     List<Callable<Boolean>> tasks = new ArrayList<Callable<Boolean>>();
-    for (int i = 0; i < NUM_APPS; i++) {
+    for (int i = 0; i < numApps; i++) {
       final ApplicationId appId = ApplicationId.newInstance(0L, i);
       Callable<Boolean> task = new Callable<Boolean>() {
         public Boolean call() {
@@ -101,7 +101,7 @@ public class TestNMTimelineCollectorManager {
       };
       tasks.add(task);
     }
-    ExecutorService executor = Executors.newFixedThreadPool(NUM_APPS);
+    ExecutorService executor = Executors.newFixedThreadPool(numApps);
     try {
       List<Future<Boolean>> futures = executor.invokeAll(tasks);
       for (Future<Boolean> future: futures) {
@@ -111,7 +111,7 @@ public class TestNMTimelineCollectorManager {
       executor.shutdownNow();
     }
     // check the keys
-    for (int i = 0; i < NUM_APPS; i++) {
+    for (int i = 0; i < numApps; i++) {
       final ApplicationId appId = ApplicationId.newInstance(0L, i);
       assertTrue(collectorManager.containsTimelineCollector(appId));
     }
@@ -119,9 +119,9 @@ public class TestNMTimelineCollectorManager {
 
   @Test
   public void testMultithreadedAddAndRemove() throws Exception {
-    final int NUM_APPS = 5;
+    final int numApps = 5;
     List<Callable<Boolean>> tasks = new ArrayList<Callable<Boolean>>();
-    for (int i = 0; i < NUM_APPS; i++) {
+    for (int i = 0; i < numApps; i++) {
       final ApplicationId appId = ApplicationId.newInstance(0L, i);
       Callable<Boolean> task = new Callable<Boolean>() {
         public Boolean call() {
@@ -134,7 +134,7 @@ public class TestNMTimelineCollectorManager {
       };
       tasks.add(task);
     }
-    ExecutorService executor = Executors.newFixedThreadPool(NUM_APPS);
+    ExecutorService executor = Executors.newFixedThreadPool(numApps);
     try {
       List<Future<Boolean>> futures = executor.invokeAll(tasks);
       for (Future<Boolean> future: futures) {
@@ -144,16 +144,16 @@ public class TestNMTimelineCollectorManager {
       executor.shutdownNow();
     }
     // check the keys
-    for (int i = 0; i < NUM_APPS; i++) {
+    for (int i = 0; i < numApps; i++) {
       final ApplicationId appId = ApplicationId.newInstance(0L, i);
       assertFalse(collectorManager.containsTimelineCollector(appId));
     }
   }
 
   private NodeTimelineCollectorManager createCollectorManager() {
-    final NodeTimelineCollectorManager collectorManager =
+    final NodeTimelineCollectorManager cm =
         spy(new NodeTimelineCollectorManager());
-    doReturn(new Configuration()).when(collectorManager).getConfig();
+    doReturn(new Configuration()).when(cm).getConfig();
     CollectorNodemanagerProtocol nmCollectorService =
         mock(CollectorNodemanagerProtocol.class);
     GetTimelineCollectorContextResponse response =
@@ -164,7 +164,7 @@ public class TestNMTimelineCollectorManager {
     } catch (YarnException | IOException e) {
       fail();
     }
-    doReturn(nmCollectorService).when(collectorManager).getNMCollectorService();
-    return collectorManager;
+    doReturn(nmCollectorService).when(cm).getNMCollectorService();
+    return cm;
   }
 }

+ 8 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java

@@ -111,7 +111,7 @@ public class TestPerNodeTimelineCollectorsAuxService {
     // a configured period
     assertTrue(auxService.hasApplication(appAttemptId.getApplicationId()));
     for (int i = 0; i < 4; i++) {
-      Thread.sleep(500l);
+      Thread.sleep(500L);
       if (!auxService.hasApplication(appAttemptId.getApplicationId())) {
         break;
       }
@@ -154,7 +154,7 @@ public class TestPerNodeTimelineCollectorsAuxService {
 
   private PerNodeTimelineCollectorsAuxService
       createCollectorAndAddApplication() {
-    PerNodeTimelineCollectorsAuxService auxService = createCollector();
+    PerNodeTimelineCollectorsAuxService service = createCollector();
     // create an AM container
     ContainerId containerId = getAMContainerId();
     ContainerInitializationContext context =
@@ -162,17 +162,17 @@ public class TestPerNodeTimelineCollectorsAuxService {
     when(context.getContainerId()).thenReturn(containerId);
     when(context.getContainerType()).thenReturn(
         ContainerType.APPLICATION_MASTER);
-    auxService.initializeContainer(context);
-    return auxService;
+    service.initializeContainer(context);
+    return service;
   }
 
   private PerNodeTimelineCollectorsAuxService createCollector() {
     NodeTimelineCollectorManager collectorManager = createCollectorManager();
-    PerNodeTimelineCollectorsAuxService auxService =
+    PerNodeTimelineCollectorsAuxService service =
         spy(new PerNodeTimelineCollectorsAuxService(collectorManager));
-    auxService.init(conf);
-    auxService.start();
-    return auxService;
+    service.init(conf);
+    service.start();
+    return service;
   }
 
   private NodeTimelineCollectorManager createCollectorManager() {

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderUtils.java

@@ -45,11 +45,11 @@ public class TestTimelineReaderUtils {
   public void testJoinAndEscapeStrings() throws Exception {
     assertEquals("*!cluster!*!b**o***!xer!oozie**",
         TimelineReaderUtils.joinAndEscapeStrings(
-            new String[] { "!cluster", "!b*o*!xer", "oozie*"}, '!', '*'));
+            new String[] {"!cluster", "!b*o*!xer", "oozie*"}, '!', '*'));
     assertEquals("*!cluster!*!b**o***!xer!!",
         TimelineReaderUtils.joinAndEscapeStrings(
-            new String[] { "!cluster", "!b*o*!xer", "", ""}, '!', '*'));
+            new String[] {"!cluster", "!b*o*!xer", "", ""}, '!', '*'));
     assertNull(TimelineReaderUtils.joinAndEscapeStrings(
-        new String[] { "!cluster", "!b*o*!xer", null, ""}, '!', '*'));
+        new String[] {"!cluster", "!b*o*!xer", null, ""}, '!', '*'));
   }
 }

+ 7 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java

@@ -110,7 +110,8 @@ public class TestTimelineReaderWebServices {
         client.resource(uri).accept(MediaType.APPLICATION_JSON)
         .type(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertNotNull(resp);
-    assertEquals(resp.getClientResponseStatus(), expectedStatus);
+    assertEquals(resp.getStatusInfo().getStatusCode(),
+        expectedStatus.getStatusCode());
   }
 
   private static Client createClient() {
@@ -126,10 +127,11 @@ public class TestTimelineReaderWebServices {
         client.resource(uri).accept(MediaType.APPLICATION_JSON)
         .type(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     if (resp == null ||
-        resp.getClientResponseStatus() != ClientResponse.Status.OK) {
+        resp.getStatusInfo().getStatusCode() !=
+            ClientResponse.Status.OK.getStatusCode()) {
       String msg = new String();
       if (resp != null) {
-        msg = resp.getClientResponseStatus().toString();
+        msg = String.valueOf(resp.getStatusInfo().getStatusCode());
       }
       throw new IOException("Incorrect response from timeline reader. " +
           "Status=" + msg);
@@ -141,7 +143,8 @@ public class TestTimelineReaderWebServices {
       implements HttpURLConnectionFactory {
 
     @Override
-    public HttpURLConnection getHttpURLConnection(final URL url) throws IOException {
+    public HttpURLConnection getHttpURLConnection(final URL url)
+        throws IOException {
       try {
         return (HttpURLConnection)url.openConnection();
       } catch (UndeclaredThrowableException e) {

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesUtils.java

@@ -779,8 +779,8 @@ public class TestTimelineReaderWebServicesUtils {
                 ),
                 new TimelineFilterList(
                     new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
-                        "type4", Sets.newHashSet((Object)"entity43","entity44",
-                            "entity47","entity49")),
+                        "type4", Sets.newHashSet((Object)"entity43", "entity44",
+                            "entity47", "entity49")),
                     new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
                         "type7", Sets.newHashSet((Object)"entity71"))
                 )

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineUIDConverter.java

@@ -60,7 +60,7 @@ public class TestTimelineUIDConverter {
         + "YARN_CONTAINER!container_1111111111_1111_01_000001", uid);
     assertEquals(
         context, TimelineUIDConverter.GENERIC_ENTITY_UID.decodeUID(uid));
-    context = new TimelineReaderContext("yarn_cluster",null, null, null,
+    context = new TimelineReaderContext("yarn_cluster", null, null, null,
         "application_1111111111_1111", "YARN_CONTAINER",
         "container_1111111111_1111_01_000001");
     uid = TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID(context);

+ 22 - 22
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java

@@ -58,9 +58,9 @@ import org.junit.Test;
 
 public class TestFileSystemTimelineReaderImpl {
 
-  private static final String rootDir =
+  private static final String ROOT_DIR =
       FileSystemTimelineReaderImpl.DEFAULT_TIMELINE_SERVICE_STORAGE_DIR_ROOT;
-  FileSystemTimelineReaderImpl reader;
+  private FileSystemTimelineReaderImpl reader;
 
   @BeforeClass
   public static void setup() throws Exception {
@@ -68,22 +68,22 @@ public class TestFileSystemTimelineReaderImpl {
     // Create app flow mapping file.
     CSVFormat format =
         CSVFormat.DEFAULT.withHeader("APP", "USER", "FLOW", "FLOWRUN");
-    String appFlowMappingFile = rootDir + "/entities/cluster1/" +
+    String appFlowMappingFile = ROOT_DIR + "/entities/cluster1/" +
         FileSystemTimelineReaderImpl.APP_FLOW_MAPPING_FILE;
     try (PrintWriter out =
         new PrintWriter(new BufferedWriter(
             new FileWriter(appFlowMappingFile, true)));
         CSVPrinter printer = new CSVPrinter(out, format)){
       printer.printRecord("app1", "user1", "flow1", 1);
-      printer.printRecord("app2","user1","flow1,flow",1);
+      printer.printRecord("app2", "user1", "flow1,flow", 1);
       printer.close();
     }
-    (new File(rootDir)).deleteOnExit();
+    (new File(ROOT_DIR)).deleteOnExit();
   }
 
   @AfterClass
   public static void tearDown() throws Exception {
-    FileUtils.deleteDirectory(new File(rootDir));
+    FileUtils.deleteDirectory(new File(ROOT_DIR));
   }
 
   @Before
@@ -91,7 +91,7 @@ public class TestFileSystemTimelineReaderImpl {
     reader = new FileSystemTimelineReaderImpl();
     Configuration conf = new YarnConfiguration();
     conf.set(FileSystemTimelineReaderImpl.TIMELINE_SERVICE_STORAGE_DIR_ROOT,
-        rootDir);
+        ROOT_DIR);
     reader.init(conf);
   }
 
@@ -112,7 +112,7 @@ public class TestFileSystemTimelineReaderImpl {
   }
 
   private static void loadEntityData() throws Exception {
-    File appDir = new File(rootDir +
+    File appDir = new File(ROOT_DIR +
         "/entities/cluster1/user1/flow1/1/app1/app/");
     TimelineEntity entity11 = new TimelineEntity();
     entity11.setId("id_1");
@@ -138,7 +138,7 @@ public class TestFileSystemTimelineReaderImpl {
     metric2.addValue(1425016502016L, 34);
     metrics.add(metric2);
     entity11.setMetrics(metrics);
-    Map<String,String> configs = new HashMap<String, String>();
+    Map<String, String> configs = new HashMap<String, String>();
     configs.put("config_1", "127");
     entity11.setConfigs(configs);
     entity11.addRelatesToEntity("flow", "flow1");
@@ -179,7 +179,7 @@ public class TestFileSystemTimelineReaderImpl {
     Map<String, Object> info2 = new HashMap<String, Object>();
     info1.put("info2", 4);
     entity2.addInfo(info2);
-    Map<String,String> configs2 = new HashMap<String, String>();
+    Map<String, String> configs2 = new HashMap<String, String>();
     configs2.put("config_1", "129");
     configs2.put("config_3", "def");
     entity2.setConfigs(configs2);
@@ -216,7 +216,7 @@ public class TestFileSystemTimelineReaderImpl {
     info3.put("info2", 3.5);
     info3.put("info4", 20);
     entity3.addInfo(info3);
-    Map<String,String> configs3 = new HashMap<String, String>();
+    Map<String, String> configs3 = new HashMap<String, String>();
     configs3.put("config_1", "123");
     configs3.put("config_3", "abc");
     entity3.setConfigs(configs3);
@@ -254,7 +254,7 @@ public class TestFileSystemTimelineReaderImpl {
     entity4.addEvent(event44);
     writeEntityFile(entity4, appDir);
 
-    File appDir2 = new File(rootDir +
+    File appDir2 = new File(ROOT_DIR +
             "/entities/cluster1/user1/flow1,flow/1/app2/app/");
     TimelineEntity entity5 = new TimelineEntity();
     entity5.setId("id_5");
@@ -298,7 +298,7 @@ public class TestFileSystemTimelineReaderImpl {
     Assert.assertEquals(0, result.getMetrics().size());
   }
 
-  /** This test checks whether we can handle commas in app flow mapping csv */
+  /** This test checks whether we can handle commas in app flow mapping csv. */
   @Test
   public void testAppFlowMappingCsv() throws Exception {
     // Test getting an entity by cluster and app where flow entry
@@ -317,7 +317,7 @@ public class TestFileSystemTimelineReaderImpl {
   public void testGetEntityCustomFields() throws Exception {
     // Specified fields in addition to default view will be returned.
     TimelineEntity result = reader.getEntity(
-        new TimelineReaderContext("cluster1","user1", "flow1", 1L, "app1",
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", "id_1"),
         new TimelineDataToRetrieve(null, null,
         EnumSet.of(Field.INFO, Field.CONFIGS, Field.METRICS), null));
@@ -336,7 +336,7 @@ public class TestFileSystemTimelineReaderImpl {
   public void testGetEntityAllFields() throws Exception {
     // All fields of TimelineEntity will be returned.
     TimelineEntity result = reader.getEntity(
-        new TimelineReaderContext("cluster1","user1", "flow1", 1L, "app1",
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", "id_1"),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     Assert.assertEquals(
@@ -381,9 +381,9 @@ public class TestFileSystemTimelineReaderImpl {
         "app", null),
         new TimelineEntityFilters(3L, null, null, null, null, null, null,
         null, null), new TimelineDataToRetrieve());
-     // Even though 2 entities out of 4 have same created time, one entity
-     // is left out due to limit
-     Assert.assertEquals(3, result.size());
+    // Even though 2 entities out of 4 have same created time, one entity
+    // is left out due to limit
+    Assert.assertEquals(3, result.size());
   }
 
   @Test
@@ -474,9 +474,9 @@ public class TestFileSystemTimelineReaderImpl {
     // Get entities based on event filters.
     TimelineFilterList eventFilters = new TimelineFilterList();
     eventFilters.addFilter(
-        new TimelineExistsFilter(TimelineCompareOp.EQUAL,"event_2"));
+        new TimelineExistsFilter(TimelineCompareOp.EQUAL, "event_2"));
     eventFilters.addFilter(
-        new TimelineExistsFilter(TimelineCompareOp.EQUAL,"event_4"));
+        new TimelineExistsFilter(TimelineCompareOp.EQUAL, "event_4"));
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
@@ -642,7 +642,7 @@ public class TestFileSystemTimelineReaderImpl {
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList2, null),
         new TimelineDataToRetrieve());
-   Assert.assertEquals(1, result.size());
+    Assert.assertEquals(1, result.size());
     for (TimelineEntity entity : result) {
       if (!entity.getId().equals("id_1")) {
         Assert.fail("Incorrect filtering based on metric filters");
@@ -757,7 +757,7 @@ public class TestFileSystemTimelineReaderImpl {
         Assert.fail("Incorrect filtering based on info filters");
       }
     }
-   }
+  }
 
   @Test
   public void testGetEntitiesByRelations() throws Exception {

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java

@@ -41,7 +41,8 @@ import org.junit.Test;
 public class TestFileSystemTimelineWriterImpl {
 
   /**
-   * Unit test for PoC YARN 3264
+   * Unit test for PoC YARN 3264.
+   *
    * @throws Exception
    */
   @Test
@@ -84,8 +85,8 @@ public class TestFileSystemTimelineWriterImpl {
           "app_id", te);
 
       String fileName = fsi.getOutputRoot() +
-          "/entities/cluster_id/user_id/flow_name/flow_version/12345678/app_id/" +
-          type + "/" + id +
+          "/entities/cluster_id/user_id/flow_name/flow_version/12345678/" +
+          "app_id/" + type + "/" + id +
           FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
       Path path = Paths.get(fileName);
       File f = new File(fileName);

+ 12 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java

@@ -60,7 +60,8 @@ public class TestRowKeys {
     if (sepByteLen <= byteArr.length) {
       for (int i = 0; i < sepByteLen; i++) {
         byteArr[byteArr.length - sepByteLen + i] =
-            (byte) (longMaxByteArr[byteArr.length - sepByteLen + i] - QUALIFIER_SEP_BYTES[i]);
+            (byte) (longMaxByteArr[byteArr.length - sepByteLen + i] -
+                QUALIFIER_SEP_BYTES[i]);
       }
     }
     clusterTs = Bytes.toLong(byteArr);
@@ -73,7 +74,8 @@ public class TestRowKeys {
     for (int i = 0; i < sepLen; i++) {
       assertTrue(
           "Row key prefix not encoded properly.",
-          byteRowKeyPrefix[byteRowKeyPrefix.length - sepLen + i] == QUALIFIER_SEP_BYTES[i]);
+          byteRowKeyPrefix[byteRowKeyPrefix.length - sepLen + i] ==
+              QUALIFIER_SEP_BYTES[i]);
     }
   }
 
@@ -94,9 +96,9 @@ public class TestRowKeys {
             .getRowKeyPrefix();
     byte[][] splits =
         Separator.QUALIFIERS.split(byteRowKeyPrefix,
-            new int[] { Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+            new int[] {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
                 Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
-                Separator.VARIABLE_SIZE });
+                Separator.VARIABLE_SIZE});
     assertEquals(5, splits.length);
     assertEquals(0, splits[4].length);
     assertEquals(FLOW_NAME,
@@ -153,10 +155,10 @@ public class TestRowKeys {
     byte[][] splits =
         Separator.QUALIFIERS.split(
             byteRowKeyPrefix,
-            new int[] { Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+            new int[] {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
                 Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
                 AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE,
-                Separator.VARIABLE_SIZE });
+                Separator.VARIABLE_SIZE});
     assertEquals(7, splits.length);
     assertEquals(0, splits[6].length);
     assertEquals(APPLICATION_ID, new AppIdKeyConverter().decode(splits[4]));
@@ -170,9 +172,9 @@ public class TestRowKeys {
     splits =
         Separator.QUALIFIERS.split(
             byteRowKeyPrefix,
-            new int[] { Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+            new int[] {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
                 Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
-                AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE });
+                AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE});
     assertEquals(6, splits.length);
     assertEquals(0, splits[5].length);
     AppIdKeyConverter appIdKeyConverter = new AppIdKeyConverter();
@@ -207,8 +209,8 @@ public class TestRowKeys {
         new FlowActivityRowKeyPrefix(CLUSTER, ts).getRowKeyPrefix();
     splits =
         Separator.QUALIFIERS.split(byteRowKeyPrefix,
-            new int[] { Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
-                Separator.VARIABLE_SIZE });
+            new int[] {Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
+                Separator.VARIABLE_SIZE});
     assertEquals(3, splits.length);
     assertEquals(0, splits[2].length);
     assertEquals(CLUSTER,

+ 14 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java

@@ -86,7 +86,7 @@ public class TestSeparator {
     testEncodeDecode("Double-escape %2$ and %9$ or %%2$ or %%3$, nor  %%%2$" +
         "= no problem!",
         Separator.QUALIFIERS, Separator.VALUES, Separator.SPACE, Separator.TAB);
-   }
+  }
 
   @Test
   public void testSplits() {
@@ -101,10 +101,10 @@ public class TestSeparator {
       byte[] intVal1Arr = Bytes.add(sepByteArr, Bytes.copy(maxIntBytes,
           sepByteArr.length, Bytes.SIZEOF_INT - sepByteArr.length));
       byte[] arr = separator.join(
-          Bytes.toBytes(separator.encode(str1)),longVal1Arr,
+          Bytes.toBytes(separator.encode(str1)), longVal1Arr,
           Bytes.toBytes(separator.encode(str2)), intVal1Arr);
-      int[] sizes = { Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
-          Separator.VARIABLE_SIZE, Bytes.SIZEOF_INT };
+      int[] sizes = {Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
+          Separator.VARIABLE_SIZE, Bytes.SIZEOF_INT};
       byte[][] splits = separator.split(arr, sizes);
       assertEquals(4, splits.length);
       assertEquals(str1, separator.decode(Bytes.toString(splits[0])));
@@ -116,7 +116,7 @@ public class TestSeparator {
           sepByteArr.length), sepByteArr);
       intVal1Arr = Bytes.add(Bytes.copy(maxIntBytes, 0, Bytes.SIZEOF_INT -
           sepByteArr.length), sepByteArr);
-      arr = separator.join(Bytes.toBytes(separator.encode(str1)),longVal1Arr,
+      arr = separator.join(Bytes.toBytes(separator.encode(str1)), longVal1Arr,
           Bytes.toBytes(separator.encode(str2)), intVal1Arr);
       splits = separator.split(arr, sizes);
       assertEquals(4, splits.length);
@@ -129,7 +129,7 @@ public class TestSeparator {
           sepByteArr.length, 4 - sepByteArr.length), sepByteArr);
       longVal1Arr = Bytes.add(longVal1Arr, Bytes.copy(maxLongBytes, 4, 3 -
               sepByteArr.length), sepByteArr);
-      arr = separator.join(Bytes.toBytes(separator.encode(str1)),longVal1Arr,
+      arr = separator.join(Bytes.toBytes(separator.encode(str1)), longVal1Arr,
           Bytes.toBytes(separator.encode(str2)), intVal1Arr);
       splits = separator.split(arr, sizes);
       assertEquals(4, splits.length);
@@ -140,8 +140,8 @@ public class TestSeparator {
 
       arr = separator.join(Bytes.toBytes(separator.encode(str1)),
           Bytes.toBytes(separator.encode(str2)), intVal1Arr, longVal1Arr);
-      int[] sizes1 = { Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
-          Bytes.SIZEOF_INT, Bytes.SIZEOF_LONG };
+      int[] sizes1 = {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+          Bytes.SIZEOF_INT, Bytes.SIZEOF_LONG};
       splits = separator.split(arr, sizes1);
       assertEquals(4, splits.length);
       assertEquals(str1, separator.decode(Bytes.toString(splits[0])));
@@ -150,15 +150,15 @@ public class TestSeparator {
       assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[3]));
 
       try {
-        int[] sizes2 = { Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
-            Bytes.SIZEOF_INT, 7 };
+        int[] sizes2 = {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+            Bytes.SIZEOF_INT, 7};
         splits = separator.split(arr, sizes2);
         fail("Exception should have been thrown.");
       } catch (IllegalArgumentException e) {}
 
       try {
-        int[] sizes2 = { Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, 2,
-            Bytes.SIZEOF_LONG };
+        int[] sizes2 = {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, 2,
+            Bytes.SIZEOF_LONG};
         splits = separator.split(arr, sizes2);
         fail("Exception should have been thrown.");
       } catch (IllegalArgumentException e) {}
@@ -196,12 +196,12 @@ public class TestSeparator {
     split = Separator.VALUES.splitEncoded(joined);
     assertTrue(Iterables.elementsEqual(stringList, split));
 
-    String[] stringArray1 = { "else" };
+    String[] stringArray1 = {"else"};
     joined = Separator.VALUES.joinEncoded(stringArray1);
     split = Separator.VALUES.splitEncoded(joined);
     assertTrue(Iterables.elementsEqual(Arrays.asList(stringArray1), split));
 
-    String[] stringArray2 = { "d", "e?", "f" };
+    String[] stringArray2 = {"d", "e?", "f"};
     joined = Separator.VALUES.joinEncoded(stringArray2);
     split = Separator.VALUES.splitEncoded(joined);
     assertTrue(Iterables.elementsEqual(Arrays.asList(stringArray2), split));

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md

@@ -1197,4 +1197,3 @@ container ID. Similarly, application attempt can be queried by specifying entity
 1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned.
 1. If flow context information cannot be retrieved or entity for the given entity id cannot be found, HTTP 404 (Not Found) is returned.
 1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
-