Browse Source

AMBARI-10622. Add daily aggregation to AMS (useful for reporting over months of data). (swagle)

Siddharth Wagle 10 years ago
parent
commit
67c425acfd
22 changed files with 1065 additions and 737 deletions
  1. 1 0
      ambari-metrics/ambari-metrics-timelineservice/pom.xml
  2. 26 14
      ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/HBaseTimelineMetricStore.java
  3. 70 74
      ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/PhoenixHBaseAccessor.java
  4. 4 2
      ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/Precision.java
  5. 27 2
      ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricConfiguration.java
  6. 46 12
      ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/AbstractTimelineAggregator.java
  7. 0 42
      ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineClusterMetricReader.java
  8. 19 132
      ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineMetricAggregator.java
  9. 179 9
      ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineMetricAggregatorFactory.java
  10. 68 167
      ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineMetricClusterAggregator.java
  11. 0 175
      ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineMetricClusterAggregatorHourly.java
  12. 201 0
      ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineMetricClusterAggregatorMinute.java
  13. 113 0
      ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineMetricHostAggregator.java
  14. 36 0
      ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineMetricReadHelper.java
  15. 44 38
      ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/query/PhoenixTransactSQL.java
  16. 3 2
      ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/AbstractMiniHBaseClusterTest.java
  17. 74 25
      ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/ITClusterAggregator.java
  18. 70 12
      ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/ITMetricAggregator.java
  19. 14 20
      ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/ITPhoenixHBaseAccessor.java
  20. 2 5
      ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/MetricTestHelper.java
  21. 2 2
      ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TestMetricHostAggregate.java
  22. 66 4
      ambari-server/src/main/resources/common-services/AMBARI_METRICS/0.1.0/configuration/ams-site.xml

+ 1 - 0
ambari-metrics/ambari-metrics-timelineservice/pom.xml

@@ -249,6 +249,7 @@
         <configuration>
         <configuration>
           <redirectTestOutputToFile>true</redirectTestOutputToFile>
           <redirectTestOutputToFile>true</redirectTestOutputToFile>
           <forkMode>always</forkMode>
           <forkMode>always</forkMode>
+          <argLine>-XX:-UseSplitVerifier</argLine>
         </configuration>
         </configuration>
       </plugin>
       </plugin>
     </plugins>
     </plugins>

+ 26 - 14
ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/HBaseTimelineMetricStore.java

@@ -28,8 +28,6 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.Function;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.Function;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricAggregator;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricAggregator;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricAggregatorFactory;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricAggregatorFactory;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricClusterAggregator;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricClusterAggregatorHourly;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.Condition;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.Condition;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.DefaultCondition;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.DefaultCondition;
 
 
@@ -69,39 +67,53 @@ public class HBaseTimelineMetricStore extends AbstractService
     hBaseAccessor = new PhoenixHBaseAccessor(hbaseConf, metricsConf);
     hBaseAccessor = new PhoenixHBaseAccessor(hbaseConf, metricsConf);
     hBaseAccessor.initMetricSchema();
     hBaseAccessor.initMetricSchema();
 
 
-    // Start the cluster aggregator
-    TimelineMetricClusterAggregator minuteClusterAggregator =
-      new TimelineMetricClusterAggregator(hBaseAccessor, metricsConf);
+    // Start the cluster aggregator minute
+    TimelineMetricAggregator minuteClusterAggregator =
+      TimelineMetricAggregatorFactory.createTimelineClusterAggregatorMinute(hBaseAccessor, metricsConf);
     if (!minuteClusterAggregator.isDisabled()) {
     if (!minuteClusterAggregator.isDisabled()) {
       Thread aggregatorThread = new Thread(minuteClusterAggregator);
       Thread aggregatorThread = new Thread(minuteClusterAggregator);
       aggregatorThread.start();
       aggregatorThread.start();
     }
     }
 
 
-    // Start the cluster aggregator hourly
-    TimelineMetricClusterAggregatorHourly hourlyClusterAggregator =
-      new TimelineMetricClusterAggregatorHourly(hBaseAccessor, metricsConf);
+    // Start the hourly cluster aggregator
+    TimelineMetricAggregator hourlyClusterAggregator =
+      TimelineMetricAggregatorFactory.createTimelineClusterAggregatorHourly(hBaseAccessor, metricsConf);
     if (!hourlyClusterAggregator.isDisabled()) {
     if (!hourlyClusterAggregator.isDisabled()) {
       Thread aggregatorThread = new Thread(hourlyClusterAggregator);
       Thread aggregatorThread = new Thread(hourlyClusterAggregator);
       aggregatorThread.start();
       aggregatorThread.start();
     }
     }
 
 
-    // Start the 5 minute aggregator
+    // Start the daily cluster aggregator
+    TimelineMetricAggregator dailyClusterAggregator =
+      TimelineMetricAggregatorFactory.createTimelineClusterAggregatorDaily(hBaseAccessor, metricsConf);
+    if (!dailyClusterAggregator.isDisabled()) {
+      Thread aggregatorThread = new Thread(dailyClusterAggregator);
+      aggregatorThread.start();
+    }
+
+    // Start the minute host aggregator
     TimelineMetricAggregator minuteHostAggregator =
     TimelineMetricAggregator minuteHostAggregator =
-      TimelineMetricAggregatorFactory.createTimelineMetricAggregatorMinute
-        (hBaseAccessor, metricsConf);
+      TimelineMetricAggregatorFactory.createTimelineMetricAggregatorMinute(hBaseAccessor, metricsConf);
     if (!minuteHostAggregator.isDisabled()) {
     if (!minuteHostAggregator.isDisabled()) {
       Thread minuteAggregatorThread = new Thread(minuteHostAggregator);
       Thread minuteAggregatorThread = new Thread(minuteHostAggregator);
       minuteAggregatorThread.start();
       minuteAggregatorThread.start();
     }
     }
 
 
-    // Start hourly host aggregator
+    // Start the hourly host aggregator
     TimelineMetricAggregator hourlyHostAggregator =
     TimelineMetricAggregator hourlyHostAggregator =
-      TimelineMetricAggregatorFactory.createTimelineMetricAggregatorHourly
-        (hBaseAccessor, metricsConf);
+      TimelineMetricAggregatorFactory.createTimelineMetricAggregatorHourly(hBaseAccessor, metricsConf);
     if (!hourlyHostAggregator.isDisabled()) {
     if (!hourlyHostAggregator.isDisabled()) {
       Thread aggregatorHourlyThread = new Thread(hourlyHostAggregator);
       Thread aggregatorHourlyThread = new Thread(hourlyHostAggregator);
       aggregatorHourlyThread.start();
       aggregatorHourlyThread.start();
     }
     }
+
+    // Start the daily host aggregator
+    TimelineMetricAggregator dailyHostAggregator =
+      TimelineMetricAggregatorFactory.createTimelineMetricAggregatorDaily(hBaseAccessor, metricsConf);
+    if (!dailyHostAggregator.isDisabled()) {
+      Thread aggregatorDailyThread = new Thread(dailyHostAggregator);
+      aggregatorDailyThread.start();
+    }
   }
   }
 
 
   @Override
   @Override

+ 70 - 74
ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/PhoenixHBaseAccessor.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.type.TypeReference;
 import org.codehaus.jackson.type.TypeReference;
+
 import java.io.IOException;
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.PreparedStatement;
@@ -51,7 +52,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
+
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_DAILY_TABLE_TTL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_HOUR_TABLE_TTL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_HOUR_TABLE_TTL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_MINUTE_TABLE_TTL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_MINUTE_TABLE_TTL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.GLOBAL_MAX_RETRIES;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.GLOBAL_MAX_RETRIES;
@@ -59,19 +62,21 @@ import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.ti
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.GLOBAL_RETRY_INTERVAL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.GLOBAL_RETRY_INTERVAL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HBASE_COMPRESSION_SCHEME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HBASE_COMPRESSION_SCHEME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HBASE_ENCODING_SCHEME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HBASE_ENCODING_SCHEME;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_DAILY_TABLE_TTL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_HOUR_TABLE_TTL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_HOUR_TABLE_TTL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_MINUTE_TABLE_TTL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_MINUTE_TABLE_TTL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.PRECISION_TABLE_TTL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.PRECISION_TABLE_TTL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.ALTER_SQL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.ALTER_SQL;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.CREATE_METRICS_AGGREGATE_HOURLY_TABLE_SQL;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.CREATE_METRICS_AGGREGATE_MINUTE_TABLE_SQL;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.CREATE_METRICS_AGGREGATE_TABLE_SQL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.CREATE_METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_SQL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.CREATE_METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_SQL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.CREATE_METRICS_CLUSTER_AGGREGATE_TABLE_SQL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.CREATE_METRICS_CLUSTER_AGGREGATE_TABLE_SQL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.CREATE_METRICS_TABLE_SQL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.CREATE_METRICS_TABLE_SQL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.DEFAULT_ENCODING;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.DEFAULT_ENCODING;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.DEFAULT_TABLE_COMPRESSION;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.DEFAULT_TABLE_COMPRESSION;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_DAILY_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_HOURLY_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_HOURLY_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_MINUTE_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_MINUTE_TABLE_NAME;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_DAILY_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_RECORD_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_RECORD_TABLE_NAME;
@@ -85,13 +90,9 @@ import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.ti
  */
  */
 public class PhoenixHBaseAccessor {
 public class PhoenixHBaseAccessor {
 
 
+  static final int PHOENIX_MAX_MUTATION_STATE_SIZE = 50000;
   private static final Log LOG = LogFactory.getLog(PhoenixHBaseAccessor.class);
   private static final Log LOG = LogFactory.getLog(PhoenixHBaseAccessor.class);
   private static final TimelineMetricReadHelper TIMELINE_METRIC_READ_HELPER = new TimelineMetricReadHelper();
   private static final TimelineMetricReadHelper TIMELINE_METRIC_READ_HELPER = new TimelineMetricReadHelper();
-  private final Configuration hbaseConf;
-  private final Configuration metricsConf;
-  private final RetryCounterFactory retryCounterFactory;
-
-  static final int PHOENIX_MAX_MUTATION_STATE_SIZE = 50000;
   /**
   /**
    * 4 metrics/min * 60 * 24: Retrieve data for 1 day.
    * 4 metrics/min * 60 * 24: Retrieve data for 1 day.
    */
    */
@@ -99,9 +100,11 @@ public class PhoenixHBaseAccessor {
   public static int RESULTSET_LIMIT = (int)TimeUnit.DAYS.toMinutes(1) *
   public static int RESULTSET_LIMIT = (int)TimeUnit.DAYS.toMinutes(1) *
     METRICS_PER_MINUTE;
     METRICS_PER_MINUTE;
   private static ObjectMapper mapper = new ObjectMapper();
   private static ObjectMapper mapper = new ObjectMapper();
-
   private static TypeReference<Map<Long, Double>> metricValuesTypeRef =
   private static TypeReference<Map<Long, Double>> metricValuesTypeRef =
     new TypeReference<Map<Long, Double>>() {};
     new TypeReference<Map<Long, Double>>() {};
+  private final Configuration hbaseConf;
+  private final Configuration metricsConf;
+  private final RetryCounterFactory retryCounterFactory;
   private final ConnectionProvider dataSource;
   private final ConnectionProvider dataSource;
 
 
   public PhoenixHBaseAccessor(Configuration hbaseConf,
   public PhoenixHBaseAccessor(Configuration hbaseConf,
@@ -127,37 +130,6 @@ public class PhoenixHBaseAccessor {
       (int) SECONDS.toMillis(metricsConf.getInt(GLOBAL_RETRY_INTERVAL, 5)));
       (int) SECONDS.toMillis(metricsConf.getInt(GLOBAL_RETRY_INTERVAL, 5)));
   }
   }
 
 
-
-  private Connection getConnectionRetryingOnException()
-    throws SQLException, InterruptedException {
-    RetryCounter retryCounter = retryCounterFactory.create();
-    while (true) {
-      try{
-        return getConnection();
-      } catch (SQLException e) {
-        if(!retryCounter.shouldRetry()){
-          LOG.error("HBaseAccessor getConnection failed after "
-            + retryCounter.getMaxAttempts() + " attempts");
-          throw e;
-        }
-      }
-      retryCounter.sleepUntilNextRetry();
-    }
-  }
-
-  /**
-   * Get JDBC connection to HBase store. Assumption is that the hbase
-   * configuration is present on the classpath and loaded by the caller into
-   * the Configuration object.
-   * Phoenix already caches the HConnection between the client and HBase
-   * cluster.
-   *
-   * @return @java.sql.Connection
-   */
-  public Connection getConnection() throws SQLException {
-    return dataSource.getConnection();
-  }
-
   private static TimelineMetric getLastTimelineMetricFromResultSet(ResultSet rs)
   private static TimelineMetric getLastTimelineMetricFromResultSet(ResultSet rs)
     throws SQLException, IOException {
     throws SQLException, IOException {
     TimelineMetric metric = TIMELINE_METRIC_READ_HELPER
     TimelineMetric metric = TIMELINE_METRIC_READ_HELPER
@@ -218,8 +190,7 @@ public class PhoenixHBaseAccessor {
   }
   }
 
 
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
-  public static Map<Long, Double>  readMetricFromJSON(String json)
-    throws IOException {
+  public static Map<Long, Double>  readMetricFromJSON(String json) throws IOException {
     return (Map<Long, Double>) mapper.readValue(json, metricValuesTypeRef);
     return (Map<Long, Double>) mapper.readValue(json, metricValuesTypeRef);
   }
   }
 
 
@@ -247,17 +218,34 @@ public class PhoenixHBaseAccessor {
     return metricHostAggregate;
     return metricHostAggregate;
   }
   }
 
 
-  public static MetricClusterAggregate getMetricClusterAggregateFromResultSet(ResultSet rs)
-    throws SQLException {
-    MetricClusterAggregate agg = new MetricClusterAggregate();
-    agg.setSum(rs.getDouble("METRIC_SUM"));
-    agg.setMax(rs.getDouble("METRIC_MAX"));
-    agg.setMin(rs.getDouble("METRIC_MIN"));
-    agg.setNumberOfHosts(rs.getInt("HOSTS_COUNT"));
-
-    agg.setDeviation(0.0);
+  private Connection getConnectionRetryingOnException()
+    throws SQLException, InterruptedException {
+    RetryCounter retryCounter = retryCounterFactory.create();
+    while (true) {
+      try{
+        return getConnection();
+      } catch (SQLException e) {
+        if(!retryCounter.shouldRetry()){
+          LOG.error("HBaseAccessor getConnection failed after "
+            + retryCounter.getMaxAttempts() + " attempts");
+          throw e;
+        }
+      }
+      retryCounter.sleepUntilNextRetry();
+    }
+  }
 
 
-    return agg;
+  /**
+   * Get JDBC connection to HBase store. Assumption is that the hbase
+   * configuration is present on the classpath and loaded by the caller into
+   * the Configuration object.
+   * Phoenix already caches the HConnection between the client and HBase
+   * cluster.
+   *
+   * @return @java.sql.Connection
+   */
+  public Connection getConnection() throws SQLException {
+    return dataSource.getConnection();
   }
   }
 
 
   protected void initMetricSchema() {
   protected void initMetricSchema() {
@@ -269,24 +257,33 @@ public class PhoenixHBaseAccessor {
     String precisionTtl = metricsConf.get(PRECISION_TABLE_TTL, "86400");
     String precisionTtl = metricsConf.get(PRECISION_TABLE_TTL, "86400");
     String hostMinTtl = metricsConf.get(HOST_MINUTE_TABLE_TTL, "604800");
     String hostMinTtl = metricsConf.get(HOST_MINUTE_TABLE_TTL, "604800");
     String hostHourTtl = metricsConf.get(HOST_HOUR_TABLE_TTL, "2592000");
     String hostHourTtl = metricsConf.get(HOST_HOUR_TABLE_TTL, "2592000");
+    String hostDailyTtl = metricsConf.get(HOST_DAILY_TABLE_TTL, "31536000");
     String clusterMinTtl = metricsConf.get(CLUSTER_MINUTE_TABLE_TTL, "2592000");
     String clusterMinTtl = metricsConf.get(CLUSTER_MINUTE_TABLE_TTL, "2592000");
     String clusterHourTtl = metricsConf.get(CLUSTER_HOUR_TABLE_TTL, "31536000");
     String clusterHourTtl = metricsConf.get(CLUSTER_HOUR_TABLE_TTL, "31536000");
+    String clusterDailyTtl = metricsConf.get(CLUSTER_DAILY_TABLE_TTL, "31536000");
 
 
     try {
     try {
       LOG.info("Initializing metrics schema...");
       LOG.info("Initializing metrics schema...");
       conn = getConnectionRetryingOnException();
       conn = getConnectionRetryingOnException();
       stmt = conn.createStatement();
       stmt = conn.createStatement();
 
 
+      // Host level
       stmt.executeUpdate(String.format(CREATE_METRICS_TABLE_SQL,
       stmt.executeUpdate(String.format(CREATE_METRICS_TABLE_SQL,
         encoding, precisionTtl, compression));
         encoding, precisionTtl, compression));
-      stmt.executeUpdate(String.format(CREATE_METRICS_AGGREGATE_HOURLY_TABLE_SQL,
-        encoding, hostHourTtl, compression));
-      stmt.executeUpdate(String.format(CREATE_METRICS_AGGREGATE_MINUTE_TABLE_SQL,
-        encoding, hostMinTtl, compression));
+      stmt.executeUpdate(String.format(CREATE_METRICS_AGGREGATE_TABLE_SQL,
+        METRICS_AGGREGATE_MINUTE_TABLE_NAME, encoding, hostMinTtl, compression));
+      stmt.executeUpdate(String.format(CREATE_METRICS_AGGREGATE_TABLE_SQL,
+        METRICS_AGGREGATE_HOURLY_TABLE_NAME, encoding, hostHourTtl, compression));
+      stmt.executeUpdate(String.format(CREATE_METRICS_AGGREGATE_TABLE_SQL,
+        METRICS_AGGREGATE_DAILY_TABLE_NAME, encoding, hostDailyTtl, compression));
+
+      // Cluster level
       stmt.executeUpdate(String.format(CREATE_METRICS_CLUSTER_AGGREGATE_TABLE_SQL,
       stmt.executeUpdate(String.format(CREATE_METRICS_CLUSTER_AGGREGATE_TABLE_SQL,
-        encoding, clusterMinTtl, compression));
+        METRICS_CLUSTER_AGGREGATE_TABLE_NAME, encoding, clusterMinTtl, compression));
+      stmt.executeUpdate(String.format(CREATE_METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_SQL,
+        METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME, encoding, clusterHourTtl, compression));
       stmt.executeUpdate(String.format(CREATE_METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_SQL,
       stmt.executeUpdate(String.format(CREATE_METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_SQL,
-        encoding, clusterHourTtl, compression));
+        METRICS_CLUSTER_AGGREGATE_DAILY_TABLE_NAME, encoding, clusterDailyTtl, compression));
 
 
       //alter TTL options to update tables
       //alter TTL options to update tables
       stmt.executeUpdate(String.format(ALTER_SQL,
       stmt.executeUpdate(String.format(ALTER_SQL,
@@ -298,12 +295,18 @@ public class PhoenixHBaseAccessor {
       stmt.executeUpdate(String.format(ALTER_SQL,
       stmt.executeUpdate(String.format(ALTER_SQL,
         METRICS_AGGREGATE_HOURLY_TABLE_NAME,
         METRICS_AGGREGATE_HOURLY_TABLE_NAME,
         hostHourTtl));
         hostHourTtl));
+      stmt.executeUpdate(String.format(ALTER_SQL,
+        METRICS_AGGREGATE_DAILY_TABLE_NAME,
+        hostDailyTtl));
       stmt.executeUpdate(String.format(ALTER_SQL,
       stmt.executeUpdate(String.format(ALTER_SQL,
         METRICS_CLUSTER_AGGREGATE_TABLE_NAME,
         METRICS_CLUSTER_AGGREGATE_TABLE_NAME,
         clusterMinTtl));
         clusterMinTtl));
       stmt.executeUpdate(String.format(ALTER_SQL,
       stmt.executeUpdate(String.format(ALTER_SQL,
         METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME,
         METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME,
         clusterHourTtl));
         clusterHourTtl));
+      stmt.executeUpdate(String.format(ALTER_SQL,
+        METRICS_CLUSTER_AGGREGATE_DAILY_TABLE_NAME,
+        clusterDailyTtl));
 
 
       conn.commit();
       conn.commit();
     } catch (SQLException sql) {
     } catch (SQLException sql) {
@@ -726,9 +729,8 @@ public class PhoenixHBaseAccessor {
     }
     }
   }
   }
 
 
-  public void saveHostAggregateRecords(Map<TimelineMetric,
-    MetricHostAggregate> hostAggregateMap, String phoenixTableName)
-    throws SQLException {
+  public void saveHostAggregateRecords(Map<TimelineMetric, MetricHostAggregate> hostAggregateMap,
+                                       String phoenixTableName) throws SQLException {
 
 
     if (hostAggregateMap == null || hostAggregateMap.isEmpty()) {
     if (hostAggregateMap == null || hostAggregateMap.isEmpty()) {
       LOG.debug("Empty aggregate records.");
       LOG.debug("Empty aggregate records.");
@@ -809,9 +811,8 @@ public class PhoenixHBaseAccessor {
    *
    *
    * @throws SQLException
    * @throws SQLException
    */
    */
-  public void saveClusterAggregateRecords(
-    Map<TimelineClusterMetric, MetricClusterAggregate> records)
-    throws SQLException {
+  public void saveClusterAggregateRecords(Map<TimelineClusterMetric, MetricClusterAggregate> records)
+      throws SQLException {
 
 
     if (records == null || records.isEmpty()) {
     if (records == null || records.isEmpty()) {
       LOG.debug("Empty aggregate records.");
       LOG.debug("Empty aggregate records.");
@@ -819,11 +820,11 @@ public class PhoenixHBaseAccessor {
     }
     }
 
 
     long start = System.currentTimeMillis();
     long start = System.currentTimeMillis();
-
+    String sqlStr = String.format(UPSERT_CLUSTER_AGGREGATE_SQL, METRICS_CLUSTER_AGGREGATE_TABLE_NAME);
     Connection conn = getConnection();
     Connection conn = getConnection();
     PreparedStatement stmt = null;
     PreparedStatement stmt = null;
     try {
     try {
-      stmt = conn.prepareStatement(UPSERT_CLUSTER_AGGREGATE_SQL);
+      stmt = conn.prepareStatement(sqlStr);
       int rowCount = 0;
       int rowCount = 0;
 
 
       for (Map.Entry<TimelineClusterMetric, MetricClusterAggregate>
       for (Map.Entry<TimelineClusterMetric, MetricClusterAggregate>
@@ -892,10 +893,8 @@ public class PhoenixHBaseAccessor {
    *
    *
    * @throws SQLException
    * @throws SQLException
    */
    */
-  public void saveClusterAggregateHourlyRecords(
-    Map<TimelineClusterMetric, MetricHostAggregate> records,
-    String tableName)
-    throws SQLException {
+  public void saveClusterTimeAggregateRecords(Map<TimelineClusterMetric, MetricHostAggregate> records,
+                                              String tableName) throws SQLException {
     if (records == null || records.isEmpty()) {
     if (records == null || records.isEmpty()) {
       LOG.debug("Empty aggregate records.");
       LOG.debug("Empty aggregate records.");
       return;
       return;
@@ -906,12 +905,10 @@ public class PhoenixHBaseAccessor {
     Connection conn = getConnection();
     Connection conn = getConnection();
     PreparedStatement stmt = null;
     PreparedStatement stmt = null;
     try {
     try {
-      stmt = conn.prepareStatement(String.format
-        (UPSERT_CLUSTER_AGGREGATE_TIME_SQL, tableName));
+      stmt = conn.prepareStatement(String.format(UPSERT_CLUSTER_AGGREGATE_TIME_SQL, tableName));
       int rowCount = 0;
       int rowCount = 0;
 
 
-      for (Map.Entry<TimelineClusterMetric, MetricHostAggregate>
-        aggregateEntry : records.entrySet()) {
+      for (Map.Entry<TimelineClusterMetric, MetricHostAggregate> aggregateEntry : records.entrySet()) {
         TimelineClusterMetric clusterMetric = aggregateEntry.getKey();
         TimelineClusterMetric clusterMetric = aggregateEntry.getKey();
         MetricHostAggregate aggregate = aggregateEntry.getValue();
         MetricHostAggregate aggregate = aggregateEntry.getValue();
 
 
@@ -928,7 +925,6 @@ public class PhoenixHBaseAccessor {
         stmt.setLong(4, clusterMetric.getTimestamp());
         stmt.setLong(4, clusterMetric.getTimestamp());
         stmt.setString(5, clusterMetric.getType());
         stmt.setString(5, clusterMetric.getType());
         stmt.setDouble(6, aggregate.getSum());
         stmt.setDouble(6, aggregate.getSum());
-//        stmt.setInt(7, aggregate.getNumberOfHosts());
         stmt.setLong(7, aggregate.getNumberOfSamples());
         stmt.setLong(7, aggregate.getNumberOfSamples());
         stmt.setDouble(8, aggregate.getMax());
         stmt.setDouble(8, aggregate.getMax());
         stmt.setDouble(9, aggregate.getMin());
         stmt.setDouble(9, aggregate.getMin());

+ 4 - 2
ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/Precision.java

@@ -26,7 +26,8 @@ package org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline
 public enum Precision {
 public enum Precision {
   SECONDS,
   SECONDS,
   MINUTES,
   MINUTES,
-  HOURS;
+  HOURS,
+  DAYS;
 
 
   public static class PrecisionFormatException extends IllegalArgumentException {
   public static class PrecisionFormatException extends IllegalArgumentException {
     public PrecisionFormatException(String message, Throwable cause) {
     public PrecisionFormatException(String message, Throwable cause) {
@@ -41,7 +42,8 @@ public enum Precision {
     try {
     try {
       return Precision.valueOf(precision.toUpperCase());
       return Precision.valueOf(precision.toUpperCase());
     } catch (IllegalArgumentException e) {
     } catch (IllegalArgumentException e) {
-      throw new PrecisionFormatException("precision should be seconds, minutes or hours", e);
+      throw new PrecisionFormatException("precision should be seconds, " +
+        "minutes, hours or days", e);
     }
     }
   }
   }
 }
 }

+ 27 - 2
ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricConfiguration.java

@@ -54,15 +54,25 @@ public class TimelineMetricConfiguration {
 
 
   public static final String PRECISION_TABLE_TTL =
   public static final String PRECISION_TABLE_TTL =
     "timeline.metrics.host.aggregator.ttl";
     "timeline.metrics.host.aggregator.ttl";
+
   public static final String HOST_MINUTE_TABLE_TTL =
   public static final String HOST_MINUTE_TABLE_TTL =
     "timeline.metrics.host.aggregator.minute.ttl";
     "timeline.metrics.host.aggregator.minute.ttl";
+
+  public static final String HOST_DAILY_TABLE_TTL =
+    "timeline.metrics.host.aggregator.daily.ttl";
+
   public static final String HOST_HOUR_TABLE_TTL =
   public static final String HOST_HOUR_TABLE_TTL =
     "timeline.metrics.host.aggregator.hourly.ttl";
     "timeline.metrics.host.aggregator.hourly.ttl";
+
   public static final String CLUSTER_MINUTE_TABLE_TTL =
   public static final String CLUSTER_MINUTE_TABLE_TTL =
     "timeline.metrics.cluster.aggregator.minute.ttl";
     "timeline.metrics.cluster.aggregator.minute.ttl";
+
   public static final String CLUSTER_HOUR_TABLE_TTL =
   public static final String CLUSTER_HOUR_TABLE_TTL =
     "timeline.metrics.cluster.aggregator.hourly.ttl";
     "timeline.metrics.cluster.aggregator.hourly.ttl";
 
 
+  public static final String CLUSTER_DAILY_TABLE_TTL =
+    "timeline.metrics.cluster.aggregator.daily.ttl";
+
   public static final String CLUSTER_AGGREGATOR_TIMESLICE_INTERVAL =
   public static final String CLUSTER_AGGREGATOR_TIMESLICE_INTERVAL =
     "timeline.metrics.cluster.aggregator.minute.timeslice.interval";
     "timeline.metrics.cluster.aggregator.minute.timeslice.interval";
 
 
@@ -78,26 +88,35 @@ public class TimelineMetricConfiguration {
   public static final String HOST_AGGREGATOR_HOUR_SLEEP_INTERVAL =
   public static final String HOST_AGGREGATOR_HOUR_SLEEP_INTERVAL =
     "timeline.metrics.host.aggregator.hourly.interval";
     "timeline.metrics.host.aggregator.hourly.interval";
 
 
+  public static final String HOST_AGGREGATOR_DAILY_SLEEP_INTERVAL =
+    "timeline.metrics.host.aggregator.daily.interval";
+
   public static final String CLUSTER_AGGREGATOR_MINUTE_SLEEP_INTERVAL =
   public static final String CLUSTER_AGGREGATOR_MINUTE_SLEEP_INTERVAL =
     "timeline.metrics.cluster.aggregator.minute.interval";
     "timeline.metrics.cluster.aggregator.minute.interval";
 
 
   public static final String CLUSTER_AGGREGATOR_HOUR_SLEEP_INTERVAL =
   public static final String CLUSTER_AGGREGATOR_HOUR_SLEEP_INTERVAL =
     "timeline.metrics.cluster.aggregator.hourly.interval";
     "timeline.metrics.cluster.aggregator.hourly.interval";
 
 
+  public static final String CLUSTER_AGGREGATOR_DAILY_SLEEP_INTERVAL =
+    "timeline.metrics.cluster.aggregator.daily.interval";
+
   public static final String HOST_AGGREGATOR_MINUTE_CHECKPOINT_CUTOFF_MULTIPLIER =
   public static final String HOST_AGGREGATOR_MINUTE_CHECKPOINT_CUTOFF_MULTIPLIER =
     "timeline.metrics.host.aggregator.minute.checkpointCutOffMultiplier";
     "timeline.metrics.host.aggregator.minute.checkpointCutOffMultiplier";
 
 
   public static final String HOST_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_MULTIPLIER =
   public static final String HOST_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_MULTIPLIER =
     "timeline.metrics.host.aggregator.hourly.checkpointCutOffMultiplier";
     "timeline.metrics.host.aggregator.hourly.checkpointCutOffMultiplier";
 
 
+  public static final String HOST_AGGREGATOR_DAILY_CHECKPOINT_CUTOFF_MULTIPLIER =
+    "timeline.metrics.host.aggregator.daily.checkpointCutOffMultiplier";
+
   public static final String CLUSTER_AGGREGATOR_MINUTE_CHECKPOINT_CUTOFF_MULTIPLIER =
   public static final String CLUSTER_AGGREGATOR_MINUTE_CHECKPOINT_CUTOFF_MULTIPLIER =
     "timeline.metrics.cluster.aggregator.minute.checkpointCutOffMultiplier";
     "timeline.metrics.cluster.aggregator.minute.checkpointCutOffMultiplier";
 
 
   public static final String CLUSTER_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_MULTIPLIER =
   public static final String CLUSTER_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_MULTIPLIER =
     "timeline.metrics.cluster.aggregator.hourly.checkpointCutOffMultiplier";
     "timeline.metrics.cluster.aggregator.hourly.checkpointCutOffMultiplier";
 
 
-  public static final String CLUSTER_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_INTERVAL =
-    "timeline.metrics.cluster.aggregator.hourly.checkpointCutOffInterval";
+  public static final String CLUSTER_AGGREGATOR_DAILY_CHECKPOINT_CUTOFF_MULTIPLIER =
+    "timeline.metrics.cluster.aggregator.daily.checkpointCutOffMultiplier";
 
 
   public static final String GLOBAL_RESULT_LIMIT =
   public static final String GLOBAL_RESULT_LIMIT =
     "timeline.metrics.service.default.result.limit";
     "timeline.metrics.service.default.result.limit";
@@ -114,12 +133,18 @@ public class TimelineMetricConfiguration {
   public static final String HOST_AGGREGATOR_HOUR_DISABLED =
   public static final String HOST_AGGREGATOR_HOUR_DISABLED =
     "timeline.metrics.host.aggregator.hourly.disabled";
     "timeline.metrics.host.aggregator.hourly.disabled";
 
 
+  public static final String HOST_AGGREGATOR_DAILY_DISABLED =
+    "timeline.metrics.host.aggregator.hourly.disabled";
+
   public static final String CLUSTER_AGGREGATOR_MINUTE_DISABLED =
   public static final String CLUSTER_AGGREGATOR_MINUTE_DISABLED =
     "timeline.metrics.cluster.aggregator.minute.disabled";
     "timeline.metrics.cluster.aggregator.minute.disabled";
 
 
   public static final String CLUSTER_AGGREGATOR_HOUR_DISABLED =
   public static final String CLUSTER_AGGREGATOR_HOUR_DISABLED =
     "timeline.metrics.cluster.aggregator.hourly.disabled";
     "timeline.metrics.cluster.aggregator.hourly.disabled";
 
 
+  public static final String CLUSTER_AGGREGATOR_DAILY_DISABLED =
+    "timeline.metrics.cluster.aggregator.daily.disabled";
+
   public static final String DISABLE_APPLICATION_TIMELINE_STORE =
   public static final String DISABLE_APPLICATION_TIMELINE_STORE =
     "timeline.service.disable.application.timeline.store";
     "timeline.service.disable.application.timeline.store";
 
 

+ 46 - 12
ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/AbstractTimelineAggregator.java

@@ -41,19 +41,21 @@ import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.ti
  * Base class for all runnable aggregators. Provides common functions like
  * Base class for all runnable aggregators. Provides common functions like
  * check pointing and scheduling.
  * check pointing and scheduling.
  */
  */
-public abstract class AbstractTimelineAggregator implements Runnable {
+public abstract class AbstractTimelineAggregator implements TimelineMetricAggregator {
   protected final PhoenixHBaseAccessor hBaseAccessor;
   protected final PhoenixHBaseAccessor hBaseAccessor;
   private final Log LOG;
   private final Log LOG;
-
   private Clock clock;
   private Clock clock;
   protected final long checkpointDelayMillis;
   protected final long checkpointDelayMillis;
   protected final Integer resultsetFetchSize;
   protected final Integer resultsetFetchSize;
   protected Configuration metricsConf;
   protected Configuration metricsConf;
 
 
-  public AbstractTimelineAggregator(PhoenixHBaseAccessor hBaseAccessor,
-                                    Configuration metricsConf) {
-    this(hBaseAccessor, metricsConf, new SystemClock());
-  }
+  private String checkpointLocation;
+  private Long sleepIntervalMillis;
+  private Integer checkpointCutOffMultiplier;
+  private String aggregatorDisableParam;
+  protected String tableName;
+  protected String outputTableName;
+  protected Long nativeTimeRangeDelay;
 
 
   public AbstractTimelineAggregator(PhoenixHBaseAccessor hBaseAccessor,
   public AbstractTimelineAggregator(PhoenixHBaseAccessor hBaseAccessor,
                                     Configuration metricsConf, Clock clk) {
                                     Configuration metricsConf, Clock clk) {
@@ -66,6 +68,30 @@ public abstract class AbstractTimelineAggregator implements Runnable {
     this.clock = clk;
     this.clock = clk;
   }
   }
 
 
+  public AbstractTimelineAggregator(PhoenixHBaseAccessor hBaseAccessor,
+                                    Configuration metricsConf) {
+    this(hBaseAccessor, metricsConf, new SystemClock());
+  }
+
+  public AbstractTimelineAggregator(PhoenixHBaseAccessor hBaseAccessor,
+                                    Configuration metricsConf,
+                                    String checkpointLocation,
+                                    Long sleepIntervalMillis,
+                                    Integer checkpointCutOffMultiplier,
+                                    String aggregatorDisableParam,
+                                    String tableName,
+                                    String outputTableName,
+                                    Long nativeTimeRangeDelay) {
+    this(hBaseAccessor, metricsConf);
+    this.checkpointLocation = checkpointLocation;
+    this.sleepIntervalMillis = sleepIntervalMillis;
+    this.checkpointCutOffMultiplier = checkpointCutOffMultiplier;
+    this.aggregatorDisableParam = aggregatorDisableParam;
+    this.tableName = tableName;
+    this.outputTableName = outputTableName;
+    this.nativeTimeRangeDelay =  nativeTimeRangeDelay;
+  }
+
   @Override
   @Override
   public void run() {
   public void run() {
     LOG.info("Started Timeline aggregator thread @ " + new Date());
     LOG.info("Started Timeline aggregator thread @ " + new Date());
@@ -198,6 +224,7 @@ public abstract class AbstractTimelineAggregator implements Runnable {
    * @param startTime Sample start time
    * @param startTime Sample start time
    * @param endTime Sample end time
    * @param endTime Sample end time
    */
    */
+  @Override
   public boolean doWork(long startTime, long endTime) {
   public boolean doWork(long startTime, long endTime) {
     LOG.info("Start aggregation cycle @ " + new Date() + ", " +
     LOG.info("Start aggregation cycle @ " + new Date() + ", " +
       "startTime = " + new Date(startTime) + ", endTime = " + new Date(endTime));
       "startTime = " + new Date(startTime) + ", endTime = " + new Date(endTime));
@@ -257,18 +284,25 @@ public abstract class AbstractTimelineAggregator implements Runnable {
 
 
   protected abstract Condition prepareMetricQueryCondition(long startTime, long endTime);
   protected abstract Condition prepareMetricQueryCondition(long startTime, long endTime);
 
 
-  protected abstract void aggregate(ResultSet rs, long startTime, long endTime)
-    throws IOException, SQLException;
+  protected abstract void aggregate(ResultSet rs, long startTime, long endTime) throws IOException, SQLException;
 
 
-  protected abstract Long getSleepIntervalMillis();
+  protected Long getSleepIntervalMillis() {
+    return sleepIntervalMillis;
+  }
 
 
-  protected abstract Integer getCheckpointCutOffMultiplier();
+  protected Integer getCheckpointCutOffMultiplier() {
+    return checkpointCutOffMultiplier;
+  }
 
 
   protected Long getCheckpointCutOffIntervalMillis() {
   protected Long getCheckpointCutOffIntervalMillis() {
     return getCheckpointCutOffMultiplier() * getSleepIntervalMillis();
     return getCheckpointCutOffMultiplier() * getSleepIntervalMillis();
   }
   }
 
 
-  public abstract boolean isDisabled();
+  public boolean isDisabled() {
+    return metricsConf.getBoolean(aggregatorDisableParam, false);
+  }
 
 
-  protected abstract String getCheckpointLocation();
+  protected String getCheckpointLocation() {
+    return checkpointLocation;
+  }
 }
 }

+ 0 - 42
ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineClusterMetricReader.java

@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators;
-
-import java.sql.ResultSet;
-import java.sql.SQLException;
-
-public class TimelineClusterMetricReader {
-
-  private boolean ignoreInstance;
-
-  public TimelineClusterMetricReader(boolean ignoreInstance) {
-    this.ignoreInstance = ignoreInstance;
-  }
-
-  public TimelineClusterMetric fromResultSet(ResultSet rs)
-    throws SQLException {
-
-    return new TimelineClusterMetric(
-      rs.getString("METRIC_NAME"),
-      rs.getString("APP_ID"),
-      ignoreInstance ? null : rs.getString("INSTANCE_ID"),
-      rs.getLong("SERVER_TIME"),
-      rs.getString("UNITS"));
-  }
-}
-

+ 19 - 132
ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineMetricAggregator.java

@@ -1,3 +1,5 @@
+package org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators;
+
 /**
 /**
  * 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
@@ -6,142 +8,27 @@
  * to you under the Apache License, Version 2.0 (the
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
  * Unless required by applicable law or agreed to in writing, software
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixHBaseAccessor;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.Condition;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.DefaultCondition;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL;
-import java.io.IOException;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.HashMap;
-import java.util.Map;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.GET_METRIC_AGGREGATE_ONLY_SQL;
-
-public class TimelineMetricAggregator extends AbstractTimelineAggregator {
-  private static final Log LOG = LogFactory.getLog
-    (TimelineMetricAggregator.class);
-
-  private final String checkpointLocation;
-  private final Long sleepIntervalMillis;
-  private final Integer checkpointCutOffMultiplier;
-  private final String hostAggregatorDisabledParam;
-  private final String tableName;
-  private final String outputTableName;
-  private final Long nativeTimeRangeDelay;
-
-  public TimelineMetricAggregator(PhoenixHBaseAccessor hBaseAccessor,
-                                  Configuration metricsConf,
-                                  String checkpointLocation,
-                                  Long sleepIntervalMillis,
-                                  Integer checkpointCutOffMultiplier,
-                                  String hostAggregatorDisabledParam,
-                                  String tableName,
-                                  String outputTableName,
-                                  Long nativeTimeRangeDelay) {
-    super(hBaseAccessor, metricsConf);
-    this.checkpointLocation = checkpointLocation;
-    this.sleepIntervalMillis = sleepIntervalMillis;
-    this.checkpointCutOffMultiplier = checkpointCutOffMultiplier;
-    this.hostAggregatorDisabledParam = hostAggregatorDisabledParam;
-    this.tableName = tableName;
-    this.outputTableName = outputTableName;
-    this.nativeTimeRangeDelay =  nativeTimeRangeDelay;
-  }
-
-  @Override
-  protected String getCheckpointLocation() {
-    return checkpointLocation;
-  }
-
-  @Override
-  protected void aggregate(ResultSet rs, long startTime, long endTime)
-    throws IOException, SQLException {
-    Map<TimelineMetric, MetricHostAggregate> hostAggregateMap =
-      aggregateMetricsFromResultSet(rs);
-
-    LOG.info("Saving " + hostAggregateMap.size() + " metric aggregates.");
-    hBaseAccessor.saveHostAggregateRecords(hostAggregateMap,
-      outputTableName);
-  }
-
-  @Override
-  protected Condition prepareMetricQueryCondition(long startTime, long endTime) {
-    Condition condition = new DefaultCondition(null, null, null, null, startTime,
-      endTime, null, null, true);
-    condition.setNoLimit();
-    condition.setFetchSize(resultsetFetchSize);
-    condition.setStatement(String.format(GET_METRIC_AGGREGATE_ONLY_SQL,
-      PhoenixTransactSQL.getNaiveTimeRangeHint(startTime, nativeTimeRangeDelay),
-      tableName));
-    condition.addOrderByColumn("METRIC_NAME");
-    condition.addOrderByColumn("HOSTNAME");
-    condition.addOrderByColumn("APP_ID");
-    condition.addOrderByColumn("INSTANCE_ID");
-    condition.addOrderByColumn("SERVER_TIME");
-    return condition;
-  }
-
-  private Map<TimelineMetric, MetricHostAggregate> aggregateMetricsFromResultSet
-      (ResultSet rs) throws IOException, SQLException {
-    TimelineMetric existingMetric = null;
-    MetricHostAggregate hostAggregate = null;
-    Map<TimelineMetric, MetricHostAggregate> hostAggregateMap =
-      new HashMap<TimelineMetric, MetricHostAggregate>();
-
-    while (rs.next()) {
-      TimelineMetric currentMetric =
-        PhoenixHBaseAccessor.getTimelineMetricKeyFromResultSet(rs);
-      MetricHostAggregate currentHostAggregate =
-        PhoenixHBaseAccessor.getMetricHostAggregateFromResultSet(rs);
-
-      if (existingMetric == null) {
-        // First row
-        existingMetric = currentMetric;
-        hostAggregate = new MetricHostAggregate();
-        hostAggregateMap.put(currentMetric, hostAggregate);
-      }
-
-      if (existingMetric.equalsExceptTime(currentMetric)) {
-        // Recalculate totals with current metric
-        hostAggregate.updateAggregates(currentHostAggregate);
-      } else {
-        // Switched over to a new metric - save existing - create new aggregate
-        hostAggregate = new MetricHostAggregate();
-        hostAggregate.updateAggregates(currentHostAggregate);
-        hostAggregateMap.put(currentMetric, hostAggregate);
-        existingMetric = currentMetric;
-      }
-    }
-    return hostAggregateMap;
-  }
-
-  @Override
-  protected Long getSleepIntervalMillis() {
-    return sleepIntervalMillis;
-  }
-
-  @Override
-  protected Integer getCheckpointCutOffMultiplier() {
-    return checkpointCutOffMultiplier;
-  }
-
-  @Override
-  public boolean isDisabled() {
-    return metricsConf.getBoolean(hostAggregatorDisabledParam, false);
-  }
+public interface TimelineMetricAggregator extends Runnable {
+  /**
+   * Aggregate metric data within the time bounds.
+   * @param startTime start time millis
+   * @param endTime end time millis
+   * @return success
+   */
+  public boolean doWork(long startTime, long endTime);
+
+  /**
+   * Is aggregator is disabled by configuration.
+   * @return true/false
+   */
+  public boolean isDisabled();
 }
 }

+ 179 - 9
ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineMetricAggregatorFactory.java

@@ -22,10 +22,20 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixHBaseAccessor;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixHBaseAccessor;
 
 
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_HOURLY_TABLE_NAME;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_MINUTE_TABLE_NAME;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_RECORD_TABLE_NAME;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_DAILY_CHECKPOINT_CUTOFF_MULTIPLIER;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_DAILY_DISABLED;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_DAILY_SLEEP_INTERVAL;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_MULTIPLIER;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_HOUR_DISABLED;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_HOUR_SLEEP_INTERVAL;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_MINUTE_CHECKPOINT_CUTOFF_MULTIPLIER;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_MINUTE_DISABLED;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_MINUTE_SLEEP_INTERVAL;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_TIMESLICE_INTERVAL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.DEFAULT_CHECKPOINT_LOCATION;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.DEFAULT_CHECKPOINT_LOCATION;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_AGGREGATOR_DAILY_CHECKPOINT_CUTOFF_MULTIPLIER;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_AGGREGATOR_DAILY_DISABLED;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_AGGREGATOR_DAILY_SLEEP_INTERVAL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_MULTIPLIER;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_MULTIPLIER;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_AGGREGATOR_HOUR_DISABLED;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_AGGREGATOR_HOUR_DISABLED;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_AGGREGATOR_HOUR_SLEEP_INTERVAL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_AGGREGATOR_HOUR_SLEEP_INTERVAL;
@@ -33,20 +43,42 @@ import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.ti
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_AGGREGATOR_MINUTE_DISABLED;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_AGGREGATOR_MINUTE_DISABLED;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_AGGREGATOR_MINUTE_SLEEP_INTERVAL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.HOST_AGGREGATOR_MINUTE_SLEEP_INTERVAL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_DAILY_TABLE_NAME;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_HOURLY_TABLE_NAME;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_MINUTE_TABLE_NAME;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_DAILY_TABLE_NAME;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_RECORD_TABLE_NAME;
 
 
+/**
+ * Factory class that knows how to create a aggregator instance using
+ * @TimelineMetricConfiguration
+ */
 public class TimelineMetricAggregatorFactory {
 public class TimelineMetricAggregatorFactory {
-  private static final String MINUTE_AGGREGATE_CHECKPOINT_FILE =
+  private static final String HOST_AGGREGATE_MINUTE_CHECKPOINT_FILE =
     "timeline-metrics-host-aggregator-checkpoint";
     "timeline-metrics-host-aggregator-checkpoint";
-  private static final String MINUTE_AGGREGATE_HOURLY_CHECKPOINT_FILE =
+  private static final String HOST_AGGREGATE_HOURLY_CHECKPOINT_FILE =
     "timeline-metrics-host-aggregator-hourly-checkpoint";
     "timeline-metrics-host-aggregator-hourly-checkpoint";
+  private static final String HOST_AGGREGATE_DAILY_CHECKPOINT_FILE =
+    "timeline-metrics-host-aggregator-daily-checkpoint";
+  private static final String CLUSTER_AGGREGATOR_CHECKPOINT_FILE =
+    "timeline-metrics-cluster-aggregator-checkpoint";
+  private static final String CLUSTER_AGGREGATOR_HOURLY_CHECKPOINT_FILE =
+    "timeline-metrics-cluster-aggregator-hourly-checkpoint";
+  private static final String CLUSTER_AGGREGATOR_DAILY_CHECKPOINT_FILE =
+    "timeline-metrics-cluster-aggregator-daily-checkpoint";
 
 
+  /**
+   * Minute based aggregation for hosts.
+   */
   public static TimelineMetricAggregator createTimelineMetricAggregatorMinute
   public static TimelineMetricAggregator createTimelineMetricAggregatorMinute
     (PhoenixHBaseAccessor hBaseAccessor, Configuration metricsConf) {
     (PhoenixHBaseAccessor hBaseAccessor, Configuration metricsConf) {
 
 
     String checkpointDir = metricsConf.get(
     String checkpointDir = metricsConf.get(
       TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR, DEFAULT_CHECKPOINT_LOCATION);
       TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR, DEFAULT_CHECKPOINT_LOCATION);
     String checkpointLocation = FilenameUtils.concat(checkpointDir,
     String checkpointLocation = FilenameUtils.concat(checkpointDir,
-      MINUTE_AGGREGATE_CHECKPOINT_FILE);
+      HOST_AGGREGATE_MINUTE_CHECKPOINT_FILE);
     long sleepIntervalMillis = SECONDS.toMillis(metricsConf.getLong
     long sleepIntervalMillis = SECONDS.toMillis(metricsConf.getLong
       (HOST_AGGREGATOR_MINUTE_SLEEP_INTERVAL, 300l));  // 5 mins
       (HOST_AGGREGATOR_MINUTE_SLEEP_INTERVAL, 300l));  // 5 mins
 
 
@@ -57,7 +89,7 @@ public class TimelineMetricAggregatorFactory {
     String inputTableName = METRICS_RECORD_TABLE_NAME;
     String inputTableName = METRICS_RECORD_TABLE_NAME;
     String outputTableName = METRICS_AGGREGATE_MINUTE_TABLE_NAME;
     String outputTableName = METRICS_AGGREGATE_MINUTE_TABLE_NAME;
 
 
-    return new TimelineMetricAggregator(hBaseAccessor, metricsConf,
+    return new TimelineMetricHostAggregator(hBaseAccessor, metricsConf,
       checkpointLocation,
       checkpointLocation,
       sleepIntervalMillis,
       sleepIntervalMillis,
       checkpointCutOffMultiplier,
       checkpointCutOffMultiplier,
@@ -67,13 +99,16 @@ public class TimelineMetricAggregatorFactory {
       120000l);
       120000l);
   }
   }
 
 
+  /**
+   * Hourly aggregation for hosts.
+   */
   public static TimelineMetricAggregator createTimelineMetricAggregatorHourly
   public static TimelineMetricAggregator createTimelineMetricAggregatorHourly
     (PhoenixHBaseAccessor hBaseAccessor, Configuration metricsConf) {
     (PhoenixHBaseAccessor hBaseAccessor, Configuration metricsConf) {
 
 
     String checkpointDir = metricsConf.get(
     String checkpointDir = metricsConf.get(
       TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR, DEFAULT_CHECKPOINT_LOCATION);
       TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR, DEFAULT_CHECKPOINT_LOCATION);
     String checkpointLocation = FilenameUtils.concat(checkpointDir,
     String checkpointLocation = FilenameUtils.concat(checkpointDir,
-      MINUTE_AGGREGATE_HOURLY_CHECKPOINT_FILE);
+      HOST_AGGREGATE_HOURLY_CHECKPOINT_FILE);
     long sleepIntervalMillis = SECONDS.toMillis(metricsConf.getLong
     long sleepIntervalMillis = SECONDS.toMillis(metricsConf.getLong
       (HOST_AGGREGATOR_HOUR_SLEEP_INTERVAL, 3600l));
       (HOST_AGGREGATOR_HOUR_SLEEP_INTERVAL, 3600l));
 
 
@@ -84,7 +119,37 @@ public class TimelineMetricAggregatorFactory {
     String inputTableName = METRICS_AGGREGATE_MINUTE_TABLE_NAME;
     String inputTableName = METRICS_AGGREGATE_MINUTE_TABLE_NAME;
     String outputTableName = METRICS_AGGREGATE_HOURLY_TABLE_NAME;
     String outputTableName = METRICS_AGGREGATE_HOURLY_TABLE_NAME;
 
 
-    return new TimelineMetricAggregator(hBaseAccessor, metricsConf,
+    return new TimelineMetricHostAggregator(hBaseAccessor, metricsConf,
+      checkpointLocation,
+      sleepIntervalMillis,
+      checkpointCutOffMultiplier,
+      hostAggregatorDisabledParam,
+      inputTableName,
+      outputTableName,
+      3600000l);
+  }
+
+  /**
+   * Daily aggregation for hosts.
+   */
+  public static TimelineMetricAggregator createTimelineMetricAggregatorDaily
+    (PhoenixHBaseAccessor hBaseAccessor, Configuration metricsConf) {
+
+    String checkpointDir = metricsConf.get(
+      TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR, DEFAULT_CHECKPOINT_LOCATION);
+    String checkpointLocation = FilenameUtils.concat(checkpointDir,
+      HOST_AGGREGATE_DAILY_CHECKPOINT_FILE);
+    long sleepIntervalMillis = SECONDS.toMillis(metricsConf.getLong
+      (HOST_AGGREGATOR_DAILY_SLEEP_INTERVAL, 86400l));
+
+    int checkpointCutOffMultiplier = metricsConf.getInt
+      (HOST_AGGREGATOR_DAILY_CHECKPOINT_CUTOFF_MULTIPLIER, 1);
+    String hostAggregatorDisabledParam = HOST_AGGREGATOR_DAILY_DISABLED;
+
+    String inputTableName = METRICS_AGGREGATE_HOURLY_TABLE_NAME;
+    String outputTableName = METRICS_AGGREGATE_DAILY_TABLE_NAME;
+
+    return new TimelineMetricHostAggregator(hBaseAccessor, metricsConf,
       checkpointLocation,
       checkpointLocation,
       sleepIntervalMillis,
       sleepIntervalMillis,
       checkpointCutOffMultiplier,
       checkpointCutOffMultiplier,
@@ -94,5 +159,110 @@ public class TimelineMetricAggregatorFactory {
       3600000l);
       3600000l);
   }
   }
 
 
+  /**
+   * Minute based aggregation for cluster.
+   */
+  public static TimelineMetricAggregator createTimelineClusterAggregatorMinute(
+      PhoenixHBaseAccessor hBaseAccessor, Configuration metricsConf) {
+
+    String checkpointDir = metricsConf.get(
+      TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR, DEFAULT_CHECKPOINT_LOCATION);
+
+    String checkpointLocation = FilenameUtils.concat(checkpointDir,
+      CLUSTER_AGGREGATOR_CHECKPOINT_FILE);
+
+    long sleepIntervalMillis = SECONDS.toMillis(metricsConf.getLong
+      (CLUSTER_AGGREGATOR_MINUTE_SLEEP_INTERVAL, 120l));
+
+    long timeSliceIntervalMillis = SECONDS.toMillis(metricsConf.getInt
+      (CLUSTER_AGGREGATOR_TIMESLICE_INTERVAL, 15));
+
+    int checkpointCutOffMultiplier =
+      metricsConf.getInt(CLUSTER_AGGREGATOR_MINUTE_CHECKPOINT_CUTOFF_MULTIPLIER, 2);
+
+    String inputTableName = METRICS_RECORD_TABLE_NAME;
+    String outputTableName = METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
+    String aggregatorDisabledParam = CLUSTER_AGGREGATOR_MINUTE_DISABLED;
+
+    // Minute based aggregation have added responsibility of time slicing
+    return new TimelineMetricClusterAggregatorMinute(
+      hBaseAccessor, metricsConf,
+      checkpointLocation,
+      sleepIntervalMillis,
+      checkpointCutOffMultiplier,
+      aggregatorDisabledParam,
+      inputTableName,
+      outputTableName,
+      120000l,
+      timeSliceIntervalMillis
+    );
+  }
+
+  /**
+   * Hourly aggregation for cluster.
+   */
+  public static TimelineMetricAggregator createTimelineClusterAggregatorHourly(
+    PhoenixHBaseAccessor hBaseAccessor, Configuration metricsConf) {
+
+    String checkpointDir = metricsConf.get(
+      TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR, DEFAULT_CHECKPOINT_LOCATION);
+
+    String checkpointLocation = FilenameUtils.concat(checkpointDir,
+      CLUSTER_AGGREGATOR_HOURLY_CHECKPOINT_FILE);
+
+    long sleepIntervalMillis = SECONDS.toMillis(metricsConf.getLong
+      (CLUSTER_AGGREGATOR_HOUR_SLEEP_INTERVAL, 3600l));
+
+    int checkpointCutOffMultiplier = metricsConf.getInt
+      (CLUSTER_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_MULTIPLIER, 2);
+
+    String inputTableName = METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
+    String outputTableName = METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME;
+    String aggregatorDisabledParam = CLUSTER_AGGREGATOR_HOUR_DISABLED;
+
+    return new TimelineMetricClusterAggregator(
+      hBaseAccessor, metricsConf,
+      checkpointLocation,
+      sleepIntervalMillis,
+      checkpointCutOffMultiplier,
+      aggregatorDisabledParam,
+      inputTableName,
+      outputTableName,
+      120000l
+    );
+  }
+
+  /**
+   * Daily aggregation for cluster.
+   */
+  public static TimelineMetricAggregator createTimelineClusterAggregatorDaily(
+    PhoenixHBaseAccessor hBaseAccessor, Configuration metricsConf) {
+
+    String checkpointDir = metricsConf.get(
+      TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR, DEFAULT_CHECKPOINT_LOCATION);
+
+    String checkpointLocation = FilenameUtils.concat(checkpointDir,
+      CLUSTER_AGGREGATOR_DAILY_CHECKPOINT_FILE);
+
+    long sleepIntervalMillis = SECONDS.toMillis(metricsConf.getLong
+      (CLUSTER_AGGREGATOR_DAILY_SLEEP_INTERVAL, 86400l));
+
+    int checkpointCutOffMultiplier = metricsConf.getInt
+      (CLUSTER_AGGREGATOR_DAILY_CHECKPOINT_CUTOFF_MULTIPLIER, 1);
 
 
+    String inputTableName = METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME;
+    String outputTableName = METRICS_CLUSTER_AGGREGATE_DAILY_TABLE_NAME;
+    String aggregatorDisabledParam = CLUSTER_AGGREGATOR_DAILY_DISABLED;
+
+    return new TimelineMetricClusterAggregator(
+      hBaseAccessor, metricsConf,
+      checkpointLocation,
+      sleepIntervalMillis,
+      checkpointCutOffMultiplier,
+      aggregatorDisabledParam,
+      inputTableName,
+      outputTableName,
+      120000l
+    );
+  }
 }
 }

+ 68 - 167
ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineMetricClusterAggregator.java

@@ -17,12 +17,9 @@
  */
  */
 package org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators;
 package org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators;
 
 
-
-import org.apache.commons.io.FilenameUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixHBaseAccessor;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixHBaseAccessor;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.Condition;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.Condition;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.DefaultCondition;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.DefaultCondition;
@@ -30,74 +27,32 @@ import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.
 import java.io.IOException;
 import java.io.IOException;
 import java.sql.ResultSet;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.SQLException;
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Map;
-import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_MINUTE_CHECKPOINT_CUTOFF_MULTIPLIER;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_MINUTE_DISABLED;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_MINUTE_SLEEP_INTERVAL;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_TIMESLICE_INTERVAL;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.DEFAULT_CHECKPOINT_LOCATION;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.GET_METRIC_SQL;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_RECORD_TABLE_NAME;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.GET_CLUSTER_AGGREGATE_SQL;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.GET_CLUSTER_AGGREGATE_TIME_SQL;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.NATIVE_TIME_RANGE_DELTA;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.NATIVE_TIME_RANGE_DELTA;
 
 
-/**
- * Aggregates a metric across all hosts in the cluster. Reads metrics from
- * the precision table and saves into the aggregate.
- */
 public class TimelineMetricClusterAggregator extends AbstractTimelineAggregator {
 public class TimelineMetricClusterAggregator extends AbstractTimelineAggregator {
+  private final TimelineMetricReadHelper readHelper = new TimelineMetricReadHelper(true);
   private static final Log LOG = LogFactory.getLog(TimelineMetricClusterAggregator.class);
   private static final Log LOG = LogFactory.getLog(TimelineMetricClusterAggregator.class);
-  private static final String CLUSTER_AGGREGATOR_CHECKPOINT_FILE =
-    "timeline-metrics-cluster-aggregator-checkpoint";
-  private final String checkpointLocation;
-  private final Long sleepIntervalMillis;
-  public final int timeSliceIntervalMillis;
-  private final Integer checkpointCutOffMultiplier;
-  private TimelineMetricReadHelper timelineMetricReadHelper = new TimelineMetricReadHelper(true);
-  // Aggregator to perform app-level aggregates for host metrics
-  private final TimelineMetricAppAggregator appAggregator;
+  private final boolean isClusterPrecisionInputTable;
 
 
   public TimelineMetricClusterAggregator(PhoenixHBaseAccessor hBaseAccessor,
   public TimelineMetricClusterAggregator(PhoenixHBaseAccessor hBaseAccessor,
-                                         Configuration metricsConf) {
-    super(hBaseAccessor, metricsConf);
-
-    String checkpointDir = metricsConf.get(
-      TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR, DEFAULT_CHECKPOINT_LOCATION);
-
-    checkpointLocation = FilenameUtils.concat(checkpointDir,
-      CLUSTER_AGGREGATOR_CHECKPOINT_FILE);
-
-    sleepIntervalMillis = SECONDS.toMillis(metricsConf.getLong
-      (CLUSTER_AGGREGATOR_MINUTE_SLEEP_INTERVAL, 120l));
-    timeSliceIntervalMillis = (int)SECONDS.toMillis(metricsConf.getInt
-      (CLUSTER_AGGREGATOR_TIMESLICE_INTERVAL, 15));
-    checkpointCutOffMultiplier =
-      metricsConf.getInt(CLUSTER_AGGREGATOR_MINUTE_CHECKPOINT_CUTOFF_MULTIPLIER, 2);
-
-    appAggregator = new TimelineMetricAppAggregator(metricsConf);
-  }
-
-  @Override
-  protected String getCheckpointLocation() {
-    return checkpointLocation;
-  }
-
-  @Override
-  protected void aggregate(ResultSet rs, long startTime, long endTime)
-    throws SQLException, IOException {
-    List<Long[]> timeSlices = getTimeSlices(startTime, endTime);
-    // Initialize app aggregates for host metrics
-    appAggregator.init();
-    Map<TimelineClusterMetric, MetricClusterAggregate>
-      aggregateClusterMetrics = aggregateMetricsFromResultSet(rs, timeSlices);
-
-    LOG.info("Saving " + aggregateClusterMetrics.size() + " metric aggregates.");
-    hBaseAccessor.saveClusterAggregateRecords(aggregateClusterMetrics);
-    appAggregator.cleanup();
+                                         Configuration metricsConf,
+                                         String checkpointLocation,
+                                         Long sleepIntervalMillis,
+                                         Integer checkpointCutOffMultiplier,
+                                         String hostAggregatorDisabledParam,
+                                         String inputTableName,
+                                         String outputTableName,
+                                         Long nativeTimeRangeDelay) {
+    super(hBaseAccessor, metricsConf, checkpointLocation,
+      sleepIntervalMillis, checkpointCutOffMultiplier,
+      hostAggregatorDisabledParam, inputTableName, outputTableName,
+      nativeTimeRangeDelay);
+    isClusterPrecisionInputTable = inputTableName.equals(METRICS_CLUSTER_AGGREGATE_TABLE_NAME);
   }
   }
 
 
   @Override
   @Override
@@ -106,9 +61,17 @@ public class TimelineMetricClusterAggregator extends AbstractTimelineAggregator
       endTime, null, null, true);
       endTime, null, null, true);
     condition.setNoLimit();
     condition.setNoLimit();
     condition.setFetchSize(resultsetFetchSize);
     condition.setFetchSize(resultsetFetchSize);
-    condition.setStatement(String.format(GET_METRIC_SQL,
+    String sqlStr = String.format(GET_CLUSTER_AGGREGATE_TIME_SQL,
       PhoenixTransactSQL.getNaiveTimeRangeHint(startTime, NATIVE_TIME_RANGE_DELTA),
       PhoenixTransactSQL.getNaiveTimeRangeHint(startTime, NATIVE_TIME_RANGE_DELTA),
-      METRICS_RECORD_TABLE_NAME));
+      tableName);
+    // HOST_COUNT vs METRIC_COUNT
+    if (isClusterPrecisionInputTable) {
+      sqlStr = String.format(GET_CLUSTER_AGGREGATE_SQL,
+        PhoenixTransactSQL.getNaiveTimeRangeHint(startTime, NATIVE_TIME_RANGE_DELTA),
+        tableName);
+    }
+
+    condition.setStatement(sqlStr);
     condition.addOrderByColumn("METRIC_NAME");
     condition.addOrderByColumn("METRIC_NAME");
     condition.addOrderByColumn("APP_ID");
     condition.addOrderByColumn("APP_ID");
     condition.addOrderByColumn("INSTANCE_ID");
     condition.addOrderByColumn("INSTANCE_ID");
@@ -116,120 +79,58 @@ public class TimelineMetricClusterAggregator extends AbstractTimelineAggregator
     return condition;
     return condition;
   }
   }
 
 
-  private List<Long[]> getTimeSlices(long startTime, long endTime) {
-    List<Long[]> timeSlices = new ArrayList<Long[]>();
-    long sliceStartTime = startTime;
-    while (sliceStartTime < endTime) {
-      timeSlices.add(new Long[] { sliceStartTime, sliceStartTime + timeSliceIntervalMillis});
-      sliceStartTime += timeSliceIntervalMillis;
-    }
-    return timeSlices;
-  }
-
-  private Map<TimelineClusterMetric, MetricClusterAggregate> aggregateMetricsFromResultSet(ResultSet rs, List<Long[]> timeSlices)
-      throws SQLException, IOException {
-    Map<TimelineClusterMetric, MetricClusterAggregate> aggregateClusterMetrics =
-      new HashMap<TimelineClusterMetric, MetricClusterAggregate>();
-    // Create time slices
-
-    while (rs.next()) {
-      TimelineMetric metric = timelineMetricReadHelper.getTimelineMetricFromResultSet(rs);
-
-      Map<TimelineClusterMetric, Double> clusterMetrics =
-        sliceFromTimelineMetric(metric, timeSlices);
-
-      if (clusterMetrics != null && !clusterMetrics.isEmpty()) {
-        for (Map.Entry<TimelineClusterMetric, Double> clusterMetricEntry :
-            clusterMetrics.entrySet()) {
-
-          TimelineClusterMetric clusterMetric = clusterMetricEntry.getKey();
-          Double avgValue = clusterMetricEntry.getValue();
-
-          MetricClusterAggregate aggregate = aggregateClusterMetrics.get(clusterMetric);
-
-          if (aggregate == null) {
-            aggregate = new MetricClusterAggregate(avgValue, 1, null, avgValue, avgValue);
-            aggregateClusterMetrics.put(clusterMetric, aggregate);
-          } else {
-            aggregate.updateSum(avgValue);
-            aggregate.updateNumberOfHosts(1);
-            aggregate.updateMax(avgValue);
-            aggregate.updateMin(avgValue);
-          }
-          // Update app level aggregates
-          appAggregator.processTimelineClusterMetric(clusterMetric,
-            metric.getHostName(), avgValue);
-        }
-      }
-    }
-    // Add app level aggregates to save
-    aggregateClusterMetrics.putAll(appAggregator.getAggregateClusterMetrics());
-    return aggregateClusterMetrics;
-  }
-
   @Override
   @Override
-  protected Long getSleepIntervalMillis() {
-    return sleepIntervalMillis;
-  }
+  protected void aggregate(ResultSet rs, long startTime, long endTime) throws IOException, SQLException {
+    Map<TimelineClusterMetric, MetricHostAggregate> hostAggregateMap = aggregateMetricsFromResultSet(rs);
 
 
-  @Override
-  protected Integer getCheckpointCutOffMultiplier() {
-    return checkpointCutOffMultiplier;
+    LOG.info("Saving " + hostAggregateMap.size() + " metric aggregates.");
+    hBaseAccessor.saveClusterTimeAggregateRecords(hostAggregateMap, outputTableName);
   }
   }
 
 
-  @Override
-  public boolean isDisabled() {
-    return metricsConf.getBoolean(CLUSTER_AGGREGATOR_MINUTE_DISABLED, false);
-  }
+  private Map<TimelineClusterMetric, MetricHostAggregate> aggregateMetricsFromResultSet(ResultSet rs)
+    throws IOException, SQLException {
 
 
-  private Map<TimelineClusterMetric, Double> sliceFromTimelineMetric(
-        TimelineMetric timelineMetric, List<Long[]> timeSlices) {
+    TimelineClusterMetric existingMetric = null;
+    MetricHostAggregate hostAggregate = null;
+    Map<TimelineClusterMetric, MetricHostAggregate> hostAggregateMap =
+      new HashMap<TimelineClusterMetric, MetricHostAggregate>();
 
 
-    if (timelineMetric.getMetricValues().isEmpty()) {
-      return null;
-    }
+    while (rs.next()) {
+      TimelineClusterMetric currentMetric = readHelper.fromResultSet(rs);
+
+      MetricClusterAggregate currentHostAggregate =
+        isClusterPrecisionInputTable ?
+          readHelper.getMetricClusterAggregateFromResultSet(rs) :
+          readHelper.getMetricClusterTimeAggregateFromResultSet(rs);
+
+      if (existingMetric == null) {
+        // First row
+        existingMetric = currentMetric;
+        hostAggregate = new MetricHostAggregate();
+        hostAggregateMap.put(currentMetric, hostAggregate);
+      }
 
 
-    Map<TimelineClusterMetric, Double> timelineClusterMetricMap =
-      new HashMap<TimelineClusterMetric, Double>();
+      if (existingMetric.equalsExceptTime(currentMetric)) {
+        // Recalculate totals with current metric
+        updateAggregatesFromHost(hostAggregate, currentHostAggregate);
 
 
-    for (Map.Entry<Long, Double> metric : timelineMetric.getMetricValues().entrySet()) {
-      // TODO: investigate null values - pre filter
-      if (metric.getValue() == null) {
-        continue;
-      }
-      Long timestamp = getSliceTimeForMetric(timeSlices,
-                       Long.parseLong(metric.getKey().toString()));
-      if (timestamp != -1) {
-        // Metric is within desired time range
-        TimelineClusterMetric clusterMetric = new TimelineClusterMetric(
-          timelineMetric.getMetricName(),
-          timelineMetric.getAppId(),
-          timelineMetric.getInstanceId(),
-          timestamp,
-          timelineMetric.getType());
-        if (!timelineClusterMetricMap.containsKey(clusterMetric)) {
-          timelineClusterMetricMap.put(clusterMetric, metric.getValue());
-        } else {
-          Double oldValue = timelineClusterMetricMap.get(clusterMetric);
-          Double newValue = (oldValue + metric.getValue()) / 2;
-          timelineClusterMetricMap.put(clusterMetric, newValue);
-        }
+      } else {
+        // Switched over to a new metric - save existing
+        hostAggregate = new MetricHostAggregate();
+        updateAggregatesFromHost(hostAggregate, currentHostAggregate);
+        hostAggregateMap.put(currentMetric, hostAggregate);
+        existingMetric = currentMetric;
       }
       }
+
     }
     }
 
 
-    return timelineClusterMetricMap;
+    return hostAggregateMap;
   }
   }
 
 
-  /**
-   * Return beginning of the time slice into which the metric fits.
-   */
-  private Long getSliceTimeForMetric(List<Long[]> timeSlices, Long timestamp) {
-    for (Long[] timeSlice : timeSlices) {
-      if (timestamp >= timeSlice[0] && timestamp < timeSlice[1]) {
-        return timeSlice[0];
-      }
-    }
-    return -1l;
+  private void updateAggregatesFromHost(MetricHostAggregate agg, MetricClusterAggregate currentClusterAggregate) {
+    agg.updateMax(currentClusterAggregate.getMax());
+    agg.updateMin(currentClusterAggregate.getMin());
+    agg.updateSum(currentClusterAggregate.getSum());
+    agg.updateNumberOfSamples(currentClusterAggregate.getNumberOfHosts());
   }
   }
-
 }
 }

+ 0 - 175
ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineMetricClusterAggregatorHourly.java

@@ -1,175 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators;
-
-import org.apache.commons.io.FilenameUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixHBaseAccessor;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.Condition;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.DefaultCondition;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL;
-import java.io.IOException;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.HashMap;
-import java.util.Map;
-import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixHBaseAccessor.getMetricClusterAggregateFromResultSet;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.GET_CLUSTER_AGGREGATE_SQL;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_INTERVAL;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_MULTIPLIER;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_HOUR_DISABLED;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_HOUR_SLEEP_INTERVAL;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.DEFAULT_CHECKPOINT_LOCATION;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR;
-
-public class TimelineMetricClusterAggregatorHourly extends AbstractTimelineAggregator {
-  private static final Log LOG = LogFactory.getLog
-    (TimelineMetricClusterAggregatorHourly.class);
-  private static final String CLUSTER_AGGREGATOR_HOURLY_CHECKPOINT_FILE =
-    "timeline-metrics-cluster-aggregator-hourly-checkpoint";
-  private final String checkpointLocation;
-  private final long sleepIntervalMillis;
-  private final Integer checkpointCutOffMultiplier;
-  private long checkpointCutOffIntervalMillis;
-  private static final Long NATIVE_TIME_RANGE_DELTA = 3600000l; // 1 hour
-  private final TimelineClusterMetricReader timelineClusterMetricReader
-     = new TimelineClusterMetricReader(true);
-
-  public TimelineMetricClusterAggregatorHourly(
-    PhoenixHBaseAccessor hBaseAccessor, Configuration metricsConf) {
-    super(hBaseAccessor, metricsConf);
-
-    String checkpointDir = metricsConf.get(
-      TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR, DEFAULT_CHECKPOINT_LOCATION);
-
-    checkpointLocation = FilenameUtils.concat(checkpointDir,
-      CLUSTER_AGGREGATOR_HOURLY_CHECKPOINT_FILE);
-
-    sleepIntervalMillis = SECONDS.toMillis(metricsConf.getLong
-      (CLUSTER_AGGREGATOR_HOUR_SLEEP_INTERVAL, 3600l));
-    checkpointCutOffIntervalMillis =  SECONDS.toMillis(metricsConf.getLong
-      (CLUSTER_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_INTERVAL, 7200l));
-    checkpointCutOffMultiplier = metricsConf.getInt
-      (CLUSTER_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_MULTIPLIER, 2);
-  }
-
-  @Override
-  protected String getCheckpointLocation() {
-    return checkpointLocation;
-  }
-
-  @Override
-  protected void aggregate(ResultSet rs, long startTime, long endTime)
-    throws SQLException, IOException {
-      Map<TimelineClusterMetric, MetricHostAggregate> hostAggregateMap =
-        aggregateMetricsFromResultSet(rs);
-
-    LOG.info("Saving " + hostAggregateMap.size() + " metric aggregates.");
-    hBaseAccessor.saveClusterAggregateHourlyRecords(hostAggregateMap,
-      METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME);
-  }
-
-  @Override
-  protected Condition prepareMetricQueryCondition(long startTime,
-                                                  long endTime) {
-    Condition condition = new DefaultCondition(null, null, null, null, startTime,
-      endTime, null, null, true);
-    condition.setNoLimit();
-    condition.setFetchSize(resultsetFetchSize);
-    condition.setStatement(String.format(GET_CLUSTER_AGGREGATE_SQL,
-      PhoenixTransactSQL.getNaiveTimeRangeHint(startTime, NATIVE_TIME_RANGE_DELTA),
-        METRICS_CLUSTER_AGGREGATE_TABLE_NAME));
-    condition.addOrderByColumn("METRIC_NAME");
-    condition.addOrderByColumn("APP_ID");
-    condition.addOrderByColumn("INSTANCE_ID");
-    condition.addOrderByColumn("SERVER_TIME");
-    return condition;
-  }
-
-  private Map<TimelineClusterMetric, MetricHostAggregate> aggregateMetricsFromResultSet(ResultSet rs)
-      throws IOException, SQLException {
-
-    TimelineClusterMetric existingMetric = null;
-    MetricHostAggregate hostAggregate = null;
-    Map<TimelineClusterMetric, MetricHostAggregate> hostAggregateMap =
-      new HashMap<TimelineClusterMetric, MetricHostAggregate>();
-
-    while (rs.next()) {
-      TimelineClusterMetric currentMetric =
-        timelineClusterMetricReader.fromResultSet(rs);
-      MetricClusterAggregate currentHostAggregate =
-        getMetricClusterAggregateFromResultSet(rs);
-
-      if (existingMetric == null) {
-        // First row
-        existingMetric = currentMetric;
-        hostAggregate = new MetricHostAggregate();
-        hostAggregateMap.put(currentMetric, hostAggregate);
-      }
-
-      if (existingMetric.equalsExceptTime(currentMetric)) {
-        // Recalculate totals with current metric
-        updateAggregatesFromHost(hostAggregate, currentHostAggregate);
-
-      } else {
-        // Switched over to a new metric - save existing
-        hostAggregate = new MetricHostAggregate();
-        updateAggregatesFromHost(hostAggregate, currentHostAggregate);
-        hostAggregateMap.put(currentMetric, hostAggregate);
-        existingMetric = currentMetric;
-      }
-
-    }
-
-    return hostAggregateMap;
-  }
-
-  private void updateAggregatesFromHost(MetricHostAggregate agg, MetricClusterAggregate currentClusterAggregate) {
-    agg.updateMax(currentClusterAggregate.getMax());
-    agg.updateMin(currentClusterAggregate.getMin());
-    agg.updateSum(currentClusterAggregate.getSum());
-    agg.updateNumberOfSamples(currentClusterAggregate.getNumberOfHosts());
-  }
-
-  @Override
-  protected Long getSleepIntervalMillis() {
-    return sleepIntervalMillis;
-  }
-
-  @Override
-  protected Integer getCheckpointCutOffMultiplier() {
-    return checkpointCutOffMultiplier;
-  }
-
-  @Override
-  protected Long getCheckpointCutOffIntervalMillis() {
-    return checkpointCutOffIntervalMillis;
-  }
-
-  @Override
-  public boolean isDisabled() {
-    return metricsConf.getBoolean(CLUSTER_AGGREGATOR_HOUR_DISABLED, false);
-  }
-
-
-}

+ 201 - 0
ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineMetricClusterAggregatorMinute.java

@@ -0,0 +1,201 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators;
+
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixHBaseAccessor;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.Condition;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.DefaultCondition;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL;
+import java.io.IOException;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.GET_METRIC_SQL;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_RECORD_TABLE_NAME;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.NATIVE_TIME_RANGE_DELTA;
+
+/**
+ * Aggregates a metric across all hosts in the cluster. Reads metrics from
+ * the precision table and saves into the aggregate.
+ */
+public class TimelineMetricClusterAggregatorMinute extends AbstractTimelineAggregator {
+  private static final Log LOG = LogFactory.getLog(TimelineMetricClusterAggregatorMinute.class);
+  public Long timeSliceIntervalMillis;
+  private TimelineMetricReadHelper timelineMetricReadHelper = new TimelineMetricReadHelper(true);
+  // Aggregator to perform app-level aggregates for host metrics
+  private final TimelineMetricAppAggregator appAggregator;
+
+  public TimelineMetricClusterAggregatorMinute(PhoenixHBaseAccessor hBaseAccessor,
+                                               Configuration metricsConf,
+                                               String checkpointLocation,
+                                               Long sleepIntervalMillis,
+                                               Integer checkpointCutOffMultiplier,
+                                               String aggregatorDisabledParam,
+                                               String tableName,
+                                               String outputTableName,
+                                               Long nativeTimeRangeDelay,
+                                               Long timeSliceInterval) {
+    super(hBaseAccessor, metricsConf, checkpointLocation, sleepIntervalMillis,
+      checkpointCutOffMultiplier, aggregatorDisabledParam, tableName,
+      outputTableName, nativeTimeRangeDelay);
+
+    appAggregator = new TimelineMetricAppAggregator(metricsConf);
+    this.timeSliceIntervalMillis = timeSliceInterval;
+  }
+
+  @Override
+  protected void aggregate(ResultSet rs, long startTime, long endTime) throws SQLException, IOException {
+    List<Long[]> timeSlices = getTimeSlices(startTime, endTime);
+    // Initialize app aggregates for host metrics
+    appAggregator.init();
+    Map<TimelineClusterMetric, MetricClusterAggregate> aggregateClusterMetrics =
+      aggregateMetricsFromResultSet(rs, timeSlices);
+
+    LOG.info("Saving " + aggregateClusterMetrics.size() + " metric aggregates.");
+    hBaseAccessor.saveClusterAggregateRecords(aggregateClusterMetrics);
+    appAggregator.cleanup();
+  }
+
+  @Override
+  protected Condition prepareMetricQueryCondition(long startTime, long endTime) {
+    Condition condition = new DefaultCondition(null, null, null, null, startTime,
+      endTime, null, null, true);
+    condition.setNoLimit();
+    condition.setFetchSize(resultsetFetchSize);
+    condition.setStatement(String.format(GET_METRIC_SQL,
+      PhoenixTransactSQL.getNaiveTimeRangeHint(startTime, NATIVE_TIME_RANGE_DELTA),
+      METRICS_RECORD_TABLE_NAME));
+    condition.addOrderByColumn("METRIC_NAME");
+    condition.addOrderByColumn("APP_ID");
+    condition.addOrderByColumn("INSTANCE_ID");
+    condition.addOrderByColumn("SERVER_TIME");
+    return condition;
+  }
+
+  private List<Long[]> getTimeSlices(long startTime, long endTime) {
+    List<Long[]> timeSlices = new ArrayList<Long[]>();
+    long sliceStartTime = startTime;
+    while (sliceStartTime < endTime) {
+      timeSlices.add(new Long[] { sliceStartTime, sliceStartTime + timeSliceIntervalMillis});
+      sliceStartTime += timeSliceIntervalMillis;
+    }
+    return timeSlices;
+  }
+
+  private Map<TimelineClusterMetric, MetricClusterAggregate> aggregateMetricsFromResultSet(ResultSet rs, List<Long[]> timeSlices)
+      throws SQLException, IOException {
+    Map<TimelineClusterMetric, MetricClusterAggregate> aggregateClusterMetrics =
+      new HashMap<TimelineClusterMetric, MetricClusterAggregate>();
+    // Create time slices
+
+    while (rs.next()) {
+      TimelineMetric metric = timelineMetricReadHelper.getTimelineMetricFromResultSet(rs);
+
+      Map<TimelineClusterMetric, Double> clusterMetrics =
+        sliceFromTimelineMetric(metric, timeSlices);
+
+      if (clusterMetrics != null && !clusterMetrics.isEmpty()) {
+        for (Map.Entry<TimelineClusterMetric, Double> clusterMetricEntry :
+            clusterMetrics.entrySet()) {
+
+          TimelineClusterMetric clusterMetric = clusterMetricEntry.getKey();
+          Double avgValue = clusterMetricEntry.getValue();
+
+          MetricClusterAggregate aggregate = aggregateClusterMetrics.get(clusterMetric);
+
+          if (aggregate == null) {
+            aggregate = new MetricClusterAggregate(avgValue, 1, null, avgValue, avgValue);
+            aggregateClusterMetrics.put(clusterMetric, aggregate);
+          } else {
+            aggregate.updateSum(avgValue);
+            aggregate.updateNumberOfHosts(1);
+            aggregate.updateMax(avgValue);
+            aggregate.updateMin(avgValue);
+          }
+          // Update app level aggregates
+          appAggregator.processTimelineClusterMetric(clusterMetric,
+            metric.getHostName(), avgValue);
+        }
+      }
+    }
+    // Add app level aggregates to save
+    aggregateClusterMetrics.putAll(appAggregator.getAggregateClusterMetrics());
+    return aggregateClusterMetrics;
+  }
+
+  private Map<TimelineClusterMetric, Double> sliceFromTimelineMetric(
+        TimelineMetric timelineMetric, List<Long[]> timeSlices) {
+
+    if (timelineMetric.getMetricValues().isEmpty()) {
+      return null;
+    }
+
+    Map<TimelineClusterMetric, Double> timelineClusterMetricMap =
+      new HashMap<TimelineClusterMetric, Double>();
+
+    for (Map.Entry<Long, Double> metric : timelineMetric.getMetricValues().entrySet()) {
+      // TODO: investigate null values - pre filter
+      if (metric.getValue() == null) {
+        continue;
+      }
+      Long timestamp = getSliceTimeForMetric(timeSlices,
+                       Long.parseLong(metric.getKey().toString()));
+      if (timestamp != -1) {
+        // Metric is within desired time range
+        TimelineClusterMetric clusterMetric = new TimelineClusterMetric(
+          timelineMetric.getMetricName(),
+          timelineMetric.getAppId(),
+          timelineMetric.getInstanceId(),
+          timestamp,
+          timelineMetric.getType());
+        if (!timelineClusterMetricMap.containsKey(clusterMetric)) {
+          timelineClusterMetricMap.put(clusterMetric, metric.getValue());
+        } else {
+          Double oldValue = timelineClusterMetricMap.get(clusterMetric);
+          Double newValue = (oldValue + metric.getValue()) / 2;
+          timelineClusterMetricMap.put(clusterMetric, newValue);
+        }
+      }
+    }
+
+    return timelineClusterMetricMap;
+  }
+
+  /**
+   * Return beginning of the time slice into which the metric fits.
+   */
+  private Long getSliceTimeForMetric(List<Long[]> timeSlices, Long timestamp) {
+    for (Long[] timeSlice : timeSlices) {
+      if (timestamp >= timeSlice[0] && timestamp < timeSlice[1]) {
+        return timeSlice[0];
+      }
+    }
+    return -1l;
+  }
+
+}

+ 113 - 0
ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineMetricHostAggregator.java

@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixHBaseAccessor;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.Condition;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.DefaultCondition;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL;
+import java.io.IOException;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.Map;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.GET_METRIC_AGGREGATE_ONLY_SQL;
+
+public class TimelineMetricHostAggregator extends AbstractTimelineAggregator {
+  private static final Log LOG = LogFactory.getLog(TimelineMetricHostAggregator.class);
+
+  public TimelineMetricHostAggregator(PhoenixHBaseAccessor hBaseAccessor,
+                                      Configuration metricsConf,
+                                      String checkpointLocation,
+                                      Long sleepIntervalMillis,
+                                      Integer checkpointCutOffMultiplier,
+                                      String hostAggregatorDisabledParam,
+                                      String tableName,
+                                      String outputTableName,
+                                      Long nativeTimeRangeDelay) {
+    super(hBaseAccessor, metricsConf, checkpointLocation, sleepIntervalMillis,
+      checkpointCutOffMultiplier, hostAggregatorDisabledParam, tableName,
+      outputTableName, nativeTimeRangeDelay);
+  }
+
+  @Override
+  protected void aggregate(ResultSet rs, long startTime, long endTime) throws IOException, SQLException {
+
+    Map<TimelineMetric, MetricHostAggregate> hostAggregateMap = aggregateMetricsFromResultSet(rs);
+
+    LOG.info("Saving " + hostAggregateMap.size() + " metric aggregates.");
+    hBaseAccessor.saveHostAggregateRecords(hostAggregateMap, outputTableName);
+  }
+
+  @Override
+  protected Condition prepareMetricQueryCondition(long startTime, long endTime) {
+    Condition condition = new DefaultCondition(null, null, null, null, startTime,
+      endTime, null, null, true);
+    condition.setNoLimit();
+    condition.setFetchSize(resultsetFetchSize);
+    condition.setStatement(String.format(GET_METRIC_AGGREGATE_ONLY_SQL,
+      PhoenixTransactSQL.getNaiveTimeRangeHint(startTime, nativeTimeRangeDelay),
+      tableName));
+    condition.addOrderByColumn("METRIC_NAME");
+    condition.addOrderByColumn("HOSTNAME");
+    condition.addOrderByColumn("APP_ID");
+    condition.addOrderByColumn("INSTANCE_ID");
+    condition.addOrderByColumn("SERVER_TIME");
+    return condition;
+  }
+
+  private Map<TimelineMetric, MetricHostAggregate> aggregateMetricsFromResultSet(ResultSet rs)
+      throws IOException, SQLException {
+    TimelineMetric existingMetric = null;
+    MetricHostAggregate hostAggregate = null;
+    Map<TimelineMetric, MetricHostAggregate> hostAggregateMap =
+      new HashMap<TimelineMetric, MetricHostAggregate>();
+
+    while (rs.next()) {
+      TimelineMetric currentMetric =
+        PhoenixHBaseAccessor.getTimelineMetricKeyFromResultSet(rs);
+      MetricHostAggregate currentHostAggregate =
+        PhoenixHBaseAccessor.getMetricHostAggregateFromResultSet(rs);
+
+      if (existingMetric == null) {
+        // First row
+        existingMetric = currentMetric;
+        hostAggregate = new MetricHostAggregate();
+        hostAggregateMap.put(currentMetric, hostAggregate);
+      }
+
+      if (existingMetric.equalsExceptTime(currentMetric)) {
+        // Recalculate totals with current metric
+        hostAggregate.updateAggregates(currentHostAggregate);
+      } else {
+        // Switched over to a new metric - save existing - create new aggregate
+        hostAggregate = new MetricHostAggregate();
+        hostAggregate.updateAggregates(currentHostAggregate);
+        hostAggregateMap.put(currentMetric, hostAggregate);
+        existingMetric = currentMetric;
+      }
+    }
+    return hostAggregateMap;
+  }
+
+
+}

+ 36 - 0
ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/aggregators/TimelineMetricReadHelper.java

@@ -62,5 +62,41 @@ public class TimelineMetricReadHelper {
     return metric;
     return metric;
   }
   }
 
 
+  public MetricClusterAggregate getMetricClusterAggregateFromResultSet(ResultSet rs)
+      throws SQLException {
+    MetricClusterAggregate agg = new MetricClusterAggregate();
+    agg.setSum(rs.getDouble("METRIC_SUM"));
+    agg.setMax(rs.getDouble("METRIC_MAX"));
+    agg.setMin(rs.getDouble("METRIC_MIN"));
+    agg.setNumberOfHosts(rs.getInt("HOSTS_COUNT"));
+
+    agg.setDeviation(0.0);
+
+    return agg;
+  }
+
+  public MetricClusterAggregate getMetricClusterTimeAggregateFromResultSet(ResultSet rs)
+    throws SQLException {
+    MetricClusterAggregate agg = new MetricClusterAggregate();
+    agg.setSum(rs.getDouble("METRIC_SUM"));
+    agg.setMax(rs.getDouble("METRIC_MAX"));
+    agg.setMin(rs.getDouble("METRIC_MIN"));
+    agg.setNumberOfHosts(rs.getInt("METRIC_COUNT"));
+
+    agg.setDeviation(0.0);
+
+    return agg;
+  }
+
+
+  public TimelineClusterMetric fromResultSet(ResultSet rs) throws SQLException {
+    return new TimelineClusterMetric(
+      rs.getString("METRIC_NAME"),
+      rs.getString("APP_ID"),
+      ignoreInstance ? null : rs.getString("INSTANCE_ID"),
+      rs.getLong("SERVER_TIME"),
+      rs.getString("UNITS"));
+  }
+
 }
 }
 
 

+ 44 - 38
ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/query/PhoenixTransactSQL.java

@@ -58,24 +58,8 @@ public class PhoenixTransactSQL {
     "INSTANCE_ID)) DATA_BLOCK_ENCODING='%s', IMMUTABLE_ROWS=true, " +
     "INSTANCE_ID)) DATA_BLOCK_ENCODING='%s', IMMUTABLE_ROWS=true, " +
     "TTL=%s, COMPRESSION='%s'";
     "TTL=%s, COMPRESSION='%s'";
 
 
-  public static final String CREATE_METRICS_AGGREGATE_HOURLY_TABLE_SQL =
-    "CREATE TABLE IF NOT EXISTS METRIC_RECORD_HOURLY " +
-      "(METRIC_NAME VARCHAR, " +
-      "HOSTNAME VARCHAR, " +
-      "APP_ID VARCHAR, " +
-      "INSTANCE_ID VARCHAR, " +
-      "SERVER_TIME UNSIGNED_LONG NOT NULL, " +
-      "UNITS CHAR(20), " +
-      "METRIC_SUM DOUBLE," +
-      "METRIC_COUNT UNSIGNED_INT, " +
-      "METRIC_MAX DOUBLE," +
-      "METRIC_MIN DOUBLE CONSTRAINT pk " +
-      "PRIMARY KEY (METRIC_NAME, HOSTNAME, APP_ID, INSTANCE_ID, " +
-      "SERVER_TIME)) DATA_BLOCK_ENCODING='%s', IMMUTABLE_ROWS=true, " +
-      "TTL=%s, COMPRESSION='%s'";
-
-  public static final String CREATE_METRICS_AGGREGATE_MINUTE_TABLE_SQL =
-    "CREATE TABLE IF NOT EXISTS METRIC_RECORD_MINUTE " +
+  public static final String CREATE_METRICS_AGGREGATE_TABLE_SQL =
+    "CREATE TABLE IF NOT EXISTS %s " +
       "(METRIC_NAME VARCHAR, " +
       "(METRIC_NAME VARCHAR, " +
       "HOSTNAME VARCHAR, " +
       "HOSTNAME VARCHAR, " +
       "APP_ID VARCHAR, " +
       "APP_ID VARCHAR, " +
@@ -91,7 +75,7 @@ public class PhoenixTransactSQL {
       " COMPRESSION='%s'";
       " COMPRESSION='%s'";
 
 
   public static final String CREATE_METRICS_CLUSTER_AGGREGATE_TABLE_SQL =
   public static final String CREATE_METRICS_CLUSTER_AGGREGATE_TABLE_SQL =
-    "CREATE TABLE IF NOT EXISTS METRIC_AGGREGATE " +
+    "CREATE TABLE IF NOT EXISTS %s " +
       "(METRIC_NAME VARCHAR, " +
       "(METRIC_NAME VARCHAR, " +
       "APP_ID VARCHAR, " +
       "APP_ID VARCHAR, " +
       "INSTANCE_ID VARCHAR, " +
       "INSTANCE_ID VARCHAR, " +
@@ -105,8 +89,9 @@ public class PhoenixTransactSQL {
       "SERVER_TIME)) DATA_BLOCK_ENCODING='%s', IMMUTABLE_ROWS=true, " +
       "SERVER_TIME)) DATA_BLOCK_ENCODING='%s', IMMUTABLE_ROWS=true, " +
       "TTL=%s, COMPRESSION='%s'";
       "TTL=%s, COMPRESSION='%s'";
 
 
+  // HOSTS_COUNT vs METRIC_COUNT
   public static final String CREATE_METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_SQL =
   public static final String CREATE_METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_SQL =
-    "CREATE TABLE IF NOT EXISTS METRIC_AGGREGATE_HOURLY " +
+    "CREATE TABLE IF NOT EXISTS %s " +
       "(METRIC_NAME VARCHAR, " +
       "(METRIC_NAME VARCHAR, " +
       "APP_ID VARCHAR, " +
       "APP_ID VARCHAR, " +
       "INSTANCE_ID VARCHAR, " +
       "INSTANCE_ID VARCHAR, " +
@@ -139,7 +124,7 @@ public class PhoenixTransactSQL {
     "(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     "(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
 
   public static final String UPSERT_CLUSTER_AGGREGATE_SQL = "UPSERT INTO " +
   public static final String UPSERT_CLUSTER_AGGREGATE_SQL = "UPSERT INTO " +
-    "METRIC_AGGREGATE (METRIC_NAME, APP_ID, INSTANCE_ID, SERVER_TIME, " +
+    "%s (METRIC_NAME, APP_ID, INSTANCE_ID, SERVER_TIME, " +
     "UNITS, " +
     "UNITS, " +
     "METRIC_SUM, " +
     "METRIC_SUM, " +
     "HOSTS_COUNT, " +
     "HOSTS_COUNT, " +
@@ -156,7 +141,6 @@ public class PhoenixTransactSQL {
     "METRIC_MIN) " +
     "METRIC_MIN) " +
     "VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
     "VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
 
-
   public static final String UPSERT_AGGREGATE_RECORD_SQL = "UPSERT INTO " +
   public static final String UPSERT_AGGREGATE_RECORD_SQL = "UPSERT INTO " +
     "%s (METRIC_NAME, HOSTNAME, APP_ID, INSTANCE_ID, " +
     "%s (METRIC_NAME, HOSTNAME, APP_ID, INSTANCE_ID, " +
     "SERVER_TIME, " +
     "SERVER_TIME, " +
@@ -198,25 +182,29 @@ public class PhoenixTransactSQL {
     "METRIC_MIN " +
     "METRIC_MIN " +
     "FROM %s";
     "FROM %s";
 
 
-  public static final String GET_CLUSTER_AGGREGATE_HOURLY_SQL = "SELECT %s " +
-      "METRIC_NAME, APP_ID, " +
-      "INSTANCE_ID, SERVER_TIME, " +
-      "UNITS, " +
-      "METRIC_SUM, " +
-      "METRIC_COUNT, " +
-      "METRIC_MAX, " +
-      "METRIC_MIN " +
-      "FROM %s";
+  public static final String GET_CLUSTER_AGGREGATE_TIME_SQL = "SELECT %s " +
+    "METRIC_NAME, APP_ID, " +
+    "INSTANCE_ID, SERVER_TIME, " +
+    "UNITS, " +
+    "METRIC_SUM, " +
+    "METRIC_COUNT, " +
+    "METRIC_MAX, " +
+    "METRIC_MIN " +
+    "FROM %s";
 
 
   public static final String METRICS_RECORD_TABLE_NAME = "METRIC_RECORD";
   public static final String METRICS_RECORD_TABLE_NAME = "METRIC_RECORD";
   public static final String METRICS_AGGREGATE_MINUTE_TABLE_NAME =
   public static final String METRICS_AGGREGATE_MINUTE_TABLE_NAME =
     "METRIC_RECORD_MINUTE";
     "METRIC_RECORD_MINUTE";
   public static final String METRICS_AGGREGATE_HOURLY_TABLE_NAME =
   public static final String METRICS_AGGREGATE_HOURLY_TABLE_NAME =
     "METRIC_RECORD_HOURLY";
     "METRIC_RECORD_HOURLY";
+  public static final String METRICS_AGGREGATE_DAILY_TABLE_NAME =
+    "METRIC_RECORD_DAILY";
   public static final String METRICS_CLUSTER_AGGREGATE_TABLE_NAME =
   public static final String METRICS_CLUSTER_AGGREGATE_TABLE_NAME =
     "METRIC_AGGREGATE";
     "METRIC_AGGREGATE";
   public static final String METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME =
   public static final String METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME =
     "METRIC_AGGREGATE_HOURLY";
     "METRIC_AGGREGATE_HOURLY";
+  public static final String METRICS_CLUSTER_AGGREGATE_DAILY_TABLE_NAME =
+    "METRIC_AGGREGATE_DAILY";
   public static final String DEFAULT_TABLE_COMPRESSION = "SNAPPY";
   public static final String DEFAULT_TABLE_COMPRESSION = "SNAPPY";
   public static final String DEFAULT_ENCODING = "FAST_DIFF";
   public static final String DEFAULT_ENCODING = "FAST_DIFF";
   public static final long NATIVE_TIME_RANGE_DELTA = 120000; // 2 minutes
   public static final long NATIVE_TIME_RANGE_DELTA = 120000; // 2 minutes
@@ -250,7 +238,11 @@ public class PhoenixTransactSQL {
         long endTime = condition.getEndTime() == null ? System.currentTimeMillis() : condition.getEndTime();
         long endTime = condition.getEndTime() == null ? System.currentTimeMillis() : condition.getEndTime();
         long startTime = condition.getStartTime() == null ? 0 : condition.getStartTime();
         long startTime = condition.getStartTime() == null ? 0 : condition.getStartTime();
         Long timeRange = endTime - startTime;
         Long timeRange = endTime - startTime;
-        if (timeRange > 5 * DAY) {
+        if (timeRange > 7 * DAY) {
+          metricsTable = METRICS_AGGREGATE_DAILY_TABLE_NAME;
+          query = GET_METRIC_AGGREGATE_ONLY_SQL;
+          condition.setPrecision(Precision.DAYS);
+        } else if (timeRange < 7 * DAY && timeRange > DAY) {
           metricsTable = METRICS_AGGREGATE_HOURLY_TABLE_NAME;
           metricsTable = METRICS_AGGREGATE_HOURLY_TABLE_NAME;
           query = GET_METRIC_AGGREGATE_ONLY_SQL;
           query = GET_METRIC_AGGREGATE_ONLY_SQL;
           condition.setPrecision(Precision.HOURS);
           condition.setPrecision(Precision.HOURS);
@@ -265,6 +257,10 @@ public class PhoenixTransactSQL {
         }
         }
       } else {
       } else {
         switch (condition.getPrecision()) {
         switch (condition.getPrecision()) {
+          case DAYS:
+            metricsTable = METRICS_AGGREGATE_DAILY_TABLE_NAME;
+            query = GET_METRIC_AGGREGATE_ONLY_SQL;
+            break;
           case HOURS:
           case HOURS:
             metricsTable = METRICS_AGGREGATE_HOURLY_TABLE_NAME;
             metricsTable = METRICS_AGGREGATE_HOURLY_TABLE_NAME;
             query = GET_METRIC_AGGREGATE_ONLY_SQL;
             query = GET_METRIC_AGGREGATE_ONLY_SQL;
@@ -462,9 +458,13 @@ public class PhoenixTransactSQL {
       long endTime = condition.getEndTime() == null ? System.currentTimeMillis() : condition.getEndTime();
       long endTime = condition.getEndTime() == null ? System.currentTimeMillis() : condition.getEndTime();
       long startTime = condition.getStartTime() == null ? 0 : condition.getStartTime();
       long startTime = condition.getStartTime() == null ? 0 : condition.getStartTime();
       Long timeRange = endTime - startTime;
       Long timeRange = endTime - startTime;
-      if (timeRange > 5 * DAY) {
+      if (timeRange > 7 * DAY) {
+        metricsAggregateTable = METRICS_CLUSTER_AGGREGATE_DAILY_TABLE_NAME;
+        queryStmt = GET_CLUSTER_AGGREGATE_TIME_SQL;
+        condition.setPrecision(Precision.DAYS);
+      } else if (timeRange < 7 * DAY && timeRange > DAY) {
         metricsAggregateTable = METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME;
         metricsAggregateTable = METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME;
-        queryStmt = GET_CLUSTER_AGGREGATE_HOURLY_SQL;
+        queryStmt = GET_CLUSTER_AGGREGATE_TIME_SQL;
         condition.setPrecision(Precision.HOURS);
         condition.setPrecision(Precision.HOURS);
       } else {
       } else {
         metricsAggregateTable = METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
         metricsAggregateTable = METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
@@ -473,9 +473,13 @@ public class PhoenixTransactSQL {
       }
       }
     } else {
     } else {
       switch (condition.getPrecision()) {
       switch (condition.getPrecision()) {
+        case DAYS:
+          metricsAggregateTable = METRICS_CLUSTER_AGGREGATE_DAILY_TABLE_NAME;
+          queryStmt = GET_CLUSTER_AGGREGATE_TIME_SQL;
+          break;
         case HOURS:
         case HOURS:
           metricsAggregateTable = METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME;
           metricsAggregateTable = METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME;
-          queryStmt = GET_CLUSTER_AGGREGATE_HOURLY_SQL;
+          queryStmt = GET_CLUSTER_AGGREGATE_TIME_SQL;
           break;
           break;
         default:
         default:
           metricsAggregateTable = METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
           metricsAggregateTable = METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
@@ -483,6 +487,10 @@ public class PhoenixTransactSQL {
       }
       }
     }
     }
 
 
+    queryStmt = String.format(queryStmt,
+      getNaiveTimeRangeHint(condition.getStartTime(), NATIVE_TIME_RANGE_DELTA),
+      metricsAggregateTable);
+
     StringBuilder sb = new StringBuilder(queryStmt);
     StringBuilder sb = new StringBuilder(queryStmt);
     sb.append(" WHERE ");
     sb.append(" WHERE ");
     sb.append(condition.getConditionClause());
     sb.append(condition.getConditionClause());
@@ -491,9 +499,7 @@ public class PhoenixTransactSQL {
       sb.append(" LIMIT ").append(condition.getLimit());
       sb.append(" LIMIT ").append(condition.getLimit());
     }
     }
 
 
-    String query = String.format(sb.toString(),
-      PhoenixTransactSQL.getNaiveTimeRangeHint(condition.getStartTime(),
-        NATIVE_TIME_RANGE_DELTA), metricsAggregateTable);
+    String query = sb.toString();
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("SQL => " + query + ", condition => " + condition);
       LOG.debug("SQL => " + query + ", condition => " + condition);
     }
     }

+ 3 - 2
ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/AbstractMiniHBaseClusterTest.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.IntegrationTestingUtility;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.ConnectionProvider;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.ConnectionProvider;
 import org.apache.phoenix.hbase.index.write.IndexWriterUtils;
 import org.apache.phoenix.hbase.index.write.IndexWriterUtils;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.BaseTest;
@@ -49,11 +50,11 @@ public abstract class AbstractMiniHBaseClusterTest extends BaseTest {
   @BeforeClass
   @BeforeClass
   public static void doSetup() throws Exception {
   public static void doSetup() throws Exception {
     Map<String, String> props = getDefaultProps();
     Map<String, String> props = getDefaultProps();
+    props.put(IntegrationTestingUtility.IS_DISTRIBUTED_CLUSTER, "false");
     props.put(QueryServices.QUEUE_SIZE_ATTRIB, Integer.toString(5000));
     props.put(QueryServices.QUEUE_SIZE_ATTRIB, Integer.toString(5000));
     props.put(IndexWriterUtils.HTABLE_THREAD_KEY, Integer.toString(100));
     props.put(IndexWriterUtils.HTABLE_THREAD_KEY, Integer.toString(100));
     // Make a small batch size to test multiple calls to reserve sequences
     // Make a small batch size to test multiple calls to reserve sequences
-    props.put(QueryServices.SEQUENCE_CACHE_SIZE_ATTRIB,
-      Long.toString(BATCH_SIZE));
+    props.put(QueryServices.SEQUENCE_CACHE_SIZE_ATTRIB, Long.toString(BATCH_SIZE));
     // Must update config before starting server
     // Must update config before starting server
     setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
   }
   }

+ 74 - 25
ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/ITClusterAggregator.java

@@ -20,17 +20,18 @@ package org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.MetricClusterAggregate;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.MetricClusterAggregate;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.MetricHostAggregate;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineClusterMetric;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineClusterMetric;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineClusterMetricReader;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricClusterAggregator;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricClusterAggregatorHourly;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricAggregator;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricAggregatorFactory;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricReadHelper;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.Condition;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.Condition;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.DefaultCondition;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.DefaultCondition;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL;
 import org.junit.After;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
+
 import java.sql.Connection;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.ResultSet;
@@ -39,6 +40,7 @@ import java.sql.Statement;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
+
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertNotNull;
 import static junit.framework.Assert.assertNotNull;
 import static junit.framework.Assert.fail;
 import static junit.framework.Assert.fail;
@@ -46,14 +48,14 @@ import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.ti
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.MetricTestHelper.prepareSingleTimelineMetric;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.MetricTestHelper.prepareSingleTimelineMetric;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_APP_IDS;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_AGGREGATOR_APP_IDS;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.GET_CLUSTER_AGGREGATE_SQL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.GET_CLUSTER_AGGREGATE_SQL;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.NATIVE_TIME_RANGE_DELTA;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.NATIVE_TIME_RANGE_DELTA;
 
 
 public class ITClusterAggregator extends AbstractMiniHBaseClusterTest {
 public class ITClusterAggregator extends AbstractMiniHBaseClusterTest {
   private Connection conn;
   private Connection conn;
   private PhoenixHBaseAccessor hdb;
   private PhoenixHBaseAccessor hdb;
-  private final TimelineClusterMetricReader metricReader = new
-    TimelineClusterMetricReader(false);
+  private final TimelineMetricReadHelper metricReader = new TimelineMetricReadHelper(false);
 
 
   @Before
   @Before
   public void setUp() throws Exception {
   public void setUp() throws Exception {
@@ -83,8 +85,9 @@ public class ITClusterAggregator extends AbstractMiniHBaseClusterTest {
   @Test
   @Test
   public void testShouldAggregateClusterProperly() throws Exception {
   public void testShouldAggregateClusterProperly() throws Exception {
     // GIVEN
     // GIVEN
-    TimelineMetricClusterAggregator agg =
-      new TimelineMetricClusterAggregator(hdb, new Configuration());
+    TimelineMetricAggregator agg =
+      TimelineMetricAggregatorFactory.createTimelineClusterAggregatorMinute(hdb, new Configuration());
+    TimelineMetricReadHelper readHelper = new TimelineMetricReadHelper(false);
 
 
     long startTime = System.currentTimeMillis();
     long startTime = System.currentTimeMillis();
     long ctime = startTime;
     long ctime = startTime;
@@ -118,7 +121,7 @@ public class ITClusterAggregator extends AbstractMiniHBaseClusterTest {
     while (rs.next()) {
     while (rs.next()) {
       TimelineClusterMetric currentMetric = metricReader.fromResultSet(rs);
       TimelineClusterMetric currentMetric = metricReader.fromResultSet(rs);
       MetricClusterAggregate currentHostAggregate =
       MetricClusterAggregate currentHostAggregate =
-        PhoenixHBaseAccessor.getMetricClusterAggregateFromResultSet(rs);
+        readHelper.getMetricClusterAggregateFromResultSet(rs);
 
 
       if ("disk_free".equals(currentMetric.getMetricName())) {
       if ("disk_free".equals(currentMetric.getMetricName())) {
         assertEquals(2, currentHostAggregate.getNumberOfHosts());
         assertEquals(2, currentHostAggregate.getNumberOfHosts());
@@ -136,8 +139,9 @@ public class ITClusterAggregator extends AbstractMiniHBaseClusterTest {
   @Test
   @Test
   public void testShouldAggregateClusterIgnoringInstance() throws Exception {
   public void testShouldAggregateClusterIgnoringInstance() throws Exception {
     // GIVEN
     // GIVEN
-    TimelineMetricClusterAggregator agg =
-      new TimelineMetricClusterAggregator(hdb, new Configuration());
+    TimelineMetricAggregator agg =
+      TimelineMetricAggregatorFactory.createTimelineClusterAggregatorMinute(hdb, new Configuration());
+    TimelineMetricReadHelper readHelper = new TimelineMetricReadHelper(false);
 
 
     long startTime = System.currentTimeMillis();
     long startTime = System.currentTimeMillis();
     long ctime = startTime;
     long ctime = startTime;
@@ -188,7 +192,7 @@ public class ITClusterAggregator extends AbstractMiniHBaseClusterTest {
       TimelineClusterMetric currentMetric = metricReader.fromResultSet(rs);
       TimelineClusterMetric currentMetric = metricReader.fromResultSet(rs);
 //        PhoenixHBaseAccessor.getTimelineMetricClusterKeyFromResultSet(rs);
 //        PhoenixHBaseAccessor.getTimelineMetricClusterKeyFromResultSet(rs);
       MetricClusterAggregate currentHostAggregate =
       MetricClusterAggregate currentHostAggregate =
-        PhoenixHBaseAccessor.getMetricClusterAggregateFromResultSet(rs);
+        readHelper.getMetricClusterAggregateFromResultSet(rs);
 
 
       if ("disk_free".equals(currentMetric.getMetricName())) {
       if ("disk_free".equals(currentMetric.getMetricName())) {
         System.out.println("OUTPUT: " + currentMetric+" - " +
         System.out.println("OUTPUT: " + currentMetric+" - " +
@@ -205,11 +209,11 @@ public class ITClusterAggregator extends AbstractMiniHBaseClusterTest {
   }
   }
 
 
   @Test
   @Test
-  public void testShouldAggregateDifferentMetricsOnClusterProperly()
-    throws Exception {
+  public void testShouldAggregateDifferentMetricsOnClusterProperly() throws Exception {
     // GIVEN
     // GIVEN
-    TimelineMetricClusterAggregator agg =
-      new TimelineMetricClusterAggregator(hdb, new Configuration());
+    TimelineMetricAggregator agg =
+      TimelineMetricAggregatorFactory.createTimelineClusterAggregatorMinute(hdb, new Configuration());
+    TimelineMetricReadHelper readHelper = new TimelineMetricReadHelper(false);
 
 
     // here we put some metrics tha will be aggregated
     // here we put some metrics tha will be aggregated
     long startTime = System.currentTimeMillis();
     long startTime = System.currentTimeMillis();
@@ -249,7 +253,7 @@ public class ITClusterAggregator extends AbstractMiniHBaseClusterTest {
     while (rs.next()) {
     while (rs.next()) {
       TimelineClusterMetric currentMetric = metricReader.fromResultSet(rs);
       TimelineClusterMetric currentMetric = metricReader.fromResultSet(rs);
       MetricClusterAggregate currentHostAggregate =
       MetricClusterAggregate currentHostAggregate =
-        PhoenixHBaseAccessor.getMetricClusterAggregateFromResultSet(rs);
+        readHelper.getMetricClusterAggregateFromResultSet(rs);
 
 
       if ("disk_free".equals(currentMetric.getMetricName())) {
       if ("disk_free".equals(currentMetric.getMetricName())) {
         assertEquals(2, currentHostAggregate.getNumberOfHosts());
         assertEquals(2, currentHostAggregate.getNumberOfHosts());
@@ -269,12 +273,56 @@ public class ITClusterAggregator extends AbstractMiniHBaseClusterTest {
     }
     }
   }
   }
 
 
+  @Test
+  public void testAggregateDailyClusterMetrics() throws Exception {
+    // GIVEN
+    TimelineMetricAggregator agg =
+      TimelineMetricAggregatorFactory.createTimelineClusterAggregatorDaily(hdb, new Configuration());
+
+    // this time can be virtualized! or made independent from real clock
+    long startTime = System.currentTimeMillis();
+    long ctime = startTime;
+    long hour = 3600 * 1000;
+
+    Map<TimelineClusterMetric, MetricHostAggregate> records =
+      new HashMap<TimelineClusterMetric, MetricHostAggregate>();
+
+    records.put(createEmptyTimelineClusterMetric(ctime),
+      MetricTestHelper.createMetricHostAggregate(4.0, 0.0, 2, 4.0));
+    records.put(createEmptyTimelineClusterMetric(ctime += hour),
+      MetricTestHelper.createMetricHostAggregate(4.0, 0.0, 2, 4.0));
+    records.put(createEmptyTimelineClusterMetric(ctime += hour),
+      MetricTestHelper.createMetricHostAggregate(4.0, 0.0, 2, 4.0));
+    records.put(createEmptyTimelineClusterMetric(ctime += hour),
+      MetricTestHelper.createMetricHostAggregate(4.0, 0.0, 2, 4.0));
+
+
+    hdb.saveClusterTimeAggregateRecords(records, METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME);
+
+    // WHEN
+    agg.doWork(startTime, ctime + hour + 1000);
+
+    // THEN
+    ResultSet rs = executeQuery("SELECT * FROM METRIC_AGGREGATE_DAILY");
+    int count = 0;
+    while (rs.next()) {
+      assertEquals("METRIC_NAME", "disk_used", rs.getString("METRIC_NAME"));
+      assertEquals("APP_ID", "test_app", rs.getString("APP_ID"));
+      assertEquals("METRIC_SUM", 16.0, rs.getDouble("METRIC_SUM"));
+      assertEquals("METRIC_COUNT", 8, rs.getLong("METRIC_COUNT"));
+      assertEquals("METRIC_MAX", 4.0, rs.getDouble("METRIC_MAX"));
+      assertEquals("METRIC_MIN", 0.0, rs.getDouble("METRIC_MIN"));
+      count++;
+    }
+
+    assertEquals("Day aggregated row expected ", 1, count);
+  }
 
 
   @Test
   @Test
   public void testShouldAggregateClusterOnHourProperly() throws Exception {
   public void testShouldAggregateClusterOnHourProperly() throws Exception {
     // GIVEN
     // GIVEN
-    TimelineMetricClusterAggregatorHourly agg =
-      new TimelineMetricClusterAggregatorHourly(hdb, new Configuration());
+    TimelineMetricAggregator agg =
+      TimelineMetricAggregatorFactory.createTimelineClusterAggregatorHourly(hdb, new Configuration());
 
 
     // this time can be virtualized! or made independent from real clock
     // this time can be virtualized! or made independent from real clock
     long startTime = System.currentTimeMillis();
     long startTime = System.currentTimeMillis();
@@ -315,11 +363,10 @@ public class ITClusterAggregator extends AbstractMiniHBaseClusterTest {
   }
   }
 
 
   @Test
   @Test
-  public void testShouldAggregateDifferentMetricsOnHourProperly() throws
-    Exception {
+  public void testShouldAggregateDifferentMetricsOnHourProperly() throws Exception {
     // GIVEN
     // GIVEN
-    TimelineMetricClusterAggregatorHourly agg =
-      new TimelineMetricClusterAggregatorHourly(hdb, new Configuration());
+    TimelineMetricAggregator agg =
+      TimelineMetricAggregatorFactory.createTimelineClusterAggregatorHourly(hdb, new Configuration());
 
 
     long startTime = System.currentTimeMillis();
     long startTime = System.currentTimeMillis();
     long ctime = startTime;
     long ctime = startTime;
@@ -381,7 +428,9 @@ public class ITClusterAggregator extends AbstractMiniHBaseClusterTest {
   public void testAppLevelHostMetricAggregates() throws Exception {
   public void testAppLevelHostMetricAggregates() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.set(CLUSTER_AGGREGATOR_APP_IDS, "app1");
     conf.set(CLUSTER_AGGREGATOR_APP_IDS, "app1");
-    TimelineMetricClusterAggregator agg = new TimelineMetricClusterAggregator(hdb, conf);
+    TimelineMetricAggregator agg =
+      TimelineMetricAggregatorFactory.createTimelineClusterAggregatorMinute(hdb, conf);
+    TimelineMetricReadHelper readHelper = new TimelineMetricReadHelper(false);
 
 
     long startTime = System.currentTimeMillis();
     long startTime = System.currentTimeMillis();
     long ctime = startTime;
     long ctime = startTime;
@@ -416,7 +465,7 @@ public class ITClusterAggregator extends AbstractMiniHBaseClusterTest {
     MetricClusterAggregate currentHostAggregate = null;
     MetricClusterAggregate currentHostAggregate = null;
     while (rs.next()) {
     while (rs.next()) {
       currentMetric = metricReader.fromResultSet(rs);
       currentMetric = metricReader.fromResultSet(rs);
-      currentHostAggregate = PhoenixHBaseAccessor.getMetricClusterAggregateFromResultSet(rs);
+      currentHostAggregate = readHelper.getMetricClusterAggregateFromResultSet(rs);
       recordCount++;
       recordCount++;
     }
     }
     assertEquals(4, recordCount);
     assertEquals(4, recordCount);

+ 70 - 12
ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/ITMetricAggregator.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
+
 import java.sql.Connection;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.ResultSet;
@@ -37,15 +38,17 @@ import java.util.Arrays;
 import java.util.Comparator;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
+
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertTrue;
 import static junit.framework.Assert.assertTrue;
 import static junit.framework.Assert.fail;
 import static junit.framework.Assert.fail;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.MetricTestHelper.createEmptyTimelineMetric;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.GET_METRIC_AGGREGATE_ONLY_SQL;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.GET_METRIC_AGGREGATE_ONLY_SQL;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_DAILY_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_HOURLY_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_HOURLY_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_MINUTE_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_MINUTE_TABLE_NAME;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.NATIVE_TIME_RANGE_DELTA;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.NATIVE_TIME_RANGE_DELTA;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.MetricTestHelper.createEmptyTimelineMetric;
 
 
 public class ITMetricAggregator extends AbstractMiniHBaseClusterTest {
 public class ITMetricAggregator extends AbstractMiniHBaseClusterTest {
   private Connection conn;
   private Connection conn;
@@ -102,10 +105,8 @@ public class ITMetricAggregator extends AbstractMiniHBaseClusterTest {
   @Test
   @Test
   public void testShouldAggregateMinuteProperly() throws Exception {
   public void testShouldAggregateMinuteProperly() throws Exception {
     // GIVEN
     // GIVEN
-//    TimelineMetricAggregatorMinute aggregatorMinute =
-//      new TimelineMetricAggregatorMinute(hdb, new Configuration());
-    TimelineMetricAggregator aggregatorMinute = TimelineMetricAggregatorFactory
-      .createTimelineMetricAggregatorMinute(hdb, new Configuration());
+    TimelineMetricAggregator aggregatorMinute =
+      TimelineMetricAggregatorFactory.createTimelineMetricAggregatorMinute(hdb, new Configuration());
 
 
     long startTime = System.currentTimeMillis();
     long startTime = System.currentTimeMillis();
     long ctime = startTime;
     long ctime = startTime;
@@ -162,17 +163,14 @@ public class ITMetricAggregator extends AbstractMiniHBaseClusterTest {
   }
   }
 
 
   @Test
   @Test
-  public void testShouldAggregateHourProperly() throws Exception {
+   public void testShouldAggregateHourProperly() throws Exception {
     // GIVEN
     // GIVEN
-//    TimelineMetricAggregatorHourly aggregator =
-//      new TimelineMetricAggregatorHourly(hdb, new Configuration());
-
-    TimelineMetricAggregator aggregator = TimelineMetricAggregatorFactory
-      .createTimelineMetricAggregatorHourly(hdb, new Configuration());
+    TimelineMetricAggregator aggregator =
+      TimelineMetricAggregatorFactory.createTimelineMetricAggregatorHourly(hdb, new Configuration());
     long startTime = System.currentTimeMillis();
     long startTime = System.currentTimeMillis();
 
 
     MetricHostAggregate expectedAggregate =
     MetricHostAggregate expectedAggregate =
-        MetricTestHelper.createMetricHostAggregate(2.0, 0.0, 20, 15.0);
+      MetricTestHelper.createMetricHostAggregate(2.0, 0.0, 20, 15.0);
     Map<TimelineMetric, MetricHostAggregate>
     Map<TimelineMetric, MetricHostAggregate>
       aggMap = new HashMap<TimelineMetric,
       aggMap = new HashMap<TimelineMetric,
       MetricHostAggregate>();
       MetricHostAggregate>();
@@ -226,6 +224,66 @@ public class ITMetricAggregator extends AbstractMiniHBaseClusterTest {
     }
     }
   }
   }
 
 
+  @Test
+  public void testMetricAggregateDaily() throws Exception {
+    // GIVEN
+    TimelineMetricAggregator aggregator =
+      TimelineMetricAggregatorFactory.createTimelineMetricAggregatorDaily(hdb, new Configuration());
+    long startTime = System.currentTimeMillis();
+
+    MetricHostAggregate expectedAggregate =
+      MetricTestHelper.createMetricHostAggregate(2.0, 0.0, 20, 15.0);
+    Map<TimelineMetric, MetricHostAggregate>
+      aggMap = new HashMap<TimelineMetric, MetricHostAggregate>();
+
+    int min_5 = 5 * 60 * 1000;
+    long ctime = startTime - min_5;
+    aggMap.put(createEmptyTimelineMetric(ctime += min_5), expectedAggregate);
+    aggMap.put(createEmptyTimelineMetric(ctime += min_5), expectedAggregate);
+    aggMap.put(createEmptyTimelineMetric(ctime += min_5), expectedAggregate);
+    aggMap.put(createEmptyTimelineMetric(ctime += min_5), expectedAggregate);
+    aggMap.put(createEmptyTimelineMetric(ctime += min_5), expectedAggregate);
+    aggMap.put(createEmptyTimelineMetric(ctime += min_5), expectedAggregate);
+    aggMap.put(createEmptyTimelineMetric(ctime += min_5), expectedAggregate);
+    aggMap.put(createEmptyTimelineMetric(ctime += min_5), expectedAggregate);
+    aggMap.put(createEmptyTimelineMetric(ctime += min_5), expectedAggregate);
+    aggMap.put(createEmptyTimelineMetric(ctime += min_5), expectedAggregate);
+    aggMap.put(createEmptyTimelineMetric(ctime += min_5), expectedAggregate);
+    aggMap.put(createEmptyTimelineMetric(ctime += min_5), expectedAggregate);
+
+    hdb.saveHostAggregateRecords(aggMap, METRICS_AGGREGATE_HOURLY_TABLE_NAME);
+
+    //WHEN
+    long endTime = ctime + min_5;
+    boolean success = aggregator.doWork(startTime, endTime);
+    assertTrue(success);
+
+    //THEN
+    Condition condition = new DefaultCondition(null, null, null, null, startTime,
+      endTime, null, null, true);
+    condition.setStatement(String.format(GET_METRIC_AGGREGATE_ONLY_SQL,
+      PhoenixTransactSQL.getNaiveTimeRangeHint(startTime, NATIVE_TIME_RANGE_DELTA),
+      METRICS_AGGREGATE_DAILY_TABLE_NAME));
+
+    PreparedStatement pstmt = PhoenixTransactSQL.prepareGetMetricsSqlStmt(conn, condition);
+    ResultSet rs = pstmt.executeQuery();
+
+    while (rs.next()) {
+      TimelineMetric currentMetric =
+        PhoenixHBaseAccessor.getTimelineMetricKeyFromResultSet(rs);
+      MetricHostAggregate currentHostAggregate =
+        PhoenixHBaseAccessor.getMetricHostAggregateFromResultSet(rs);
+
+      if ("disk_used".equals(currentMetric.getMetricName())) {
+        assertEquals(2.0, currentHostAggregate.getMax());
+        assertEquals(0.0, currentHostAggregate.getMin());
+        assertEquals(12 * 20, currentHostAggregate.getNumberOfSamples());
+        assertEquals(12 * 15.0, currentHostAggregate.getSum());
+        assertEquals(15.0 / 20, currentHostAggregate.getAvg());
+      }
+    }
+  }
+
   private final static Comparator<TimelineMetric> TIME_IGNORING_COMPARATOR =
   private final static Comparator<TimelineMetric> TIME_IGNORING_COMPARATOR =
     new Comparator<TimelineMetric>() {
     new Comparator<TimelineMetric>() {
       @Override
       @Override

+ 14 - 20
ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/ITPhoenixHBaseAccessor.java

@@ -26,15 +26,11 @@ import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineClusterMetric;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineClusterMetric;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricAggregator;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricAggregator;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricAggregatorFactory;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricAggregatorFactory;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricClusterAggregator;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricClusterAggregatorHourly;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.Condition;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.Condition;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.DefaultCondition;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.DefaultCondition;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
-
 import java.io.IOException;
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.Connection;
 import java.sql.SQLException;
 import java.sql.SQLException;
@@ -44,14 +40,14 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
-
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertTrue;
 import static junit.framework.Assert.assertTrue;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_MINUTE_TABLE_NAME;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.MetricTestHelper.prepareSingleTimelineMetric;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.MetricTestHelper.createEmptyTimelineClusterMetric;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.MetricTestHelper.createEmptyTimelineMetric;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.MetricTestHelper.createEmptyTimelineMetric;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.MetricTestHelper.createMetricHostAggregate;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.MetricTestHelper.createMetricHostAggregate;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.MetricTestHelper.createEmptyTimelineClusterMetric;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.MetricTestHelper.prepareSingleTimelineMetric;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_MINUTE_TABLE_NAME;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
 
 
 
 
 public class ITPhoenixHBaseAccessor extends AbstractMiniHBaseClusterTest {
 public class ITPhoenixHBaseAccessor extends AbstractMiniHBaseClusterTest {
@@ -119,8 +115,8 @@ public class ITPhoenixHBaseAccessor extends AbstractMiniHBaseClusterTest {
   @Test
   @Test
   public void testGetMetricRecordsMinutes() throws IOException, SQLException {
   public void testGetMetricRecordsMinutes() throws IOException, SQLException {
     // GIVEN
     // GIVEN
-    TimelineMetricAggregator aggregatorMinute = TimelineMetricAggregatorFactory
-        .createTimelineMetricAggregatorMinute(hdb, new Configuration());
+    TimelineMetricAggregator aggregatorMinute =
+      TimelineMetricAggregatorFactory.createTimelineMetricAggregatorMinute(hdb, new Configuration());
 
 
     long startTime = System.currentTimeMillis();
     long startTime = System.currentTimeMillis();
     long ctime = startTime;
     long ctime = startTime;
@@ -156,8 +152,8 @@ public class ITPhoenixHBaseAccessor extends AbstractMiniHBaseClusterTest {
   @Test
   @Test
   public void testGetMetricRecordsHours() throws IOException, SQLException {
   public void testGetMetricRecordsHours() throws IOException, SQLException {
     // GIVEN
     // GIVEN
-    TimelineMetricAggregator aggregator = TimelineMetricAggregatorFactory
-        .createTimelineMetricAggregatorHourly(hdb, new Configuration());
+    TimelineMetricAggregator aggregator =
+      TimelineMetricAggregatorFactory.createTimelineMetricAggregatorHourly(hdb, new Configuration());
 
 
     MetricHostAggregate expectedAggregate =
     MetricHostAggregate expectedAggregate =
         createMetricHostAggregate(2.0, 0.0, 20, 15.0);
         createMetricHostAggregate(2.0, 0.0, 20, 15.0);
@@ -207,8 +203,8 @@ public class ITPhoenixHBaseAccessor extends AbstractMiniHBaseClusterTest {
   @Test
   @Test
   public void testGetClusterMetricRecordsSeconds() throws Exception {
   public void testGetClusterMetricRecordsSeconds() throws Exception {
     // GIVEN
     // GIVEN
-    TimelineMetricClusterAggregator agg =
-        new TimelineMetricClusterAggregator(hdb, new Configuration());
+    TimelineMetricAggregator agg =
+      TimelineMetricAggregatorFactory.createTimelineClusterAggregatorMinute(hdb, new Configuration());
 
 
     long startTime = System.currentTimeMillis();
     long startTime = System.currentTimeMillis();
     long ctime = startTime + 1;
     long ctime = startTime + 1;
@@ -246,8 +242,8 @@ public class ITPhoenixHBaseAccessor extends AbstractMiniHBaseClusterTest {
   @Test
   @Test
   public void testGetClusterMetricRecordsHours() throws Exception {
   public void testGetClusterMetricRecordsHours() throws Exception {
     // GIVEN
     // GIVEN
-    TimelineMetricClusterAggregatorHourly agg =
-        new TimelineMetricClusterAggregatorHourly(hdb, new Configuration());
+    TimelineMetricAggregator agg =
+      TimelineMetricAggregatorFactory.createTimelineClusterAggregatorHourly(hdb, new Configuration());
 
 
     long startTime = System.currentTimeMillis();
     long startTime = System.currentTimeMillis();
     long ctime = startTime;
     long ctime = startTime;
@@ -286,9 +282,7 @@ public class ITPhoenixHBaseAccessor extends AbstractMiniHBaseClusterTest {
     assertEquals(2.0, metric.getMetricValues().values().iterator().next(), 0.00001);
     assertEquals(2.0, metric.getMetricValues().values().iterator().next(), 0.00001);
   }
   }
 
 
-  private Map<String, List<Function>> singletonValueFunctionMap(String
-                                                                  metricName) {
-    return Collections.singletonMap(metricName, Collections.singletonList
-      (new Function()));
+  private Map<String, List<Function>> singletonValueFunctionMap(String metricName) {
+    return Collections.singletonMap(metricName, Collections.singletonList(new Function()));
   }
   }
 }
 }

+ 2 - 5
ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/MetricTestHelper.java

@@ -28,11 +28,8 @@ import java.util.Map;
 
 
 public class MetricTestHelper {
 public class MetricTestHelper {
 
 
-  public static MetricHostAggregate
-  createMetricHostAggregate(double max, double min, int numberOfSamples,
-                            double sum) {
-    MetricHostAggregate expectedAggregate =
-        new MetricHostAggregate();
+  public static MetricHostAggregate createMetricHostAggregate(double max, double min, int numberOfSamples, double sum) {
+    MetricHostAggregate expectedAggregate = new MetricHostAggregate();
     expectedAggregate.setMax(max);
     expectedAggregate.setMax(max);
     expectedAggregate.setMin(min);
     expectedAggregate.setMin(min);
     expectedAggregate.setNumberOfSamples(numberOfSamples);
     expectedAggregate.setNumberOfSamples(numberOfSamples);

+ 2 - 2
ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TestMetricHostAggregate.java

@@ -53,8 +53,8 @@ public class TestMetricHostAggregate {
     assertThat(aggregate.getAvg()).isEqualTo((3.0 + 8.0 + 1.0) / 5);
     assertThat(aggregate.getAvg()).isEqualTo((3.0 + 8.0 + 1.0) / 5);
   }
   }
 
 
-  private MetricHostAggregate createAggregate
-    (double sum, double min, double max, int samplesCount) {
+  static MetricHostAggregate createAggregate (Double sum, Double min,
+                                              Double max, Integer samplesCount) {
     MetricHostAggregate aggregate = new MetricHostAggregate();
     MetricHostAggregate aggregate = new MetricHostAggregate();
     aggregate.setSum(sum);
     aggregate.setSum(sum);
     aggregate.setMax(max);
     aggregate.setMax(max);

+ 66 - 4
ambari-server/src/main/resources/common-services/AMBARI_METRICS/0.1.0/configuration/ams-site.xml

@@ -52,6 +52,14 @@
       location so that checkpoint ar not lost.
       location so that checkpoint ar not lost.
     </description>
     </description>
   </property>
   </property>
+  <property>
+    <name>timeline.metrics.host.aggregator.minute.interval</name>
+    <value>120</value>
+    <description>
+      Time in seconds to sleep for the minute resolution host based
+      aggregator. Default resolution is 2 minutes.
+    </description>
+  </property>
   <property>
   <property>
     <name>timeline.metrics.host.aggregator.hourly.interval</name>
     <name>timeline.metrics.host.aggregator.hourly.interval</name>
     <value>3600</value>
     <value>3600</value>
@@ -61,11 +69,11 @@
     </description>
     </description>
   </property>
   </property>
   <property>
   <property>
-    <name>timeline.metrics.host.aggregator.minute.interval</name>
-    <value>120</value>
+    <name>timeline.metrics.daily.aggregator.minute.interval</name>
+    <value>86400</value>
     <description>
     <description>
-      Time in seconds to sleep for the minute resolution host based
-      aggregator. Default resolution is 2 minutes.
+      Time in seconds to sleep for the day resolution host based
+      aggregator. Default resolution is 24 hours.
     </description>
     </description>
   </property>
   </property>
   <property>
   <property>
@@ -76,6 +84,14 @@
       aggregator. Default is 1 hour.
       aggregator. Default is 1 hour.
     </description>
     </description>
   </property>
   </property>
+  <property>
+    <name>timeline.metrics.cluster.aggregator.daily.interval</name>
+    <value>86400</value>
+    <description>
+      Time in seconds to sleep for the day resolution cluster wide
+      aggregator. Default is 24 hours.
+    </description>
+  </property>
   <property>
   <property>
     <name>timeline.metrics.cluster.aggregator.minute.interval</name>
     <name>timeline.metrics.cluster.aggregator.minute.interval</name>
     <value>120</value>
     <value>120</value>
@@ -84,6 +100,15 @@
       aggregator. Default resolution is 2 minutes.
       aggregator. Default resolution is 2 minutes.
     </description>
     </description>
   </property>
   </property>
+  <property>
+    <name>timeline.metrics.host.aggregator.daily.checkpointCutOffMultiplier</name>
+    <value>1</value>
+    <description>
+      Multiplier value * interval = Max allowed checkpoint lag. Effectively
+      if aggregator checkpoint is greater than max allowed checkpoint delay,
+      the checkpoint will be discarded by the aggregator.
+    </description>
+  </property>
   <property>
   <property>
     <name>timeline.metrics.host.aggregator.hourly.checkpointCutOffMultiplier</name>
     <name>timeline.metrics.host.aggregator.hourly.checkpointCutOffMultiplier</name>
     <value>2</value>
     <value>2</value>
@@ -120,6 +145,22 @@
       the checkpoint will be discarded by the aggregator.
       the checkpoint will be discarded by the aggregator.
     </description>
     </description>
   </property>
   </property>
+  <property>
+    <name>timeline.metrics.cluster.aggregator.daily.checkpointCutOffMultiplier</name>
+    <value>1</value>
+    <description>
+      Multiplier value * interval = Max allowed checkpoint lag. Effectively
+      if aggregator checkpoint is greater than max allowed checkpoint delay,
+      the checkpoint will be discarded by the aggregator.
+    </description>
+  </property>
+  <property>
+    <name>timeline.metrics.host.aggregator.daily.disabled</name>
+    <value>false</value>
+    <description>
+      Disable host based daily aggregations.
+    </description>
+  </property>
   <property>
   <property>
     <name>timeline.metrics.host.aggregator.hourly.disabled</name>
     <name>timeline.metrics.host.aggregator.hourly.disabled</name>
     <value>false</value>
     <value>false</value>
@@ -134,6 +175,13 @@
       Disable host based minute aggregations.
       Disable host based minute aggregations.
     </description>
     </description>
   </property>
   </property>
+  <property>
+    <name>timeline.metrics.cluster.aggregator.daily.disabled</name>
+    <value>false</value>
+    <description>
+      Disable cluster based daily aggregations.
+    </description>
+  </property>
   <property>
   <property>
     <name>timeline.metrics.cluster.aggregator.hourly.disabled</name>
     <name>timeline.metrics.cluster.aggregator.hourly.disabled</name>
     <value>false</value>
     <value>false</value>
@@ -155,6 +203,13 @@
       Lowest resolution of desired data for cluster level minute aggregates.
       Lowest resolution of desired data for cluster level minute aggregates.
     </description>
     </description>
   </property>
   </property>
+  <property>
+    <name>timeline.metrics.host.aggregator.daily.ttl</name>
+    <value>31536000</value>
+    <description>
+      Host based daily resolution data purge interval. Default is 1 year.
+    </description>
+  </property>
   <property>
   <property>
     <name>timeline.metrics.host.aggregator.hourly.ttl</name>
     <name>timeline.metrics.host.aggregator.hourly.ttl</name>
     <value>2592000</value>
     <value>2592000</value>
@@ -183,6 +238,13 @@
       Cluster wide hourly resolution data purge interval. Default is 1 year.
       Cluster wide hourly resolution data purge interval. Default is 1 year.
     </description>
     </description>
   </property>
   </property>
+  <property>
+    <name>timeline.metrics.cluster.aggregator.daily.ttl</name>
+    <value>63072000</value>
+    <description>
+      Cluster wide daily resolution data purge interval. Default is 2 years.
+    </description>
+  </property>
   <property>
   <property>
     <name>timeline.metrics.host.aggregator.ttl</name>
     <name>timeline.metrics.host.aggregator.ttl</name>
     <value>86400</value>
     <value>86400</value>