Explorar el Código

Fix javac, javadoc and checkstyle issues before branch-2 merge.

Varun Saxena hace 7 años
padre
commit
56e0873e22
Se han modificado 55 ficheros con 239 adiciones y 83 borrados
  1. 6 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
  2. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
  3. 8 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterConstants.java
  4. 7 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV1.java
  5. 9 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV2.java
  6. 3 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineMetric.java
  7. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineV2Client.java
  8. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineConnector.java
  9. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  10. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineServiceRecords.java
  11. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
  12. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java
  13. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java
  14. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/CollectorNodemanagerProtocolPB.java
  15. 3 0
      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
  16. 23 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/package-info.java
  17. 3 0
      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
  18. 23 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/package-info.java
  19. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/GetTimelineCollectorContextRequest.java
  20. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/GetTimelineCollectorContextResponse.java
  21. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewCollectorInfoRequest.java
  22. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewCollectorInfoResponse.java
  23. 4 0
      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
  24. 4 0
      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
  25. 4 0
      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
  26. 4 0
      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
  27. 5 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/package-info.java
  28. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/AppCollectorData.java
  29. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/AppCollectorDataPBImpl.java
  30. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
  31. 6 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
  32. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java
  33. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  34. 3 0
      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
  35. 11 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/authorize/NMPolicyProvider.java
  36. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java
  37. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
  38. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java
  39. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
  40. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java
  41. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
  42. 1 1
      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
  43. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java
  44. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderAuthenticationFilterInitializer.java
  45. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java
  46. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestNMTimelineCollectorManager.java
  47. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java
  48. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java
  49. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java
  50. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderUtils.java
  51. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
  52. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesUtils.java
  53. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineUIDConverter.java
  54. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java
  55. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java

+ 6 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java

@@ -141,7 +141,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaS
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
-import org.apache.hadoop.yarn.server.timelineservice.security.TimelineV2DelegationTokenSecretManagerService.TimelineV2DelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.ControlledClock;
@@ -776,7 +775,7 @@ public class TestRMContainerAllocator {
         new Text("renewer"), null);
         new Text("renewer"), null);
     ident.setSequenceNumber(1);
     ident.setSequenceNumber(1);
     Token<TimelineDelegationTokenIdentifier> collectorToken =
     Token<TimelineDelegationTokenIdentifier> collectorToken =
-        new Token<TimelineDelegationTokenIdentifier> (ident.getBytes(),
+        new Token<TimelineDelegationTokenIdentifier>(ident.getBytes(),
         new byte[0], TimelineDelegationTokenIdentifier.KIND_NAME,
         new byte[0], TimelineDelegationTokenIdentifier.KIND_NAME,
         new Text(localAddr));
         new Text(localAddr));
     org.apache.hadoop.yarn.api.records.Token token =
     org.apache.hadoop.yarn.api.records.Token token =
@@ -824,7 +823,7 @@ public class TestRMContainerAllocator {
     // new token.
     // new token.
     ident.setSequenceNumber(100);
     ident.setSequenceNumber(100);
     Token<TimelineDelegationTokenIdentifier> collectorToken1 =
     Token<TimelineDelegationTokenIdentifier> collectorToken1 =
-        new Token<TimelineDelegationTokenIdentifier> (ident.getBytes(),
+        new Token<TimelineDelegationTokenIdentifier>(ident.getBytes(),
         new byte[0], TimelineDelegationTokenIdentifier.KIND_NAME,
         new byte[0], TimelineDelegationTokenIdentifier.KIND_NAME,
         new Text(localAddr));
         new Text(localAddr));
     token = org.apache.hadoop.yarn.api.records.Token.newInstance(
     token = org.apache.hadoop.yarn.api.records.Token.newInstance(
@@ -3579,15 +3578,15 @@ public class TestRMContainerAllocator {
     }
     }
   }
   }
 
 
-  private static class MockSchedulerForTimelineCollector
+  private final static class MockSchedulerForTimelineCollector
       implements ApplicationMasterProtocol {
       implements ApplicationMasterProtocol {
-    CollectorInfo collectorInfo;
+    private CollectorInfo collectorInfo;
 
 
-    public MockSchedulerForTimelineCollector(CollectorInfo info) {
+    private MockSchedulerForTimelineCollector(CollectorInfo info) {
       this.collectorInfo = info;
       this.collectorInfo = info;
     }
     }
 
 
-    void updateCollectorInfo(CollectorInfo info) {
+    private void updateCollectorInfo(CollectorInfo info) {
       collectorInfo = info;
       collectorInfo = info;
     }
     }
 
 

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

@@ -62,7 +62,7 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
   private long startTime;
   private long startTime;
   private static final Counters EMPTY_COUNTERS = new Counters();
   private static final Counters EMPTY_COUNTERS = new Counters();
 
 
-  /** 
+  /**
    * Create an event to record the unsuccessful completion of attempts.
    * Create an event to record the unsuccessful completion of attempts.
    * @param id Attempt ID
    * @param id Attempt ID
    * @param taskType Type of the task
    * @param taskType Type of the task
@@ -232,7 +232,7 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
   public String getRackName() {
   public String getRackName() {
     return rackName == null ? null : rackName.toString();
     return rackName == null ? null : rackName.toString();
   }
   }
-  
+
   /** Gets the error string. */
   /** Gets the error string. */
   public String getError() { return error.toString(); }
   public String getError() { return error.toString(); }
   /**
   /**

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

@@ -21,20 +21,21 @@ package org.apache.hadoop.mapreduce;
 /**
 /**
  * Constants for simple entity writers.
  * Constants for simple entity writers.
  */
  */
-interface SimpleEntityWriterConstants {
+final class SimpleEntityWriterConstants {
+  private SimpleEntityWriterConstants() {}
   // constants for mtype = 1
   // constants for mtype = 1
-  String KBS_SENT = "kbs sent";
-  int KBS_SENT_DEFAULT = 1;
-  String TEST_TIMES = "testtimes";
-  int TEST_TIMES_DEFAULT = 100;
-  String TIMELINE_SERVICE_PERFORMANCE_RUN_ID =
+  static final String KBS_SENT = "kbs sent";
+  static final int KBS_SENT_DEFAULT = 1;
+  static final String TEST_TIMES = "testtimes";
+  static final int TEST_TIMES_DEFAULT = 100;
+  static final String TIMELINE_SERVICE_PERFORMANCE_RUN_ID =
       "timeline.server.performance.run.id";
       "timeline.server.performance.run.id";
 
 
   /**
   /**
    *  To ensure that the compression really gets exercised, generate a
    *  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',
+  static final char[] ALPHA_NUMS = new char[] {'a', 'b', 'c', 'd', 'e', 'f',
       'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o', 'p', 'q', 'r',
       'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o', 'p', 'q', 'r',
       's', 't', 'u', 'v', 'w', 'x', 'y', 'z', 'A', 'B', 'C', 'D',
       's', 't', 'u', 'v', 'w', 'x', 'y', 'z', 'A', 'B', 'C', 'D',
       'E', 'F', 'G', 'H', 'I', 'J', 'K', 'L', 'M', 'N', 'O', 'P',
       'E', 'F', 'G', 'H', 'I', 'J', 'K', 'L', 'M', 'N', 'O', 'P',

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

@@ -39,8 +39,7 @@ import org.apache.hadoop.yarn.client.api.TimelineClient;
    */
    */
 class SimpleEntityWriterV1
 class SimpleEntityWriterV1
     extends org.apache.hadoop.mapreduce.Mapper
     extends org.apache.hadoop.mapreduce.Mapper
-        <IntWritable, IntWritable, Writable, Writable>
-    implements SimpleEntityWriterConstants {
+        <IntWritable, IntWritable, Writable, Writable> {
   private static final Log LOG = LogFactory.getLog(SimpleEntityWriterV1.class);
   private static final Log LOG = LogFactory.getLog(SimpleEntityWriterV1.class);
 
 
   public void map(IntWritable key, IntWritable val, Context context)
   public void map(IntWritable key, IntWritable val, Context context)
@@ -48,10 +47,12 @@ class SimpleEntityWriterV1
     TimelineClient tlc = TimelineClient.createTimelineClient();
     TimelineClient tlc = TimelineClient.createTimelineClient();
     Configuration conf = context.getConfiguration();
     Configuration conf = context.getConfiguration();
 
 
-    final int kbs = conf.getInt(KBS_SENT, KBS_SENT_DEFAULT);
+    final int kbs = conf.getInt(SimpleEntityWriterConstants.KBS_SENT,
+        SimpleEntityWriterConstants.KBS_SENT_DEFAULT);
 
 
     long totalTime = 0;
     long totalTime = 0;
-    final int testtimes = conf.getInt(TEST_TIMES, TEST_TIMES_DEFAULT);
+    final int testtimes = conf.getInt(SimpleEntityWriterConstants.TEST_TIMES,
+        SimpleEntityWriterConstants.TEST_TIMES_DEFAULT);
     final Random rand = new Random();
     final Random rand = new Random();
     final TaskAttemptID taskAttemptId = context.getTaskAttemptID();
     final TaskAttemptID taskAttemptId = context.getTaskAttemptID();
     final char[] payLoad = new char[kbs * 1024];
     final char[] payLoad = new char[kbs * 1024];
@@ -60,8 +61,8 @@ class SimpleEntityWriterV1
       // Generate a fixed length random payload
       // Generate a fixed length random payload
       for (int xx = 0; xx < kbs * 1024; xx++) {
       for (int xx = 0; xx < kbs * 1024; xx++) {
         int alphaNumIdx =
         int alphaNumIdx =
-            rand.nextInt(ALPHA_NUMS.length);
-        payLoad[xx] = ALPHA_NUMS[alphaNumIdx];
+            rand.nextInt(SimpleEntityWriterConstants.ALPHA_NUMS.length);
+        payLoad[xx] = SimpleEntityWriterConstants.ALPHA_NUMS[alphaNumIdx];
       }
       }
       String entId = taskAttemptId + "_" + Integer.toString(i);
       String entId = taskAttemptId + "_" + Integer.toString(i);
       final TimelineEntity entity = new TimelineEntity();
       final TimelineEntity entity = new TimelineEntity();

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

@@ -40,8 +40,7 @@ import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollector
  * Adds simple entities with random string payload, events, metrics, and
  * Adds simple entities with random string payload, events, metrics, and
  * configuration.
  * configuration.
  */
  */
-class SimpleEntityWriterV2 extends EntityWriterV2
-    implements SimpleEntityWriterConstants {
+class SimpleEntityWriterV2 extends EntityWriterV2 {
   private static final Log LOG = LogFactory.getLog(SimpleEntityWriterV2.class);
   private static final Log LOG = LogFactory.getLog(SimpleEntityWriterV2.class);
 
 
   protected void writeEntities(Configuration tlConf,
   protected void writeEntities(Configuration tlConf,
@@ -49,7 +48,8 @@ class SimpleEntityWriterV2 extends EntityWriterV2
     Configuration conf = context.getConfiguration();
     Configuration conf = context.getConfiguration();
     // simulate the app id with the task id
     // simulate the app id with the task id
     int taskId = context.getTaskAttemptID().getTaskID().getId();
     int taskId = context.getTaskAttemptID().getTaskID().getId();
-    long timestamp = conf.getLong(TIMELINE_SERVICE_PERFORMANCE_RUN_ID, 0);
+    long timestamp = conf.getLong(
+        SimpleEntityWriterConstants.TIMELINE_SERVICE_PERFORMANCE_RUN_ID, 0);
     ApplicationId appId = ApplicationId.newInstance(timestamp, taskId);
     ApplicationId appId = ApplicationId.newInstance(timestamp, taskId);
 
 
     // create the app level timeline collector
     // create the app level timeline collector
@@ -66,10 +66,12 @@ class SimpleEntityWriterV2 extends EntityWriterV2
       tlContext.setFlowRunId(timestamp);
       tlContext.setFlowRunId(timestamp);
       tlContext.setUserId(context.getUser());
       tlContext.setUserId(context.getUser());
 
 
-      final int kbs = conf.getInt(KBS_SENT, KBS_SENT_DEFAULT);
+      final int kbs = conf.getInt(SimpleEntityWriterConstants.KBS_SENT,
+          SimpleEntityWriterConstants.KBS_SENT_DEFAULT);
 
 
       long totalTime = 0;
       long totalTime = 0;
-      final int testtimes = conf.getInt(TEST_TIMES, TEST_TIMES_DEFAULT);
+      final int testtimes = conf.getInt(SimpleEntityWriterConstants.TEST_TIMES,
+          SimpleEntityWriterConstants.TEST_TIMES_DEFAULT);
       final Random rand = new Random();
       final Random rand = new Random();
       final TaskAttemptID taskAttemptId = context.getTaskAttemptID();
       final TaskAttemptID taskAttemptId = context.getTaskAttemptID();
       final char[] payLoad = new char[kbs * 1024];
       final char[] payLoad = new char[kbs * 1024];
@@ -78,8 +80,8 @@ class SimpleEntityWriterV2 extends EntityWriterV2
         // Generate a fixed length random payload
         // Generate a fixed length random payload
         for (int xx = 0; xx < kbs * 1024; xx++) {
         for (int xx = 0; xx < kbs * 1024; xx++) {
           int alphaNumIdx =
           int alphaNumIdx =
-              rand.nextInt(ALPHA_NUMS.length);
-          payLoad[xx] = ALPHA_NUMS[alphaNumIdx];
+              rand.nextInt(SimpleEntityWriterConstants.ALPHA_NUMS.length);
+          payLoad[xx] = SimpleEntityWriterConstants.ALPHA_NUMS[alphaNumIdx];
         }
         }
         String entId = taskAttemptId + "_" + Integer.toString(i);
         String entId = taskAttemptId + "_" + Integer.toString(i);
         final TimelineEntity entity = new TimelineEntity();
         final TimelineEntity entity = new TimelineEntity();

+ 3 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineMetric.java

@@ -27,6 +27,9 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type;
 
 
 import org.junit.Test;
 import org.junit.Test;
 
 
+/**
+ * Tests {@link TimelineMetric} operations such as aggregation.
+ */
 public class TestTimelineMetric {
 public class TestTimelineMetric {
 
 
   @Test
   @Test
@@ -86,15 +89,4 @@ public class TestTimelineMetric {
     m.setValues(metricValues);
     m.setValues(metricValues);
     return m;
     return m;
   }
   }
-
-  private static TimelineMetric getTimeSeriesMetric(String id,
-      TimelineMetricOperation op, Map<Long, Number> metricValues) {
-    TimelineMetric m = new TimelineMetric();
-    m.setId(id);
-    m.setType(Type.TIME_SERIES);
-    m.setRealtimeAggregationOp(op);
-    m.setValues(metricValues);
-    return m;
-  }
-
 }
 }

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineV2Client.java

@@ -24,7 +24,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.CollectorInfo;
 import org.apache.hadoop.yarn.api.records.CollectorInfo;
-import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.client.api.impl.TimelineV2ClientImpl;
 import org.apache.hadoop.yarn.client.api.impl.TimelineV2ClientImpl;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnException;

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

@@ -131,8 +131,8 @@ public class TimelineConnector extends AbstractService {
     }
     }
   }
   }
 
 
-  private static final ConnectionConfigurator DEFAULT_TIMEOUT_CONN_CONFIGURATOR
-    = new ConnectionConfigurator() {
+  private static final ConnectionConfigurator
+      DEFAULT_TIMEOUT_CONN_CONFIGURATOR = new ConnectionConfigurator() {
         @Override
         @Override
         public HttpURLConnection configure(HttpURLConnection conn)
         public HttpURLConnection configure(HttpURLConnection conn)
             throws IOException {
             throws IOException {

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -3218,7 +3218,7 @@
     <name>yarn.app.attempt.diagnostics.limit.kc</name>
     <name>yarn.app.attempt.diagnostics.limit.kc</name>
     <value>64</value>
     <value>64</value>
   </property>
   </property>
-  
+
   <property>
   <property>
     <description>
     <description>
       Flag to enable cross-origin (CORS) support for timeline service v1.x or
       Flag to enable cross-origin (CORS) support for timeline service v1.x or

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

@@ -34,7 +34,9 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map;
 
 
-
+/**
+ * Tests timeline service API records.
+ */
 public class TestTimelineServiceRecords {
 public class TestTimelineServiceRecords {
   private static final Log LOG =
   private static final Log LOG =
       LogFactory.getLog(TestTimelineServiceRecords.class);
       LogFactory.getLog(TestTimelineServiceRecords.class);

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java

@@ -426,15 +426,14 @@ public class TestTimelineClient {
             new TimelineConnector(true, authUgi, doAsUser, token) {
             new TimelineConnector(true, authUgi, doAsUser, token) {
               @Override
               @Override
               public TimelineClientRetryOp
               public TimelineClientRetryOp
-                createRetryOpForOperateDelegationToken(
+                  createRetryOpForOperateDelegationToken(
                   final PrivilegedExceptionAction<?> action)
                   final PrivilegedExceptionAction<?> action)
                   throws IOException {
                   throws IOException {
                 TimelineClientRetryOpForOperateDelegationToken op =
                 TimelineClientRetryOpForOperateDelegationToken op =
                     spy(new TimelineClientRetryOpForOperateDelegationToken(
                     spy(new TimelineClientRetryOpForOperateDelegationToken(
                         UserGroupInformation.getCurrentUser(), action));
                         UserGroupInformation.getCurrentUser(), action));
-                doThrow(
-                    new SocketTimeoutException("Test socketTimeoutException"))
-                        .when(op).run();
+                doThrow(new SocketTimeoutException(
+                    "Test socketTimeoutException")).when(op).run();
                 return op;
                 return op;
               }
               }
             };
             };

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

@@ -49,6 +49,9 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.Test;
 import org.junit.rules.TestName;
 import org.junit.rules.TestName;
 
 
+/**
+ * Tests {@link TimelineV2ClientImpl}.
+ */
 public class TestTimelineClientV2Impl {
 public class TestTimelineClientV2Impl {
   private static final Log LOG =
   private static final Log LOG =
       LogFactory.getLog(TestTimelineClientV2Impl.class);
       LogFactory.getLog(TestTimelineClientV2Impl.class);

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java

@@ -26,6 +26,9 @@ import java.util.TreeMap;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
 
 
+/**
+ * Tests utility methods in {@link TimelineServiceHelper}.
+ */
 public class TestTimelineServiceHelper {
 public class TestTimelineServiceHelper {
 
 
   @Test
   @Test

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

@@ -22,6 +22,9 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.ipc.ProtocolInfo;
 import org.apache.hadoop.ipc.ProtocolInfo;
 import org.apache.hadoop.yarn.proto.CollectorNodemanagerProtocol.CollectorNodemanagerProtocolService;
 import org.apache.hadoop.yarn.proto.CollectorNodemanagerProtocol.CollectorNodemanagerProtocolService;
 
 
+/**
+ * Collector nodemanager protocol service implementation.
+ */
 @Private
 @Private
 @Unstable
 @Unstable
 @ProtocolInfo(
 @ProtocolInfo(

+ 3 - 0
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

@@ -45,6 +45,9 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewCollec
 
 
 import com.google.protobuf.ServiceException;
 import com.google.protobuf.ServiceException;
 
 
+/**
+ * Client implementation of {@link CollectorNodemanagerProtocol}.
+ */
 public class CollectorNodemanagerProtocolPBClientImpl implements
 public class CollectorNodemanagerProtocolPBClientImpl implements
     CollectorNodemanagerProtocol, Closeable {
     CollectorNodemanagerProtocol, Closeable {
 
 

+ 23 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/package-info.java

@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Contains protocol client implementations used to communicate with different
+ * YARN server components.
+ */
+package org.apache.hadoop.yarn.server.api.impl.pb.client;

+ 3 - 0
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

@@ -36,6 +36,9 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewCollec
 import com.google.protobuf.RpcController;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 import com.google.protobuf.ServiceException;
 
 
+/**
+ * Service implementation of CollectorNodemanagerProtocol.
+ */
 public class CollectorNodemanagerProtocolPBServiceImpl implements
 public class CollectorNodemanagerProtocolPBServiceImpl implements
     CollectorNodemanagerProtocolPB {
     CollectorNodemanagerProtocolPB {
 
 

+ 23 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/package-info.java

@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Contains interfaces which define the protocols to communicate with different
+ * YARN server components.
+ */
+package org.apache.hadoop.yarn.server.api;

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/GetTimelineCollectorContextRequest.java

@@ -21,6 +21,11 @@ package org.apache.hadoop.yarn.server.api.protocolrecords;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.Records;
 
 
+/**
+ * GetTimelineCollectorContextRequest invoked by collector to get the context
+ * information from Nodemanager. The context information includes user, flow and
+ * flow run ID and is associated with every incoming put-entity request.
+ */
 public abstract class GetTimelineCollectorContextRequest {
 public abstract class GetTimelineCollectorContextRequest {
 
 
   public static GetTimelineCollectorContextRequest newInstance(
   public static GetTimelineCollectorContextRequest newInstance(

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

@@ -20,6 +20,12 @@ package org.apache.hadoop.yarn.server.api.protocolrecords;
 
 
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.Records;
 
 
+/**
+ * GetTimelineCollectorContextResponse sent in response of
+ * GetTimelineCollectorContextRequest invoked by collector. The response
+ * includes user, flow name, flow version and flow run ID. This is then
+ * associated by collector with every incoming put-entity request.
+ */
 public abstract class GetTimelineCollectorContextResponse {
 public abstract class GetTimelineCollectorContextResponse {
 
 
   public static GetTimelineCollectorContextResponse newInstance(
   public static GetTimelineCollectorContextResponse newInstance(

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

@@ -26,6 +26,12 @@ import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
 import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.Records;
 
 
+/**
+ * Request used to report new collector info from timeline collector manager
+ * whenever a collector is launched.
+ * The request contains a list of collector data, with each collector data
+ * object represented by {@link AppCollectorData}.
+ */
 @Private
 @Private
 public abstract class ReportNewCollectorInfoRequest {
 public abstract class ReportNewCollectorInfoRequest {
 
 

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewCollectorInfoResponse.java

@@ -20,6 +20,9 @@ package org.apache.hadoop.yarn.server.api.protocolrecords;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.Records;
 
 
+/**
+ * Response to {@link ReportNewCollectorInfoRequest}.
+ */
 public abstract class ReportNewCollectorInfoResponse {
 public abstract class ReportNewCollectorInfoResponse {
 
 
   @Private
   @Private

+ 4 - 0
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

@@ -26,6 +26,10 @@ import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.GetTimelineCol
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.GetTimelineCollectorContextRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.GetTimelineCollectorContextRequestProto;
 import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest;
 
 
+/**
+ * Protocol buffer based implementation of
+ * {@link GetTimelineCollectorContextRequest}.
+ */
 public class GetTimelineCollectorContextRequestPBImpl extends
 public class GetTimelineCollectorContextRequestPBImpl extends
     GetTimelineCollectorContextRequest {
     GetTimelineCollectorContextRequest {
 
 

+ 4 - 0
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

@@ -23,6 +23,10 @@ import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.GetTimelineCol
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.GetTimelineCollectorContextResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.GetTimelineCollectorContextResponseProto;
 import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse;
 
 
+/**
+ * Protocol buffer based implementation of
+ * {@link GetTimelineCollectorContextResponse}.
+ */
 public class GetTimelineCollectorContextResponsePBImpl extends
 public class GetTimelineCollectorContextResponsePBImpl extends
     GetTimelineCollectorContextResponse {
     GetTimelineCollectorContextResponse {
 
 

+ 4 - 0
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

@@ -27,6 +27,10 @@ import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewColle
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoRequestProtoOrBuilder;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoRequestProtoOrBuilder;
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
 
 
+/**
+ * Protocol buffer based implementation of
+ * {@link ReportNewCollectorInfoRequest}.
+ */
 public class ReportNewCollectorInfoRequestPBImpl extends
 public class ReportNewCollectorInfoRequestPBImpl extends
     ReportNewCollectorInfoRequest {
     ReportNewCollectorInfoRequest {
 
 

+ 4 - 0
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

@@ -24,6 +24,10 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoR
 
 
 import com.google.protobuf.TextFormat;
 import com.google.protobuf.TextFormat;
 
 
+/**
+ * Protocol buffer based implementation of
+ * {@link ReportNewCollectorInfoResponse}.
+ */
 @Private
 @Private
 @Unstable
 @Unstable
 public class ReportNewCollectorInfoResponsePBImpl extends
 public class ReportNewCollectorInfoResponsePBImpl extends

+ 5 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestAppLevelTimelineCollector.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/package-info.java

@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
  * distributed with this work for additional information
@@ -16,8 +16,7 @@
  * limitations under the License.
  * limitations under the License.
  */
  */
 
 
-package org.apache.hadoop.yarn.server.timelineservice.collector;
-
-
-public class TestAppLevelTimelineCollector {
-}
+/**
+ * Contains protocol record implementations for YARN server protocols.
+ */
+package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/AppCollectorData.java

@@ -24,7 +24,10 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.Records;
 
 
-
+/**
+ * Collector data associated with an app collector. Includes app Id, collector
+ * address, RM identifier, version and collector token.
+ */
 @Private
 @Private
 @InterfaceStability.Unstable
 @InterfaceStability.Unstable
 public abstract class AppCollectorData {
 public abstract class AppCollectorData {

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/AppCollectorDataPBImpl.java

@@ -30,6 +30,9 @@ import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDa
 
 
 import com.google.protobuf.TextFormat;
 import com.google.protobuf.TextFormat;
 
 
+/**
+ * Protocol Buffer based implementation of {@link AppCollectorData}.
+ */
 @Private
 @Private
 @Unstable
 @Unstable
 public class AppCollectorDataPBImpl extends AppCollectorData {
 public class AppCollectorDataPBImpl extends AppCollectorData {

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java

@@ -53,7 +53,8 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
   /**
   /**
    * The configuration prefix of timeline HTTP authentication.
    * The configuration prefix of timeline HTTP authentication.
    */
    */
-  public static final String PREFIX = "yarn.timeline-service.http-authentication.";
+  public static final String PREFIX =
+      "yarn.timeline-service.http-authentication.";
 
 
   @VisibleForTesting
   @VisibleForTesting
   Map<String, String> filterConfig;
   Map<String, String> filterConfig;
@@ -104,7 +105,7 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
   }
   }
 
 
   /**
   /**
-   * Initializes {@link TimelineAuthenticationFilter}
+   * Initializes {@link TimelineAuthenticationFilter}.
    * <p>
    * <p>
    * Propagates to {@link TimelineAuthenticationFilter} configuration all YARN
    * Propagates to {@link TimelineAuthenticationFilter} configuration all YARN
    * configuration properties prefixed with {@value #PREFIX}
    * configuration properties prefixed with {@value #PREFIX}

+ 6 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java

@@ -84,6 +84,9 @@ import org.apache.hadoop.yarn.util.Records;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
 
 
+/**
+ * Tests RPC communication for different YARN server protocols.
+ */
 public class TestRPC {
 public class TestRPC {
 
 
   private static final String EXCEPTION_MSG = "test error";
   private static final String EXCEPTION_MSG = "test error";
@@ -320,7 +323,8 @@ public class TestRPC {
     Assert.assertEquals(ContainerState.RUNNING, statuses.get(0).getState());
     Assert.assertEquals(ContainerState.RUNNING, statuses.get(0).getState());
   }
   }
 
 
-  public class DummyContainerManager implements ContainerManagementProtocol {
+  private static class DummyContainerManager
+      implements ContainerManagementProtocol {
 
 
     private List<ContainerStatus> statuses = new ArrayList<ContainerStatus>();
     private List<ContainerStatus> statuses = new ArrayList<ContainerStatus>();
 
 
@@ -448,7 +452,7 @@ public class TestRPC {
 
 
   // A dummy implementation for CollectorNodemanagerProtocol for test purpose,
   // A dummy implementation for CollectorNodemanagerProtocol for test purpose,
   // it only can accept one appID, collectorAddr pair or throw exceptions
   // it only can accept one appID, collectorAddr pair or throw exceptions
-  public class DummyNMCollectorService
+  private static class DummyNMCollectorService
       implements CollectorNodemanagerProtocol {
       implements CollectorNodemanagerProtocol {
 
 
     @Override
     @Override

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java

@@ -27,7 +27,9 @@ import static org.apache.hadoop.yarn.server.timeline.security.TimelineAuthentica
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 
 
-
+/**
+ * Tests {@link TimelineAuthenticationFilterInitializer}.
+ */
 public class TestTimelineAuthenticationFilterInitializer {
 public class TestTimelineAuthenticationFilterInitializer {
 
 
   @Test
   @Test

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

@@ -429,6 +429,7 @@ public class ContainerManagerImpl extends CompositeService implements
     app.handle(new ApplicationInitEvent(appId, acls, logAggregationContext));
     app.handle(new ApplicationInitEvent(appId, acls, logAggregationContext));
   }
   }
 
 
+  @SuppressWarnings("unchecked")
   private void recoverContainer(RecoveredContainerState rcs)
   private void recoverContainer(RecoveredContainerState rcs)
       throws IOException {
       throws IOException {
     StartContainerRequest req = rcs.getStartRequest();
     StartContainerRequest req = rcs.getStartRequest();
@@ -1735,6 +1736,7 @@ public class ContainerManagerImpl extends CompositeService implements
    * @param reInitLaunchContext Target Launch Context.
    * @param reInitLaunchContext Target Launch Context.
    * @throws YarnException Yarn Exception.
    * @throws YarnException Yarn Exception.
    */
    */
+  @SuppressWarnings("unchecked")
   public void reInitializeContainer(ContainerId containerId,
   public void reInitializeContainer(ContainerId containerId,
       ContainerLaunchContext reInitLaunchContext, boolean autoCommit)
       ContainerLaunchContext reInitLaunchContext, boolean autoCommit)
       throws YarnException {
       throws YarnException {
@@ -1762,6 +1764,7 @@ public class ContainerManagerImpl extends CompositeService implements
    * @return Rollback Response.
    * @return Rollback Response.
    * @throws YarnException Yarn Exception.
    * @throws YarnException Yarn Exception.
    */
    */
+  @SuppressWarnings("unchecked")
   @Override
   @Override
   public RollbackResponse rollbackLastReInitialization(ContainerId containerId)
   public RollbackResponse rollbackLastReInitialization(ContainerId containerId)
       throws YarnException {
       throws YarnException {

+ 3 - 0
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

@@ -85,6 +85,9 @@ public class ContainersMonitorImpl extends AbstractService implements
   private static final long UNKNOWN_MEMORY_LIMIT = -1L;
   private static final long UNKNOWN_MEMORY_LIMIT = -1L;
   private int nodeCpuPercentageForYARN;
   private int nodeCpuPercentageForYARN;
 
 
+  /**
+   * Identifies the type of container metric to be published.
+   */
   @Private
   @Private
   public static enum ContainerMetric {
   public static enum ContainerMetric {
     CPU, MEMORY
     CPU, MEMORY

+ 11 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/authorize/NMPolicyProvider.java

@@ -33,21 +33,22 @@ import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocolPB;
 @InterfaceStability.Unstable
 @InterfaceStability.Unstable
 public class NMPolicyProvider extends PolicyProvider {
 public class NMPolicyProvider extends PolicyProvider {
   
   
-  private static final Service[] nodeManagerServices = 
+  private static final Service[] NODE_MANAGER_SERVICES =
       new Service[] {
       new Service[] {
-    new Service(
-        YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGEMENT_PROTOCOL, 
-        ContainerManagementProtocolPB.class),
-    new Service(YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER, 
-        LocalizationProtocolPB.class),
-    new Service(YarnConfiguration.
-        YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL,
+          new Service(YarnConfiguration.
+            YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGEMENT_PROTOCOL,
+            ContainerManagementProtocolPB.class),
+          new Service(YarnConfiguration.
+            YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER,
+            LocalizationProtocolPB.class),
+          new Service(YarnConfiguration.
+            YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL,
             CollectorNodemanagerProtocolPB.class)
             CollectorNodemanagerProtocolPB.class)
-  };
+      };
 
 
   @Override
   @Override
   public Service[] getServices() {
   public Service[] getServices() {
-    return nodeManagerServices;
+    return NODE_MANAGER_SERVICES;
   }
   }
 
 
 }
 }

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

@@ -18,6 +18,9 @@
 
 
 package org.apache.hadoop.yarn.server.nodemanager.timelineservice;
 package org.apache.hadoop.yarn.server.nodemanager.timelineservice;
 
 
+/**
+ * Tests publishing of entities from NM to ATSv2.
+ */
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotNull;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mock;
@@ -43,6 +46,9 @@ import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
 
 
+/**
+ * Tests {@link NMTimelinePublisher}.
+ */
 public class TestNMTimelinePublisher {
 public class TestNMTimelinePublisher {
   private static final String MEMORY_ID = "MEMORY";
   private static final String MEMORY_ID = "MEMORY";
   private static final String CPU_ID = "CPU";
   private static final String CPU_ID = "CPU";

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

@@ -45,7 +45,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.records.CollectorInfo;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;

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

@@ -55,7 +55,6 @@ import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt

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

@@ -75,6 +75,9 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.Test;
 
 
+/**
+ * Tests publishing of entities and metrics from RM to ATSv2.
+ */
 public class TestSystemMetricsPublisherForV2 {
 public class TestSystemMetricsPublisherForV2 {
 
 
   /**
   /**

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java

@@ -67,13 +67,13 @@ public class TestRMNMSecretKeys {
   }
   }
 
 
   private static void writeFile(String content, File file) throws IOException {
   private static void writeFile(String content, File file) throws IOException {
-      FileOutputStream outputStream = new FileOutputStream(file);
-      FileChannel fc = outputStream.getChannel();
+    FileOutputStream outputStream = new FileOutputStream(file);
+    FileChannel fc = outputStream.getChannel();
 
 
-      ByteBuffer buffer =
-          ByteBuffer.wrap(content.getBytes(StandardCharsets.UTF_8));
-      fc.write(buffer);
-      outputStream.close();
+    ByteBuffer buffer =
+        ByteBuffer.wrap(content.getBytes(StandardCharsets.UTF_8));
+    fc.write(buffer);
+    outputStream.close();
   }
   }
 
 
   @AfterClass
   @AfterClass

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

@@ -58,6 +58,9 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.Test;
 
 
+/**
+ * Tests timeline client integration with collector service.
+ */
 public class TestTimelineServiceClientIntegration {
 public class TestTimelineServiceClientIntegration {
   private static final String ROOT_DIR = new File("target",
   private static final String ROOT_DIR = new File("target",
       TestTimelineServiceClientIntegration.class.getSimpleName()).
       TestTimelineServiceClientIntegration.class.getSimpleName()).

+ 1 - 1
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

@@ -64,7 +64,7 @@ import com.sun.jersey.api.client.ClientResponse.Status;
 import com.sun.jersey.api.client.GenericType;
 import com.sun.jersey.api.client.GenericType;
 
 
 /**
 /**
- * Test TimelineReder Web Service REST API's.
+ * Test TimelineReder Web Service REST API's using HBase storage.
  */
  */
 public class TestTimelineReaderWebServicesHBaseStorage
 public class TestTimelineReaderWebServicesHBaseStorage
     extends AbstractTimelineReaderHBaseTestBase {
     extends AbstractTimelineReaderHBaseTestBase {

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

@@ -30,6 +30,10 @@ import org.junit.Test;
 
 
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterables;
 
 
+/**
+ * Tests the encoding and decoding of separators. Typically used to separate
+ * different segments in HBase row key.
+ */
 public class TestSeparator {
 public class TestSeparator {
 
 
   private static String villain = "Dr. Heinz Doofenshmirtz";
   private static String villain = "Dr. Heinz Doofenshmirtz";

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderAuthenticationFilterInitializer.java

@@ -36,7 +36,7 @@ public class TimelineReaderAuthenticationFilterInitializer extends
    * <p>
    * <p>
    * Propagates to {@link AuthenticationWithProxyUserFilter} configuration all
    * Propagates to {@link AuthenticationWithProxyUserFilter} configuration all
    * YARN configuration properties prefixed with
    * YARN configuration properties prefixed with
-   * {@value TimelineAuthenticationFilterInitializer#PREFIX}.
+   * {@link TimelineAuthenticationFilterInitializer#PREFIX}.
    *
    *
    * @param container
    * @param container
    *          The filter container
    *          The filter container

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

@@ -125,7 +125,7 @@ public interface TimelineReader extends Service {
    *    <li><b>flowRunId</b> - Context flow run id.</li>
    *    <li><b>flowRunId</b> - Context flow run id.</li>
    *    <li><b>appId</b> - Context app id.</li>
    *    <li><b>appId</b> - Context app id.</li>
    *    </ul>
    *    </ul>
-   *    Although entityIdPrefix and entityId are also part of context, 
+   *    Although entityIdPrefix and entityId are also part of context,
    *    it has no meaning for getEntities.<br>
    *    it has no meaning for getEntities.<br>
    *    Fields in context which are mandatory depends on entity type. Entity
    *    Fields in context which are mandatory depends on entity type. Entity
    *    type is always mandatory. In addition to entity type, below is the list
    *    type is always mandatory. In addition to entity type, below is the list
@@ -161,7 +161,7 @@ public interface TimelineReader extends Service {
    *    {@link TimelineDataToRetrieve} for details.
    *    {@link TimelineDataToRetrieve} for details.
    * @return A set of <cite>TimelineEntity</cite> instances of the given entity
    * @return A set of <cite>TimelineEntity</cite> instances of the given entity
    *    type in the given context scope which matches the given predicates
    *    type in the given context scope which matches the given predicates
-   *    ordered by enitityIdPrefix(for generic entities only). 
+   *    ordered by enitityIdPrefix(for generic entities only).
    *    Each entity will only contain
    *    Each entity will only contain
    *    the metadata(id, type , idPrefix and created time) plus the given
    *    the metadata(id, type , idPrefix and created time) plus the given
    *    fields to retrieve.
    *    fields to retrieve.

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

@@ -51,6 +51,9 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
+/**
+ * Tests node level timeline collector manager.
+ */
 public class TestNMTimelineCollectorManager {
 public class TestNMTimelineCollectorManager {
   private NodeTimelineCollectorManager collectorManager;
   private NodeTimelineCollectorManager collectorManager;
 
 

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

@@ -49,6 +49,10 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Test;
 import org.junit.Test;
 
 
+/**
+ * Tests the NodeManager auxiliary service started for node level timeline
+ * collector.
+ */
 public class TestPerNodeTimelineCollectorsAuxService {
 public class TestPerNodeTimelineCollectorsAuxService {
   private ApplicationAttemptId appAttemptId;
   private ApplicationAttemptId appAttemptId;
   private PerNodeTimelineCollectorsAuxService auxService;
   private PerNodeTimelineCollectorsAuxService auxService;

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java

@@ -46,6 +46,9 @@ import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verify;
 
 
+/**
+ * Tests functionality of timeline collector.
+ */
 public class TestTimelineCollector {
 public class TestTimelineCollector {
 
 
   private TimelineEntities generateTestEntities(int groups, int entities) {
   private TimelineEntities generateTestEntities(int groups, int entities) {

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

@@ -28,6 +28,9 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineR
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
 import org.junit.Test;
 import org.junit.Test;
 
 
+/**
+ * Tests {@link TimelineReaderServer} start and stop.
+ */
 public class TestTimelineReaderServer {
 public class TestTimelineReaderServer {
 
 
   @Test(timeout = 60000)
   @Test(timeout = 60000)

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

@@ -26,6 +26,9 @@ import java.util.List;
 
 
 import org.junit.Test;
 import org.junit.Test;
 
 
+/**
+ * Tests helper methods in {@link TimelineReaderUtils}.
+ */
 public class TestTimelineReaderUtils {
 public class TestTimelineReaderUtils {
 
 
   @Test
   @Test

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

@@ -59,6 +59,9 @@ import com.sun.jersey.api.client.config.DefaultClientConfig;
 import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
 import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
 import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
 import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
 
 
+/**
+ * Test TimelineReder Web Service REST API's using filesystem storage.
+ */
 public class TestTimelineReaderWebServices {
 public class TestTimelineReaderWebServices {
 
 
   private static final String ROOT_DIR = new File("target",
   private static final String ROOT_DIR = new File("target",

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

@@ -35,6 +35,9 @@ import org.junit.Test;
 
 
 import com.google.common.collect.Sets;
 import com.google.common.collect.Sets;
 
 
+/**
+ * Tests helper methods in {@link TimelineReaderWebServicesUtils}.
+ */
 public class TestTimelineReaderWebServicesUtils {
 public class TestTimelineReaderWebServicesUtils {
   private static void verifyFilterList(String expr, TimelineFilterList list,
   private static void verifyFilterList(String expr, TimelineFilterList list,
       TimelineFilterList expectedList) throws Exception {
       TimelineFilterList expectedList) throws Exception {

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

@@ -24,6 +24,9 @@ import static org.junit.Assert.fail;
 
 
 import org.junit.Test;
 import org.junit.Test;
 
 
+/**
+ * Tests Timeline UIDs' encoding and decoding.
+ */
 public class TestTimelineUIDConverter {
 public class TestTimelineUIDConverter {
 
 
   @Test
   @Test

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

@@ -59,6 +59,9 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.Test;
 
 
+/**
+ * Tests filesystem implemetation for timelineservice reader.
+ */
 public class TestFileSystemTimelineReaderImpl {
 public class TestFileSystemTimelineReaderImpl {
 
 
   private static final String ROOT_DIR = new File("target",
   private static final String ROOT_DIR = new File("target",

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

@@ -42,6 +42,9 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TemporaryFolder;
 
 
+/**
+ * Tests filesystem implemetation for timelineservice writer.
+ */
 public class TestFileSystemTimelineWriterImpl {
 public class TestFileSystemTimelineWriterImpl {
   @Rule
   @Rule
   public TemporaryFolder tmpFolder = new TemporaryFolder();
   public TemporaryFolder tmpFolder = new TemporaryFolder();