AMBARI-7965. Externalize all hard-coded config properties from Timeline 
service. (swagle)


Project: http://git-wip-us.apache.org/repos/asf/ambari/repo
Commit: http://git-wip-us.apache.org/repos/asf/ambari/commit/78dbf4f8
Tree: http://git-wip-us.apache.org/repos/asf/ambari/tree/78dbf4f8
Diff: http://git-wip-us.apache.org/repos/asf/ambari/diff/78dbf4f8

Branch: refs/heads/branch-metrics-dev
Commit: 78dbf4f855a6ff2270618ad0792daf9b6e280e4d
Parents: 5649e80
Author: Siddharth Wagle <swa...@hortonworks.com>
Authored: Fri Oct 24 14:47:12 2014 -0700
Committer: Siddharth Wagle <swa...@hortonworks.com>
Committed: Fri Oct 24 14:47:12 2014 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/yarn/conf/YarnConfig.java |  26 ---
 .../timeline/AbstractTimelineAggregator.java    |  33 ++-
 .../timeline/HBaseTimelineMetricStore.java      | 113 ++++------
 .../metrics/timeline/PhoenixHBaseAccessor.java  |  80 +++++--
 .../metrics/timeline/PhoenixTransactSQL.java    |  31 +--
 .../TimelineMetricAggregatorHourly.java         |  41 +++-
 .../TimelineMetricAggregatorMinute.java         |  42 +++-
 .../TimelineMetricClusterAggregator.java        |  48 +++-
 .../TimelineMetricClusterAggregatorHourly.java  |  41 +++-
 .../timeline/TimelineMetricConfiguration.java   |  87 +++++++
 .../yarn/util/timeline/TimelineUtilsExt.java    |  39 ----
 .../AMS/configuration/ams-hbase-env.xml         | 133 +++++++++++
 .../AMS/configuration/ams-hbase-log4j.xml       | 143 ++++++++++++
 .../AMS/configuration/ams-hbase-policy.xml      |  53 +++++
 .../AMS/configuration/ams-hbase-site.xml        | 226 +++++++++++++++++++
 .../2.2/services/AMS/configuration/ams-site.xml | 221 ++++++++++++++++++
 .../stacks/HDP/2.2/services/AMS/metainfo.xml    |  97 ++++++++
 .../AMS/package/files/hbaseSmokeVerify.sh       |  34 +++
 .../services/AMS/package/scripts/__init__.py    |  19 ++
 .../2.2/services/AMS/package/scripts/hbase.py   | 144 ++++++++++++
 .../AMS/package/scripts/hbase_master.py         |  70 ++++++
 .../AMS/package/scripts/hbase_regionserver.py   |  66 ++++++
 .../AMS/package/scripts/hbase_service.py        |  51 +++++
 .../2.2/services/AMS/package/scripts/params.py  | 142 ++++++++++++
 .../package/templates/hbase_client_jaas.conf.j2 |  23 ++
 .../templates/hbase_grant_permissions.j2        |  39 ++++
 .../package/templates/hbase_master_jaas.conf.j2 |  26 +++
 .../templates/hbase_regionserver_jaas.conf.j2   |  26 +++
 .../AMS/package/templates/regionservers.j2      |  20 ++
 29 files changed, 1897 insertions(+), 217 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/78dbf4f8/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/conf/YarnConfig.java
----------------------------------------------------------------------
diff --git 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/conf/YarnConfig.java
 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/conf/YarnConfig.java
deleted file mode 100644
index fe5a553..0000000
--- 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/conf/YarnConfig.java
+++ /dev/null
@@ -1,26 +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.conf;
-
-public class YarnConfig extends YarnConfiguration {
-  public static final String TIMELINE_METRICS_SERVICE_PREFIX =
-        TIMELINE_SERVICE_PREFIX + "metrics.";
-
-  public static final String TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR =
-        TIMELINE_METRICS_SERVICE_PREFIX + "aggregator.checkpoint.dir";
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/78dbf4f8/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/AbstractTimelineAggregator.java
----------------------------------------------------------------------
diff --git 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/AbstractTimelineAggregator.java
 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/AbstractTimelineAggregator.java
index 2630846..e2a6e62 100644
--- 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/AbstractTimelineAggregator.java
+++ 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/AbstractTimelineAggregator.java
@@ -22,6 +22,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
 import org.codehaus.jackson.annotate.JsonCreator;
 import org.codehaus.jackson.annotate.JsonProperty;
 import org.codehaus.jackson.annotate.JsonSubTypes;
@@ -31,25 +32,27 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Date;
 
+import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.AGGREGATOR_CHECKPOINT_DELAY;
+import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.RESULTSET_FETCH_SIZE;
+
 public abstract class AbstractTimelineAggregator implements Runnable {
   protected final PhoenixHBaseAccessor hBaseAccessor;
-  protected final String CHECKPOINT_LOCATION;
   private final Log LOG;
-  static final long checkpointDelay = 120000;
-  static final Integer RESULTSET_FETCH_SIZE = 2000;
   private static final ObjectMapper mapper;
+  protected final long checkpointDelay;
+  protected final Integer resultsetFetchSize;
+  protected Configuration metricsConf;
 
   static {
-    //SimpleModule simpleModule = new SimpleModule("MetricAggregator", new 
Version(1, 0, 0, null));
-    //simpleModule.addSerializer(new MetricAggregateSerializer());
     mapper = new ObjectMapper();
-    //mapper.registerModule(simpleModule);
   }
 
   public AbstractTimelineAggregator(PhoenixHBaseAccessor hBaseAccessor,
-                                    String checkpointLocation) {
+                                    Configuration metricsConf) {
     this.hBaseAccessor = hBaseAccessor;
-    this.CHECKPOINT_LOCATION = checkpointLocation;
+    this.metricsConf = metricsConf;
+    this.checkpointDelay = metricsConf.getInt(AGGREGATOR_CHECKPOINT_DELAY, 
120000);
+    this.resultsetFetchSize = metricsConf.getInt(RESULTSET_FETCH_SIZE, 2000);
     this.LOG = LogFactory.getLog(this.getClass());
   }
 
@@ -127,7 +130,7 @@ public abstract class AbstractTimelineAggregator implements 
Runnable {
 
   private long readCheckPoint() {
     try {
-      File checkpoint = new File(CHECKPOINT_LOCATION);
+      File checkpoint = new File(getCheckpointLocation());
       if (checkpoint.exists()) {
         String contents = FileUtils.readFileToString(checkpoint);
         if (contents != null && !contents.isEmpty()) {
@@ -141,12 +144,12 @@ public abstract class AbstractTimelineAggregator 
implements Runnable {
   }
 
   private void saveCheckPoint(long checkpointTime) throws IOException {
-    File checkpoint = new File(CHECKPOINT_LOCATION);
+    File checkpoint = new File(getCheckpointLocation());
     if (!checkpoint.exists()) {
       boolean done = checkpoint.createNewFile();
       if (!done) {
         throw new IOException("Could not create checkpoint at location, " +
-          CHECKPOINT_LOCATION);
+          getCheckpointLocation());
       }
     }
     FileUtils.writeStringToFile(checkpoint, String.valueOf(checkpointTime));
@@ -157,7 +160,13 @@ public abstract class AbstractTimelineAggregator 
implements Runnable {
 
   protected abstract Long getSleepInterval();
 
-  protected abstract Long getCheckpointCutOffInterval();
+  protected abstract Integer getCheckpointCutOffMultiplier();
+
+  protected Long getCheckpointCutOffInterval() {
+    return getCheckpointCutOffMultiplier() * getSleepInterval();
+  }
+
+  protected abstract String getCheckpointLocation();
 
   @JsonSubTypes({ @JsonSubTypes.Type(value = MetricClusterAggregate.class),
                 @JsonSubTypes.Type(value = MetricHostAggregate.class) })

http://git-wip-us.apache.org/repos/asf/ambari/blob/78dbf4f8/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/HBaseTimelineMetricStore.java
----------------------------------------------------------------------
diff --git 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/HBaseTimelineMetricStore.java
 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/HBaseTimelineMetricStore.java
index bc1fcf5..92cdec2 100644
--- 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/HBaseTimelineMetricStore.java
+++ 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/HBaseTimelineMetricStore.java
@@ -17,7 +17,6 @@
  */
 package 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline;
 
-import org.apache.commons.io.FilenameUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -25,7 +24,6 @@ import 
org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
 import org.apache.hadoop.metrics2.sink.timeline.TimelineMetrics;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
-import org.apache.hadoop.yarn.conf.YarnConfig;
 import java.io.IOException;
 import java.net.URL;
 import java.sql.SQLException;
@@ -37,19 +35,9 @@ import java.util.Map;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.Condition;
 
 public class HBaseTimelineMetricStore extends AbstractService
-  implements TimelineMetricStore {
+    implements TimelineMetricStore {
 
   static final Log LOG = LogFactory.getLog(HBaseTimelineMetricStore.class);
-  static final String HBASE_CONF = "hbase-site.xml";
-  static final String DEFAULT_CHECKPOINT_LOCATION = 
System.getProperty("java.io.tmpdir");
-  static final String AGGREGATOR_CHECKPOINT_FILE =
-    "timeline-metrics-aggregator-checkpoint";
-  static final String MINUTE_AGGREGATE_ROLLUP_CHECKPOINT_FILE =
-    "timeline-metrics-minute-aggregator-checkpoint";
-  static final String HOURLY_AGGREGATE_ROLLUP_CHECKPOINT_FILE =
-    "timeline-metrics-hourly-aggregator-checkpoint";
-  static final String HOURLY_ROLLUP_CHECKPOINT_FILE =
-    "timeline-metrics-hourly-checkpoint";
   private PhoenixHBaseAccessor hBaseAccessor;
 
   /**
@@ -62,60 +50,57 @@ public class HBaseTimelineMetricStore extends 
AbstractService
 
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
-    URL resUrl = getClass().getClassLoader().getResource(HBASE_CONF);
-    LOG.info("Found hbase site configuration: " + resUrl);
-    Configuration hbaseConf;
-    if (resUrl != null) {
-      hbaseConf = new Configuration(true);
-      hbaseConf.addResource(resUrl.toURI().toURL());
-      hBaseAccessor = new PhoenixHBaseAccessor(hbaseConf);
-      hBaseAccessor.initMetricSchema();
-
-      String checkpointLocation = FilenameUtils.concat(conf.get(
-        YarnConfig.TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR,
-        DEFAULT_CHECKPOINT_LOCATION), AGGREGATOR_CHECKPOINT_FILE);
-
-      // Start the cluster aggregator
-      TimelineMetricClusterAggregator clusterAggregator =
-        new TimelineMetricClusterAggregator(hBaseAccessor, checkpointLocation);
-      Thread aggregatorThread = new Thread(clusterAggregator);
-      aggregatorThread.start();
-
-      // Start the hourly cluster aggregator
-      String clusterAggregatorHourlyCheckpoint = FilenameUtils.concat(conf.get(
-        YarnConfig.TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR,
-        DEFAULT_CHECKPOINT_LOCATION), HOURLY_AGGREGATE_ROLLUP_CHECKPOINT_FILE);
-
-      TimelineMetricClusterAggregatorHourly clusterAggregatorHourly = new
-        TimelineMetricClusterAggregatorHourly(hBaseAccessor,
-        clusterAggregatorHourlyCheckpoint);
-      Thread rollupAggregatorThread = new Thread(clusterAggregatorHourly);
-      rollupAggregatorThread.start();
-
-      // Start the 5 minute aggregator
-      String minuteCheckpoint = FilenameUtils.concat(conf.get(
-        YarnConfig.TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR,
-        DEFAULT_CHECKPOINT_LOCATION), MINUTE_AGGREGATE_ROLLUP_CHECKPOINT_FILE);
-      TimelineMetricAggregatorMinute minuteAggregator = new
-        TimelineMetricAggregatorMinute(hBaseAccessor, minuteCheckpoint);
-
-      Thread minuteAggregatorThread = new Thread(minuteAggregator);
-      minuteAggregatorThread.start();
-
-      // Start hourly host aggregator
-      String hostAggregatorHourlyCheckpoint = FilenameUtils.concat(conf.get(
-        YarnConfig.TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR,
-        DEFAULT_CHECKPOINT_LOCATION), HOURLY_ROLLUP_CHECKPOINT_FILE);
-
-      TimelineMetricAggregatorHourly aggregatorHourly = new
-        TimelineMetricAggregatorHourly(hBaseAccessor, 
hostAggregatorHourlyCheckpoint);
-      Thread aggregatorHourlyThread = new Thread(aggregatorHourly);
-      aggregatorHourlyThread.start();
-
-    } else {
+    URL hbaseResUrl = getClass().getClassLoader().getResource
+      (TimelineMetricConfiguration.HBASE_SITE_CONFIGURATION_FILE);
+    URL amsResUrl = getClass().getClassLoader().getResource
+      (TimelineMetricConfiguration.METRICS_SITE_CONFIGURATION_FILE);
+    LOG.info("Found hbase site configuration: " + hbaseResUrl);
+    LOG.info("Found metric service configuration: " + amsResUrl);
+
+    if (hbaseResUrl == null) {
       throw new IllegalStateException("Unable to initialize the metrics " +
         "subsystem. No hbase-site present in the classpath.");
     }
+
+    if (amsResUrl == null) {
+      throw new IllegalStateException("Unable to initialize the metrics " +
+        "subsystem. No ams-site present in the classpath.");
+    }
+
+    Configuration hbaseConf = new Configuration(true);
+    hbaseConf.addResource(hbaseResUrl.toURI().toURL());
+    Configuration metricsConf = new Configuration(true);
+    metricsConf.addResource(amsResUrl.toURI().toURL());
+
+    initializeSubsystem(hbaseConf, metricsConf);
+  }
+
+  private void initializeSubsystem(Configuration hbaseConf,
+                                   Configuration metricsConf) {
+    hBaseAccessor = new PhoenixHBaseAccessor(hbaseConf, metricsConf);
+    hBaseAccessor.initMetricSchema();
+
+    // Start the cluster aggregator
+    TimelineMetricClusterAggregator clusterAggregator =
+      new TimelineMetricClusterAggregator(hBaseAccessor, metricsConf);
+    Thread aggregatorThread = new Thread(clusterAggregator);
+    aggregatorThread.start();
+
+    // Start the cluster aggregator hourly
+    TimelineMetricClusterAggregatorHourly clusterAggregatorHourly =
+      new TimelineMetricClusterAggregatorHourly(hBaseAccessor, metricsConf);
+
+    // Start the 5 minute aggregator
+    TimelineMetricAggregatorMinute minuteAggregator =
+      new TimelineMetricAggregatorMinute(hBaseAccessor, metricsConf);
+    Thread minuteAggregatorThread = new Thread(minuteAggregator);
+    minuteAggregatorThread.start();
+
+    // Start hourly host aggregator
+    TimelineMetricAggregatorHourly aggregatorHourly =
+      new TimelineMetricAggregatorHourly(hBaseAccessor, metricsConf);
+    Thread aggregatorHourlyThread = new Thread(aggregatorHourly);
+    aggregatorHourlyThread.start();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/ambari/blob/78dbf4f8/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/PhoenixHBaseAccessor.java
----------------------------------------------------------------------
diff --git 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/PhoenixHBaseAccessor.java
 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/PhoenixHBaseAccessor.java
index 19f41b5..24522ca 100644
--- 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/PhoenixHBaseAccessor.java
+++ 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/PhoenixHBaseAccessor.java
@@ -23,7 +23,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
 import org.apache.hadoop.metrics2.sink.timeline.TimelineMetrics;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
-import org.apache.hadoop.yarn.util.timeline.TimelineUtilsExt;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.type.TypeReference;
 
 import java.io.IOException;
 import java.sql.Connection;
@@ -43,38 +44,58 @@ import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.ti
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.CREATE_METRICS_CLUSTER_AGGREGATE_TABLE_SQL;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.CREATE_METRICS_TABLE_SQL;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.Condition;
+import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.DEFAULT_ENCODING;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.DEFAULT_TABLE_COMPRESSION;
-import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.METRICS_RECORD_CACHE_TABLE_NAME;
-import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.METRICS_RECORD_CACHE_TABLE_TTL;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.METRICS_RECORD_TABLE_NAME;
-import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.METRICS_RECORD_TABLE_TTL;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.UPSERT_AGGREGATE_RECORD_SQL;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.UPSERT_CLUSTER_AGGREGATE_SQL;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.UPSERT_METRICS_SQL;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricClusterAggregator.TimelineClusterMetric;
+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.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.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.PRECISION_TABLE_TTL;
+import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.RESULTSET_FETCH_SIZE;
 
 /**
  * Provides a facade over the Phoenix API to access HBase schema
  */
 public class PhoenixHBaseAccessor {
 
-  private final Configuration conf;
+  private final Configuration hbaseConf;
+  private final Configuration metricsConf;
   static final Log LOG = LogFactory.getLog(PhoenixHBaseAccessor.class);
   private static final String connectionUrl = "jdbc:phoenix:%s:%s:%s";
-
   private static final String ZOOKEEPER_CLIENT_PORT =
     "hbase.zookeeper.property.clientPort";
   private static final String ZOOKEEPER_QUORUM = "hbase.zookeeper.quorum";
   private static final String ZNODE_PARENT = "zookeeper.znode.parent";
   static final int PHOENIX_MAX_MUTATION_STATE_SIZE = 50000;
+  /**
+   * 4 metrics/min * 60 * 24: Retrieve data for 1 day.
+   */
+  public static int RESULTSET_LIMIT = 5760;
+  private static ObjectMapper mapper;
+
+  static {
+    mapper = new ObjectMapper();
+  }
 
-  public PhoenixHBaseAccessor(Configuration conf) {
-    this.conf = conf;
+  private static TypeReference<Map<Long, Double>> metricValuesTypeRef =
+    new TypeReference<Map<Long, Double>>() {};
+
+  public PhoenixHBaseAccessor(Configuration hbaseConf, Configuration 
metricsConf) {
+    this.hbaseConf = hbaseConf;
+    this.metricsConf = metricsConf;
+    RESULTSET_LIMIT = metricsConf.getInt(RESULTSET_FETCH_SIZE, 5760);
     try {
       Class.forName("org.apache.phoenix.jdbc.PhoenixDriver");
     } catch (ClassNotFoundException e) {
-      LOG.error("Phoenix client jar not found in the classpath.");
-      e.printStackTrace();
+      LOG.error("Phoenix client jar not found in the classpath.", e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -88,9 +109,9 @@ public class PhoenixHBaseAccessor {
    */
   protected Connection getConnection() {
     Connection connection = null;
-    String zookeeperClientPort = conf.getTrimmed(ZOOKEEPER_CLIENT_PORT, 
"2181");
-    String zookeeperQuorum = conf.getTrimmed(ZOOKEEPER_QUORUM);
-    String znodeParent = conf.getTrimmed(ZNODE_PARENT, "/hbase");
+    String zookeeperClientPort = hbaseConf.getTrimmed(ZOOKEEPER_CLIENT_PORT, 
"2181");
+    String zookeeperQuorum = hbaseConf.getTrimmed(ZOOKEEPER_QUORUM);
+    String znodeParent = hbaseConf.getTrimmed(ZNODE_PARENT, "/hbase");
 
     if (zookeeperQuorum == null || zookeeperQuorum.isEmpty()) {
       throw new IllegalStateException("Unable to find Zookeeper quorum to " +
@@ -111,6 +132,10 @@ public class PhoenixHBaseAccessor {
     return connection;
   }
 
+  public static Map readMetricFromJSON(String json) throws IOException {
+    return mapper.readValue(json, metricValuesTypeRef);
+  }
+
   @SuppressWarnings("unchecked")
   static TimelineMetric getTimelineMetricFromResultSet(ResultSet rs)
       throws SQLException, IOException {
@@ -123,8 +148,7 @@ public class PhoenixHBaseAccessor {
     metric.setStartTime(rs.getLong("START_TIME"));
     metric.setType(rs.getString("UNITS"));
     metric.setMetricValues(
-      (Map<Long, Double>) TimelineUtilsExt.readMetricFromJSON(
-        rs.getString("METRICS")));
+      (Map<Long, Double>) readMetricFromJSON(rs.getString("METRICS")));
     return metric;
   }
 
@@ -155,19 +179,27 @@ public class PhoenixHBaseAccessor {
     Connection conn = getConnection();
     Statement stmt = null;
 
+    String encoding = metricsConf.get(HBASE_ENCODING_SCHEME, DEFAULT_ENCODING);
+    String compression = metricsConf.get(HBASE_COMPRESSION_SCHEME, 
DEFAULT_TABLE_COMPRESSION);
+    String precisionTtl = metricsConf.get(PRECISION_TABLE_TTL, "86400");
+    String hostMinTtl = metricsConf.get(HOST_MINUTE_TABLE_TTL, "604800");
+    String hostHourTtl = metricsConf.get(HOST_HOUR_TABLE_TTL, "2592000");
+    String clusterMinTtl = metricsConf.get(CLUSTER_MINUTE_TABLE_TTL, 
"2592000");
+    String clusterHourTtl = metricsConf.get(CLUSTER_HOUR_TABLE_TTL, 
"31536000");
+
     try {
       LOG.info("Initializing metrics schema...");
       stmt = conn.createStatement();
       stmt.executeUpdate(String.format(CREATE_METRICS_TABLE_SQL,
-        METRICS_RECORD_TABLE_NAME, METRICS_RECORD_TABLE_TTL,
-        DEFAULT_TABLE_COMPRESSION));
-      /*stmt.executeUpdate(String.format(CREATE_METRICS_TABLE_SQL,
-        METRICS_RECORD_CACHE_TABLE_NAME, METRICS_RECORD_CACHE_TABLE_TTL,
-        "NONE"));*/
-      stmt.executeUpdate(CREATE_METRICS_AGGREGATE_HOURLY_TABLE_SQL);
-      stmt.executeUpdate(CREATE_METRICS_AGGREGATE_MINUTE_TABLE_SQL);
-      stmt.executeUpdate(CREATE_METRICS_CLUSTER_AGGREGATE_TABLE_SQL);
-      stmt.executeUpdate(CREATE_METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_SQL);
+        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_CLUSTER_AGGREGATE_TABLE_SQL,
+        encoding, clusterMinTtl, compression));
+      
stmt.executeUpdate(String.format(CREATE_METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_SQL,
+          encoding, clusterHourTtl, compression));
       conn.commit();
     } catch (SQLException sql) {
       LOG.warn("Error creating Metrics Schema in HBase using Phoenix.", sql);

http://git-wip-us.apache.org/repos/asf/ambari/blob/78dbf4f8/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/PhoenixTransactSQL.java
----------------------------------------------------------------------
diff --git 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/PhoenixTransactSQL.java
 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/PhoenixTransactSQL.java
index f74c4a9..db70913 100644
--- 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/PhoenixTransactSQL.java
+++ 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/PhoenixTransactSQL.java
@@ -38,14 +38,14 @@ public class PhoenixTransactSQL {
   * Create table to store individual metric records.
   */
   public static final String CREATE_METRICS_TABLE_SQL = "CREATE TABLE IF NOT " 
+
-    "EXISTS %s (METRIC_NAME VARCHAR, " +
+    "EXISTS METRIC_RECORD (METRIC_NAME VARCHAR, " +
     "HOSTNAME VARCHAR, SERVER_TIME UNSIGNED_LONG NOT NULL, " +
     "APP_ID VARCHAR, INSTANCE_ID VARCHAR, " +
     "START_TIME UNSIGNED_LONG, UNITS CHAR(20), " +
     "METRIC_AVG DOUBLE, METRIC_MAX DOUBLE, METRIC_MIN DOUBLE, " +
     "METRICS VARCHAR CONSTRAINT pk " +
     "PRIMARY KEY (METRIC_NAME, HOSTNAME, SERVER_TIME, APP_ID, " +
-    "INSTANCE_ID)) DATA_BLOCK_ENCODING='FAST_DIFF', IMMUTABLE_ROWS=true, " +
+    "INSTANCE_ID)) DATA_BLOCK_ENCODING='%s', IMMUTABLE_ROWS=true, " +
     "TTL=%s, COMPRESSION='%s'";
 
   public static final String CREATE_METRICS_AGGREGATE_HOURLY_TABLE_SQL =
@@ -56,8 +56,8 @@ public class PhoenixTransactSQL {
     "UNITS CHAR(20), METRIC_AVG DOUBLE, METRIC_MAX DOUBLE," +
     "METRIC_MIN DOUBLE CONSTRAINT pk " +
     "PRIMARY KEY (METRIC_NAME, HOSTNAME, APP_ID, INSTANCE_ID, " +
-    "SERVER_TIME)) DATA_BLOCK_ENCODING='FAST_DIFF', IMMUTABLE_ROWS=true, 
TTL=2592000, " +
-    "COMPRESSION='SNAPPY'";
+    "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 " +
@@ -67,8 +67,8 @@ public class PhoenixTransactSQL {
     "UNITS CHAR(20), METRIC_AVG DOUBLE, METRIC_MAX DOUBLE," +
     "METRIC_MIN DOUBLE CONSTRAINT pk " +
     "PRIMARY KEY (METRIC_NAME, HOSTNAME, APP_ID, INSTANCE_ID, " +
-    "SERVER_TIME)) DATA_BLOCK_ENCODING='FAST_DIFF', IMMUTABLE_ROWS=true, 
TTL=604800, " +
-    "COMPRESSION='SNAPPY'";
+    "SERVER_TIME)) DATA_BLOCK_ENCODING='%s', IMMUTABLE_ROWS=true, TTL=%s," +
+    " COMPRESSION='%s'";
 
   public static final String CREATE_METRICS_CLUSTER_AGGREGATE_TABLE_SQL =
     "CREATE TABLE IF NOT EXISTS METRIC_AGGREGATE " +
@@ -77,8 +77,8 @@ public class PhoenixTransactSQL {
     "UNITS CHAR(20), METRIC_SUM DOUBLE, " +
     "HOSTS_COUNT UNSIGNED_INT, METRIC_MAX DOUBLE, METRIC_MIN DOUBLE " +
     "CONSTRAINT pk PRIMARY KEY (METRIC_NAME, APP_ID, INSTANCE_ID, " +
-    "SERVER_TIME)) DATA_BLOCK_ENCODING='FAST_DIFF', IMMUTABLE_ROWS=true, 
TTL=2592000, " +
-    "COMPRESSION='SNAPPY'";
+    "SERVER_TIME)) DATA_BLOCK_ENCODING='%s', IMMUTABLE_ROWS=true, " +
+    "TTL=%s, COMPRESSION='%s'";
 
   public static final String CREATE_METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_SQL 
=
     "CREATE TABLE IF NOT EXISTS METRIC_AGGREGATE_HOURLY " +
@@ -87,8 +87,8 @@ public class PhoenixTransactSQL {
     "UNITS CHAR(20), METRIC_AVG DOUBLE, " +
     "METRIC_MAX DOUBLE, METRIC_MIN DOUBLE " +
     "CONSTRAINT pk PRIMARY KEY (METRIC_NAME, APP_ID, INSTANCE_ID, " +
-    "SERVER_TIME)) DATA_BLOCK_ENCODING='FAST_DIFF', IMMUTABLE_ROWS=true, 
TTL=31536000, " +
-    "COMPRESSION='SNAPPY'";
+    "SERVER_TIME)) DATA_BLOCK_ENCODING='%s', IMMUTABLE_ROWS=true, " +
+    "TTL=%s, COMPRESSION='%s'";
 
   /**
    * Insert into metric records table.
@@ -124,21 +124,14 @@ public class PhoenixTransactSQL {
     "INSTANCE_ID, SERVER_TIME, METRIC_SUM, HOSTS_COUNT, METRIC_MAX, " +
     "METRIC_MIN FROM METRIC_AGGREGATE";
 
-  /**
-   * 4 metrics/min * 60 * 24: Retrieve data for 1 day.
-   */
-  public static final Integer DEFAULT_RESULT_LIMIT = 5760;
   public static final String METRICS_RECORD_TABLE_NAME =
     "METRIC_RECORD";
-  public static final int METRICS_RECORD_TABLE_TTL = 86400;
-  public static final String METRICS_RECORD_CACHE_TABLE_NAME =
-    "METRIC_RECORD_TEMP";
-  public static final int METRICS_RECORD_CACHE_TABLE_TTL = 900;
   public static final String METRICS_AGGREGATE_MINUTE_TABLE_NAME =
     "METRIC_RECORD_MINUTE";
   public static final String METRICS_AGGREGATE_HOURLY_TABLE_NAME =
     "METRIC_RECORD_HOURLY";
   public static final String DEFAULT_TABLE_COMPRESSION = "SNAPPY";
+  public static final String DEFAULT_ENCODING = "FAST_DIFF";
 
   public static PreparedStatement prepareGetMetricsSqlStmt(
       Connection connection, Condition condition) throws SQLException {
@@ -380,7 +373,7 @@ public class PhoenixTransactSQL {
       if (noLimit) {
         return null;
       }
-      return limit == null ? DEFAULT_RESULT_LIMIT : limit;
+      return limit == null ? PhoenixHBaseAccessor.RESULTSET_LIMIT : limit;
     }
 
     boolean isGrouped() {

http://git-wip-us.apache.org/repos/asf/ambari/blob/78dbf4f8/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricAggregatorHourly.java
----------------------------------------------------------------------
diff --git 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricAggregatorHourly.java
 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricAggregatorHourly.java
index 6418322..95fc3d8 100644
--- 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricAggregatorHourly.java
+++ 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricAggregatorHourly.java
@@ -18,8 +18,10 @@
 
 package 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline;
 
+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.metrics2.sink.timeline.TimelineMetric;
 
 import java.io.IOException;
@@ -35,16 +37,37 @@ import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.ti
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.GET_METRIC_AGGREGATE_ONLY_SQL;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.METRICS_AGGREGATE_HOURLY_TABLE_NAME;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.METRICS_AGGREGATE_MINUTE_TABLE_NAME;
+import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.*;
+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 TimelineMetricAggregatorHourly extends AbstractTimelineAggregator 
{
-  static final Long SLEEP_INTERVAL = 3600000l;
-  static final Long CHECKPOINT_CUT_OFF_INTERVAL = SLEEP_INTERVAL * 2;
-  static final Integer RESULTSET_FETCH_SIZE = 1000;
   private static final Log LOG = 
LogFactory.getLog(TimelineMetricAggregatorHourly.class);
+  private static final String MINUTE_AGGREGATE_HOURLY_CHECKPOINT_FILE =
+    "timeline-metrics-host-aggregator-hourly-checkpoint";
+  private final String checkpointLocation;
+  private final Long sleepInterval;
+  private final Integer checkpointCutOffMultiplier;
 
   public TimelineMetricAggregatorHourly(PhoenixHBaseAccessor hBaseAccessor,
-                                        String checkpointLocation) {
-    super(hBaseAccessor, checkpointLocation);
+                                        Configuration metricsConf) {
+
+    super(hBaseAccessor, metricsConf);
+
+    String checkpointDir = metricsConf.get(
+      TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR, DEFAULT_CHECKPOINT_LOCATION);
+
+    checkpointLocation = FilenameUtils.concat(checkpointDir,
+      MINUTE_AGGREGATE_HOURLY_CHECKPOINT_FILE);
+
+    sleepInterval = metricsConf.getLong(HOST_AGGREGATOR_HOUR_SLEEP_INTERVAL, 
3600000l);
+    checkpointCutOffMultiplier =
+      metricsConf.getInt(HOST_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_MULTIPLIER, 2);
+  }
+
+  @Override
+  protected String getCheckpointLocation() {
+    return checkpointLocation;
   }
 
   @Override
@@ -55,7 +78,7 @@ public class TimelineMetricAggregatorHourly extends 
AbstractTimelineAggregator {
     Condition condition = new Condition(null, null, null, null, startTime,
                                         endTime, null, true);
     condition.setNoLimit();
-    condition.setFetchSize(RESULTSET_FETCH_SIZE);
+    condition.setFetchSize(resultsetFetchSize);
     condition.setStatement(String.format(GET_METRIC_AGGREGATE_ONLY_SQL,
       METRICS_AGGREGATE_MINUTE_TABLE_NAME));
     condition.addOrderByColumn("METRIC_NAME");
@@ -137,11 +160,11 @@ public class TimelineMetricAggregatorHourly extends 
AbstractTimelineAggregator {
 
   @Override
   protected Long getSleepInterval() {
-    return SLEEP_INTERVAL;
+    return sleepInterval;
   }
 
   @Override
-  protected Long getCheckpointCutOffInterval() {
-    return CHECKPOINT_CUT_OFF_INTERVAL;
+  protected Integer getCheckpointCutOffMultiplier() {
+    return checkpointCutOffMultiplier;
   }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/78dbf4f8/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricAggregatorMinute.java
----------------------------------------------------------------------
diff --git 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricAggregatorMinute.java
 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricAggregatorMinute.java
index 8661821..ab7b1ee 100644
--- 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricAggregatorMinute.java
+++ 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricAggregatorMinute.java
@@ -17,9 +17,12 @@
  */
 package 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline;
 
+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.metrics2.sink.timeline.TimelineMetric;
+
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
@@ -32,18 +35,39 @@ import java.util.Map;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.Condition;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.GET_METRIC_AGGREGATE_ONLY_SQL;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.METRICS_AGGREGATE_MINUTE_TABLE_NAME;
-import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.METRICS_RECORD_CACHE_TABLE_NAME;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.METRICS_RECORD_TABLE_NAME;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.prepareGetMetricsSqlStmt;
+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_MINUTE_CHECKPOINT_CUTOFF_MULTIPLIER;
+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;
 
 public class TimelineMetricAggregatorMinute extends AbstractTimelineAggregator 
{
-  static final Long SLEEP_INTERVAL = 300000l; // 5 mins
-  static final Long CHECKPOINT_CUT_OFF_INTERVAL = SLEEP_INTERVAL * 3;
   private static final Log LOG = 
LogFactory.getLog(TimelineMetricAggregatorMinute.class);
+  private static final String MINUTE_AGGREGATE_CHECKPOINT_FILE =
+    "timeline-metrics-host-aggregator-checkpoint";
+  private final String checkpointLocation;
+  private final Long sleepInterval;
+  private final Integer checkpointCutOffMultiplier;
 
   public TimelineMetricAggregatorMinute(PhoenixHBaseAccessor hBaseAccessor,
-                                        String checkpointLocation) {
-    super(hBaseAccessor, checkpointLocation);
+                                        Configuration metricsConf) {
+    super(hBaseAccessor, metricsConf);
+
+    String checkpointDir = metricsConf.get(
+      TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR, DEFAULT_CHECKPOINT_LOCATION);
+
+    checkpointLocation = FilenameUtils.concat(checkpointDir,
+      MINUTE_AGGREGATE_CHECKPOINT_FILE);
+
+    sleepInterval = metricsConf.getLong(HOST_AGGREGATOR_MINUTE_SLEEP_INTERVAL, 
300000l);  // 5 mins
+    checkpointCutOffMultiplier =
+      metricsConf.getInt(HOST_AGGREGATOR_MINUTE_CHECKPOINT_CUTOFF_MULTIPLIER, 
3);
+  }
+
+  @Override
+  protected String getCheckpointLocation() {
+    return checkpointLocation;
   }
 
   @Override
@@ -55,7 +79,7 @@ public class TimelineMetricAggregatorMinute extends 
AbstractTimelineAggregator {
     Condition condition = new Condition(null, null, null, null, startTime,
                                         endTime, null, true);
     condition.setNoLimit();
-    condition.setFetchSize(RESULTSET_FETCH_SIZE);
+    condition.setFetchSize(resultsetFetchSize);
     condition.setStatement(String.format(GET_METRIC_AGGREGATE_ONLY_SQL,
       METRICS_RECORD_TABLE_NAME));
     condition.addOrderByColumn("METRIC_NAME");
@@ -139,11 +163,11 @@ public class TimelineMetricAggregatorMinute extends 
AbstractTimelineAggregator {
 
   @Override
   protected Long getSleepInterval() {
-    return SLEEP_INTERVAL;
+    return sleepInterval;
   }
 
   @Override
-  protected Long getCheckpointCutOffInterval() {
-    return CHECKPOINT_CUT_OFF_INTERVAL;
+  protected Integer getCheckpointCutOffMultiplier() {
+    return checkpointCutOffMultiplier;
   }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/78dbf4f8/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricClusterAggregator.java
----------------------------------------------------------------------
diff --git 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricClusterAggregator.java
 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricClusterAggregator.java
index fe2f791..dae65db 100644
--- 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricClusterAggregator.java
+++ 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricClusterAggregator.java
@@ -17,8 +17,10 @@
  */
 package 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline;
 
+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.metrics2.sink.timeline.TimelineMetric;
 
 import java.io.IOException;
@@ -31,22 +33,48 @@ import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.Condition;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.GET_METRIC_SQL;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.METRICS_RECORD_TABLE_NAME;
 import static 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.prepareGetMetricsSqlStmt;
+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_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;
 
 /**
  * Aggregates a metric across all hosts in the cluster.
  */
 public class TimelineMetricClusterAggregator extends 
AbstractTimelineAggregator {
-  public static final long WAKE_UP_INTERVAL = 120000;
-  public static final int TIME_SLICE_INTERVAL = 15000;
   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 sleepInterval;
+  public final int timeSliceInterval;
+  private final Integer checkpointCutOffMultiplier;
 
   public TimelineMetricClusterAggregator(PhoenixHBaseAccessor hBaseAccessor,
-                                         String checkpointLocation) {
-    super(hBaseAccessor, checkpointLocation);
+                                         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);
+
+    sleepInterval = 
metricsConf.getLong(CLUSTER_AGGREGATOR_MINUTE_SLEEP_INTERVAL, 120000l);
+    timeSliceInterval = 
metricsConf.getInt(CLUSTER_AGGREGATOR_TIMESLICE_INTERVAL, 15000);
+    checkpointCutOffMultiplier =
+      
metricsConf.getInt(CLUSTER_AGGREGATOR_MINUTE_CHECKPOINT_CUTOFF_MULTIPLIER, 2);
+  }
+
+  @Override
+  protected String getCheckpointLocation() {
+    return checkpointLocation;
   }
 
   /**
@@ -63,7 +91,7 @@ public class TimelineMetricClusterAggregator extends 
AbstractTimelineAggregator
     boolean success = true;
     Condition condition = new Condition(null, null, null, null, startTime,
                                         endTime, null, true);
-    condition.setFetchSize(RESULTSET_FETCH_SIZE);
+    condition.setFetchSize(resultsetFetchSize);
     condition.setNoLimit();
     condition.setStatement(String.format(GET_METRIC_SQL,
       METRICS_RECORD_TABLE_NAME));
@@ -87,8 +115,8 @@ public class TimelineMetricClusterAggregator extends 
AbstractTimelineAggregator
       // Create time slices
       long sliceStartTime = startTime;
       while (sliceStartTime < endTime) {
-        timeSlices.add(new Long[] { sliceStartTime, sliceStartTime + 
TIME_SLICE_INTERVAL });
-        sliceStartTime += TIME_SLICE_INTERVAL;
+        timeSlices.add(new Long[] { sliceStartTime, sliceStartTime + 
timeSliceInterval});
+        sliceStartTime += timeSliceInterval;
       }
 
       while (rs.next()) {
@@ -137,12 +165,12 @@ public class TimelineMetricClusterAggregator extends 
AbstractTimelineAggregator
 
   @Override
   protected Long getSleepInterval() {
-    return WAKE_UP_INTERVAL;
+    return sleepInterval;
   }
 
   @Override
-  protected Long getCheckpointCutOffInterval() {
-    return 600000l;
+  protected Integer getCheckpointCutOffMultiplier() {
+    return checkpointCutOffMultiplier;
   }
 
   private Map<TimelineClusterMetric, Double> sliceFromTimelineMetric(

http://git-wip-us.apache.org/repos/asf/ambari/blob/78dbf4f8/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricClusterAggregatorHourly.java
----------------------------------------------------------------------
diff --git 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricClusterAggregatorHourly.java
 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricClusterAggregatorHourly.java
index 1caf809..6ff5c92 100644
--- 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricClusterAggregatorHourly.java
+++ 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricClusterAggregatorHourly.java
@@ -17,16 +17,42 @@
  */
 package 
org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline;
 
+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 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_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(TimelineMetricClusterAggregator.class);
-  public static final long SLEEP_INTERVAL = 3600000;
+  private static final Log LOG = 
LogFactory.getLog(TimelineMetricClusterAggregatorHourly.class);
+  private final long sleepInterval;
+  private static final String CLUSTER_AGGREGATOR_HOURLY_CHECKPOINT_FILE =
+    "timeline-metrics-cluster-aggregator-hourly-checkpoint";
+  private final String checkpointLocation;
+  private final Integer checkpointCutOffMultiplier;
 
   public TimelineMetricClusterAggregatorHourly(PhoenixHBaseAccessor 
hBaseAccessor,
-                                               String checkpointLocation) {
-    super(hBaseAccessor, checkpointLocation);
+                                               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);
+
+    sleepInterval = 
metricsConf.getLong(CLUSTER_AGGREGATOR_HOUR_SLEEP_INTERVAL, 3600000l);
+    checkpointCutOffMultiplier =
+      metricsConf.getInt(CLUSTER_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_MULTIPLIER, 
2);
+  }
+
+  @Override
+  protected String getCheckpointLocation() {
+    return checkpointLocation;
   }
 
   @Override
@@ -36,7 +62,12 @@ public class TimelineMetricClusterAggregatorHourly extends 
AbstractTimelineAggre
 
   @Override
   protected Long getSleepInterval() {
-    return SLEEP_INTERVAL;
+    return sleepInterval;
+  }
+
+  @Override
+  protected Integer getCheckpointCutOffMultiplier() {
+    return checkpointCutOffMultiplier;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/ambari/blob/78dbf4f8/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricConfiguration.java
----------------------------------------------------------------------
diff --git 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricConfiguration.java
 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricConfiguration.java
new file mode 100644
index 0000000..c64c18d
--- /dev/null
+++ 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TimelineMetricConfiguration.java
@@ -0,0 +1,87 @@
+/**
+ * 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;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Configuration class that reads properties from ams-site.xml
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface TimelineMetricConfiguration {
+  public static final String HBASE_SITE_CONFIGURATION_FILE = "hbase-site.xml";
+  public static final String METRICS_SITE_CONFIGURATION_FILE = "ams-site.xml";
+
+  public static final String TIMELINE_METRICS_AGGREGATOR_CHECKPOINT_DIR =
+    "timeline.metrics.aggregator.checkpoint.dir";
+
+  public static final String DEFAULT_CHECKPOINT_LOCATION =
+    System.getProperty("java.io.tmpdir");
+
+  public static final String HBASE_ENCODING_SCHEME =
+    "timeline.metrics.hbase.data.block.encoding";
+
+  public static final String HBASE_COMPRESSION_SCHEME =
+    "timeline.metrics.hbase.compression.scheme";
+
+  public static final String PRECISION_TABLE_TTL =
+    "timeline.metrics.host.aggregator.ttl";
+  public static final String HOST_MINUTE_TABLE_TTL =
+    "timeline.metrics.host.aggregator.minute.ttl";
+  public static final String HOST_HOUR_TABLE_TTL =
+    "timeline.metrics.host.aggregator.hourly.ttl";
+  public static final String CLUSTER_MINUTE_TABLE_TTL =
+    "timeline.metrics.cluster.aggregator.minute.ttl";
+  public static final String CLUSTER_HOUR_TABLE_TTL =
+    "timeline.metrics.cluster.aggregator.hourly.ttl";
+
+  public static final String CLUSTER_AGGREGATOR_TIMESLICE_INTERVAL =
+    "timeline.metrics.cluster.aggregator.minute.timeslice.interval";
+
+  public static final String AGGREGATOR_CHECKPOINT_DELAY =
+    "timeline.metrics.service.checkpointDelay";
+
+  public static final String RESULTSET_FETCH_SIZE =
+    "timeline.metrics.service.resultset.fetchSize";
+
+  public static final String HOST_AGGREGATOR_MINUTE_SLEEP_INTERVAL =
+    "timeline.metrics.host.aggregator.minute.interval";
+
+  public static final String HOST_AGGREGATOR_HOUR_SLEEP_INTERVAL =
+    "timeline.metrics.host.aggregator.hourly.interval";
+
+  public static final String CLUSTER_AGGREGATOR_MINUTE_SLEEP_INTERVAL =
+    "timeline.metrics.cluster.aggregator.minute.interval";
+
+  public static final String CLUSTER_AGGREGATOR_HOUR_SLEEP_INTERVAL =
+    "timeline.metrics.cluster.aggregator.hourly.interval";
+
+  public static final String 
HOST_AGGREGATOR_MINUTE_CHECKPOINT_CUTOFF_MULTIPLIER =
+    "timeline.metrics.host.aggregator.minute.checkpointCutOffMultiplier";
+
+  public static final String HOST_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_MULTIPLIER 
=
+    "timeline.metrics.host.aggregator.hourly.checkpointCutOffMultiplier";
+
+  public static final String 
CLUSTER_AGGREGATOR_MINUTE_CHECKPOINT_CUTOFF_MULTIPLIER =
+    "timeline.metrics.cluster.aggregator.minute.checkpointCutOffMultiplier";
+
+  public static final String 
CLUSTER_AGGREGATOR_HOUR_CHECKPOINT_CUTOFF_MULTIPLIER =
+    "timeline.metrics.cluster.aggregator.hourly.checkpointCutOffMultiplier";
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/78dbf4f8/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtilsExt.java
----------------------------------------------------------------------
diff --git 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtilsExt.java
 
b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtilsExt.java
deleted file mode 100644
index 31e8017..0000000
--- 
a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtilsExt.java
+++ /dev/null
@@ -1,39 +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.util.timeline;
-
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.type.TypeReference;
-
-import java.io.IOException;
-import java.util.Map;
-
-public class TimelineUtilsExt extends TimelineUtils {
-  private static ObjectMapper mapper;
-
-  static {
-    mapper = new ObjectMapper();
-  }
-
-  private static TypeReference<Map<Long, Double>> metricValuesTypeRef =
-    new TypeReference<Map<Long, Double>>() {};
-
-  public static Map readMetricFromJSON(String json) throws IOException {
-    return mapper.readValue(json, metricValuesTypeRef);
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/78dbf4f8/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-env.xml
----------------------------------------------------------------------
diff --git 
a/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-env.xml
 
b/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-env.xml
new file mode 100644
index 0000000..9306cd1
--- /dev/null
+++ 
b/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-env.xml
@@ -0,0 +1,133 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+/**
+ * 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.
+ */
+-->
+
+<configuration>
+  <property>
+    <name>hbase_log_dir</name>
+    <value>/var/log/hbase-ams/</value>
+    <description>Log Directories for HBase.</description>
+  </property>
+  <property>
+    <name>hbase_pid_dir</name>
+    <value>/var/run/hbase-ams/</value>
+    <description>Pid Directory for HBase.</description>
+  </property>
+  <property>
+    <name>hbase_regionserver_heapsize</name>
+    <value>1024</value>
+    <description>HBase RegionServer Heap Size.</description>
+  </property>
+  <property>
+    <name>hbase_regionserver_xmn_max</name>
+    <value>512</value>
+    <description>HBase RegionServer maximum value for minimum heap 
size.</description>
+  </property>
+  <property>
+    <name>hbase_regionserver_xmn_ratio</name>
+    <value>0.2</value>
+    <description>HBase RegionServer minimum heap size is calculated as a 
percentage of max heap size.</description>
+  </property>
+  <property>
+    <name>hbase_master_heapsize</name>
+    <value>1024</value>
+    <description>HBase Master Heap Size</description>
+  </property>
+  <property>
+    <name>hbase_user</name>
+    <value>hbase</value>
+    <property-type>USER</property-type>
+    <description>HBase User Name.</description>
+  </property>
+
+  <!-- hbase-env.sh -->
+  <property>
+    <name>content</name>
+    <description>This is the jinja template for hbase-env.sh file</description>
+    <value>
+      # Set environment variables here.
+
+      # The java implementation to use. Java 1.6 required.
+      export JAVA_HOME={{java64_home}}
+
+      # HBase Configuration directory
+      export HBASE_CONF_DIR=${HBASE_CONF_DIR:-{{hbase_conf_dir}}}
+
+      # Extra Java CLASSPATH elements. Optional.
+      export HBASE_CLASSPATH=${HBASE_CLASSPATH}
+
+      # The maximum amount of heap to use, in MB. Default is 1000.
+      # export HBASE_HEAPSIZE=1000
+
+      # Extra Java runtime options.
+      # Below are what we set by default. May only work with SUN JVM.
+      # For more on why as well as other possible settings,
+      # see http://wiki.apache.org/hadoop/PerformanceTuning
+      export HBASE_OPTS="-XX:+UseConcMarkSweepGC 
-XX:ErrorFile={{log_dir}}/hs_err_pid%p.log"
+      export SERVER_GC_OPTS="-verbose:gc -XX:+PrintGCDetails 
-XX:+PrintGCDateStamps -Xloggc:{{log_dir}}/gc.log-`date +'%Y%m%d%H%M'`"
+      # Uncomment below to enable java garbage collection logging.
+      # export HBASE_OPTS="$HBASE_OPTS -verbose:gc -XX:+PrintGCDetails 
-XX:+PrintGCDateStamps -Xloggc:$HBASE_HOME/logs/gc-hbase.log"
+
+      # Uncomment and adjust to enable JMX exporting
+      # See jmxremote.password and jmxremote.access in 
$JRE_HOME/lib/management to configure remote password access.
+      # More details at: 
http://java.sun.com/javase/6/docs/technotes/guides/management/agent.html
+      #
+      # export HBASE_JMX_BASE="-Dcom.sun.management.jmxremote.ssl=false 
-Dcom.sun.management.jmxremote.authenticate=false"
+      export HBASE_MASTER_OPTS="-Xmx{{master_heapsize}}"
+      export HBASE_REGIONSERVER_OPTS="-Xmn{{regionserver_xmn_size}} 
-XX:CMSInitiatingOccupancyFraction=70  -Xms{{regionserver_heapsize}} 
-Xmx{{regionserver_heapsize}}"
+      # export HBASE_THRIFT_OPTS="$HBASE_JMX_BASE 
-Dcom.sun.management.jmxremote.port=10103"
+      # export HBASE_ZOOKEEPER_OPTS="$HBASE_JMX_BASE 
-Dcom.sun.management.jmxremote.port=10104"
+
+      # File naming hosts on which HRegionServers will run. 
$HBASE_HOME/conf/regionservers by default.
+      export HBASE_REGIONSERVERS=${HBASE_CONF_DIR}/regionservers
+
+      # Extra ssh options. Empty by default.
+      # export HBASE_SSH_OPTS="-o ConnectTimeout=1 -o SendEnv=HBASE_CONF_DIR"
+
+      # Where log files are stored. $HBASE_HOME/logs by default.
+      export HBASE_LOG_DIR={{log_dir}}
+
+      # A string representing this instance of hbase. $USER by default.
+      # export HBASE_IDENT_STRING=$USER
+
+      # The scheduling priority for daemon processes. See 'man nice'.
+      # export HBASE_NICENESS=10
+
+      # The directory where pid files are stored. /tmp by default.
+      export HBASE_PID_DIR={{pid_dir}}
+
+      # Seconds to sleep between slave commands. Unset by default. This
+      # can be useful in large clusters, where, e.g., slave rsyncs can
+      # otherwise arrive faster than the master can service them.
+      # export HBASE_SLAVE_SLEEP=0.1
+
+      # Tell HBase whether it should manage it's own instance of Zookeeper or 
not.
+      export HBASE_MANAGES_ZK=false
+
+      {% if security_enabled %}
+      export HBASE_OPTS="$HBASE_OPTS 
-Djava.security.auth.login.config={{client_jaas_config_file}}"
+      export HBASE_MASTER_OPTS="$HBASE_MASTER_OPTS 
-Djava.security.auth.login.config={{master_jaas_config_file}}"
+      export HBASE_REGIONSERVER_OPTS="$HBASE_REGIONSERVER_OPTS 
-Djava.security.auth.login.config={{regionserver_jaas_config_file}}"
+      {% endif %}
+    </value>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/ambari/blob/78dbf4f8/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-log4j.xml
----------------------------------------------------------------------
diff --git 
a/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-log4j.xml
 
b/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-log4j.xml
new file mode 100644
index 0000000..24ba5b7
--- /dev/null
+++ 
b/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-log4j.xml
@@ -0,0 +1,143 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+/**
+ * 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.
+ */
+-->
+
+<configuration supports_final="false">
+
+  <property>
+    <name>content</name>
+    <description>Custom log4j.properties</description>
+    <value>
+      # 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.
+
+
+      # Define some default values that can be overridden by system properties
+      hbase.root.logger=INFO,console
+      hbase.security.logger=INFO,console
+      hbase.log.dir=.
+      hbase.log.file=hbase.log
+
+      # Define the root logger to the system property "hbase.root.logger".
+      log4j.rootLogger=${hbase.root.logger}
+
+      # Logging Threshold
+      log4j.threshold=ALL
+
+      #
+      # Daily Rolling File Appender
+      #
+      log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
+      log4j.appender.DRFA.File=${hbase.log.dir}/${hbase.log.file}
+
+      # Rollver at midnight
+      log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
+
+      # 30-day backup
+      #log4j.appender.DRFA.MaxBackupIndex=30
+      log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
+
+      # Pattern format: Date LogLevel LoggerName LogMessage
+      log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p [%t] 
%c{2}: %m%n
+
+      # Rolling File Appender properties
+      hbase.log.maxfilesize=256MB
+      hbase.log.maxbackupindex=20
+
+      # Rolling File Appender
+      log4j.appender.RFA=org.apache.log4j.RollingFileAppender
+      log4j.appender.RFA.File=${hbase.log.dir}/${hbase.log.file}
+
+      log4j.appender.RFA.MaxFileSize=${hbase.log.maxfilesize}
+      log4j.appender.RFA.MaxBackupIndex=${hbase.log.maxbackupindex}
+
+      log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
+      log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: 
%m%n
+
+      #
+      # Security audit appender
+      #
+      hbase.security.log.file=SecurityAuth.audit
+      hbase.security.log.maxfilesize=256MB
+      hbase.security.log.maxbackupindex=20
+      log4j.appender.RFAS=org.apache.log4j.RollingFileAppender
+      log4j.appender.RFAS.File=${hbase.log.dir}/${hbase.security.log.file}
+      log4j.appender.RFAS.MaxFileSize=${hbase.security.log.maxfilesize}
+      log4j.appender.RFAS.MaxBackupIndex=${hbase.security.log.maxbackupindex}
+      log4j.appender.RFAS.layout=org.apache.log4j.PatternLayout
+      log4j.appender.RFAS.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+      log4j.category.SecurityLogger=${hbase.security.logger}
+      log4j.additivity.SecurityLogger=false
+      
#log4j.logger.SecurityLogger.org.apache.hadoop.hbase.security.access.AccessController=TRACE
+
+      #
+      # Null Appender
+      #
+      log4j.appender.NullAppender=org.apache.log4j.varia.NullAppender
+
+      #
+      # console
+      # Add "console" to rootlogger above if you want to use this
+      #
+      log4j.appender.console=org.apache.log4j.ConsoleAppender
+      log4j.appender.console.target=System.err
+      log4j.appender.console.layout=org.apache.log4j.PatternLayout
+      log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p [%t] 
%c{2}: %m%n
+
+      # Custom Logging levels
+
+      log4j.logger.org.apache.zookeeper=INFO
+      #log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
+      log4j.logger.org.apache.hadoop.hbase=DEBUG
+      # Make these two classes INFO-level. Make them DEBUG to see more zk 
debug.
+      log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil=INFO
+      log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO
+      #log4j.logger.org.apache.hadoop.dfs=DEBUG
+      # Set this class to log INFO only otherwise its OTT
+      # Enable this to get detailed connection error/retry logging.
+      # 
log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=TRACE
+
+
+      # Uncomment this line to enable tracing on _every_ RPC call (this can be 
a lot of output)
+      #log4j.logger.org.apache.hadoop.ipc.HBaseServer.trace=DEBUG
+
+      # Uncomment the below if you want to remove logging of client region 
caching'
+      # and scan of .META. messages
+      # 
log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=INFO
+      # log4j.logger.org.apache.hadoop.hbase.client.MetaScanner=INFO
+
+    </value>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/ambari/blob/78dbf4f8/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-policy.xml
----------------------------------------------------------------------
diff --git 
a/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-policy.xml
 
b/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-policy.xml
new file mode 100644
index 0000000..febbd44
--- /dev/null
+++ 
b/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-policy.xml
@@ -0,0 +1,53 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+/**
+ * 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.
+ */
+-->
+
+<configuration supports_final="true">
+  <property>
+    <name>security.client.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for HRegionInterface protocol implementations (ie.
+      clients talking to HRegionServers)
+      The ACL is a comma-separated list of user and group names. The user and
+      group list is separated by a blank. For e.g. "alice,bob users,wheel".
+      A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.admin.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for HMasterInterface protocol implementation (ie.
+      clients talking to HMaster for admin operations).
+      The ACL is a comma-separated list of user and group names. The user and
+      group list is separated by a blank. For e.g. "alice,bob users,wheel".
+      A special value of "*" means all users are allowed.</description>
+  </property>
+
+  <property>
+    <name>security.masterregion.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for HMasterRegionInterface protocol implementations
+      (for HRegionServers communicating with HMaster)
+      The ACL is a comma-separated list of user and group names. The user and
+      group list is separated by a blank. For e.g. "alice,bob users,wheel".
+      A special value of "*" means all users are allowed.</description>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/ambari/blob/78dbf4f8/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-site.xml
----------------------------------------------------------------------
diff --git 
a/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-site.xml
 
b/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-site.xml
new file mode 100644
index 0000000..7c3b732
--- /dev/null
+++ 
b/ambari-server/src/main/resources/stacks/HDP/2.2/services/AMS/configuration/ams-hbase-site.xml
@@ -0,0 +1,226 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+/**
+ *
+ * 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.
+ */
+-->
+<configuration>
+  <property>
+    <name>hbase.rootdir</name>
+    <value>file:///var/lib/hbase</value>
+    <description>
+      AMS service uses HBase as default storage backend. Set the rootdir for
+      HBase to either local filesystem path if using AMS in embedded mode or
+      to a HDFS dir, example: hdfs://namenode.example.org:9000/hbase.  By
+      default HBase writes into /tmp. Change this configuration else all data
+      will be lost on machine restart.
+    </description>
+  </property>
+  <property>
+    <name>hbase.tmp.dir</name>
+    <value>/tmp</value>
+    <description>
+      Temporary directory on the local filesystem.
+      Change this setting to point to a location more permanent
+      than '/tmp' (The '/tmp' directory is often cleared on
+      machine restart).
+    </description>
+  </property>
+  <property>
+    <name>hbase.local.dir</name>
+    <value>${hbase.tmp.dir}/local</value>
+    <description>Directory on the local filesystem to be used as a local 
storage
+    </description>
+  </property>
+  <property>
+    <name>hbase.cluster.distributed</name>
+    <value>true</value>
+    <description>
+      The mode the cluster will be in. Possible values are false for
+      standalone mode and true for distributed mode. If false, startup will run
+      all HBase and ZooKeeper daemons together in the one JVM.
+    </description>
+  </property>
+  <property>
+    <name>hbase.master.wait.on.regionservers.mintostart</name>
+    <value>1</value>
+    <description>
+      Ensure that HBase Master waits for # many region server to start.
+    </description>
+  </property>
+  <property>
+    <name>hbase.zookeeper.quorum</name>
+    <value>localhost</value>
+    <description>Comma separated list of servers in the ZooKeeper Quorum.
+      For example, "host1.mydomain.com,host2.mydomain.com,host3.mydomain.com".
+      By default this is set to localhost for local and pseudo-distributed 
modes
+      of operation. For a fully-distributed setup, this should be set to a full
+      list of ZooKeeper quorum servers. If HBASE_MANAGES_ZK is set in 
hbase-env.sh
+      this is the list of servers which we will start/stop ZooKeeper on.
+    </description>
+  </property>
+  <property>
+    <name>hbase.master.info.bindAddress</name>
+    <value>0.0.0.0</value>
+    <description>The bind address for the HBase Master web UI</description>
+  </property>
+  <property>
+    <name>hbase.master.info.port</name>
+    <value>90010</value>
+    <description>The port for the HBase Master web UI.</description>
+  </property>
+  <property>
+    <name>hbase.regionserver.info.port</name>
+    <value>90030</value>
+    <description>The port for the HBase RegionServer web UI.</description>
+  </property>
+  <property>
+    <name>hbase.hregion.majorcompaction</name>
+    <value>0</value>
+    <description>
+      The time (in milliseconds) between 'major' compactions of all
+      HStoreFiles in a region.
+      0 to disable automated major compactions.
+    </description>
+  </property>
+  <property>
+    <name>phoenix.query.spoolThresholdBytes</name>
+    <value>12582912</value>
+    <description>
+      Threshold size in bytes after which results from parallelly executed
+      query results are spooled to disk. Default is 20 mb.
+    </description>
+  </property>
+  <property>
+    <name>hbase.zookeeper.property.dataDir</name>
+    <value>${hbase.tmp.dir}/zookeeper</value>
+    <description>
+      Property from ZooKeeper's config zoo.cfg.
+      The directory where the snapshot is stored.
+    </description>
+  </property>
+  <property>
+    <name>hbase.client.scanner.caching</name>
+    <value>10000</value>
+    <description>
+      Number of rows that will be fetched when calling next on a scanner
+      if it is not served from (local, client) memory.
+    </description>
+  </property>
+  <property>
+    <name>hfile.block.cache.size</name>
+    <value>0.3</value>
+    <description>
+      Percentage of maximum heap (-Xmx setting) to allocate to block cache
+      used by a StoreFile. Default of 0.4 means allocate 40%.
+    </description>
+  </property>
+  <property>
+    <name>hbase.regionserver.global.memstore.upperLimit</name>
+    <value>0.5</value>
+    <description>
+      Maximum size of all memstores in a region server before new
+      updates are blocked and flushes are forced. Defaults to 40% of heap
+    </description>
+  </property>
+  <property>
+    <name>hbase.regionserver.global.memstore.lowerLimit</name>
+    <value>0.4</value>
+    <description>
+      When memstores are being forced to flush to make room in
+      memory, keep flushing until we hit this mark. Defaults to 35% of heap.
+      This value equal to hbase.regionserver.global.memstore.upperLimit causes
+      the minimum possible flushing to occur when updates are blocked due to
+      memstore limiting.
+    </description>
+  </property>
+  <property>
+    <name>phoenix.groupby.maxCacheSize</name>
+    <value>307200000</value>
+    <description>
+      Size in bytes of pages cached during GROUP BY spilling. Default is 100Mb.
+    </description>
+  </property>
+  <property>
+    <name>hbase.hregion.memstore.block.multiplier</name>
+    <value>4</value>
+    <description>
+      Block updates if memstore has hbase.hregion.memstore.block.multiplier
+      times hbase.hregion.memstore.flush.size bytes. Useful preventing runaway
+      memstore during spikes in update traffic.
+    </description>
+  </property>
+  <property>
+    <name>hbase.hstore.flusher.count</name>
+    <value>2</value>
+    <description>
+      The number of flush threads. With fewer threads, the MemStore flushes
+      will be queued. With more threads, the flushes will be executed in 
parallel,
+      increasing the load on HDFS, and potentially causing more compactions.
+    </description>
+  </property>
+  <property>
+    <name>phoenix.query.timeoutMs</name>
+    <value>1200000</value>
+    <description>
+      Number of milliseconds after which a query will timeout on the client.
+      Default is 10 min.
+    </description>
+  </property>
+  <property>
+    <name>hbase.client.scanner.timeout.period</name>
+    <value>900000</value>
+    <description>
+      Client scanner lease period in milliseconds.
+    </description>
+  </property>
+  <property>
+    <name>hbase.regionserver.thread.compaction.large</name>
+    <value>2</value>
+    <description>
+      Configuration key for the large compaction threads.
+    </description>
+  </property>
+  <property>
+    <name>hbase.regionserver.thread.compaction.small</name>
+    <value>3</value>
+    <description>
+      Configuration key for the small compaction threads.
+    </description>
+  </property>
+  <property>
+    <name>hbase.hstore.blockingStoreFiles</name>
+    <value>200</value>
+    <description>
+      If more than this number of StoreFiles exist in any one Store
+      (one StoreFile is written per flush of MemStore), updates are blocked for
+      this region until a compaction is completed, or until
+      hbase.hstore.blockingWaitTime has been exceeded.
+    </description>
+  </property>
+  <property>
+    <name>hbase.hregion.memstore.flush.size</name>
+    <value>134217728</value>
+    <description>
+      Memstore will be flushed to disk if size of the memstore exceeds this
+      number of bytes. Value is checked by a thread that runs every
+      hbase.server.thread.wakefrequency.
+    </description>
+  </property>
+</configuration>
\ No newline at end of file

Reply via email to