You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by sw...@apache.org on 2014/12/01 20:32:38 UTC

[1/2] ambari git commit: AMBARI-8500. Metric Collector dies if HBase is not initialized.

Repository: ambari
Updated Branches:
  refs/heads/branch-metrics-dev 223465201 -> 11b9c2392


AMBARI-8500. Metric Collector dies if HBase is not initialized.


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

Branch: refs/heads/branch-metrics-dev
Commit: a5c06b2272411fa636277f8aef9131c30eae39bd
Parents: 2234652
Author: Siddharth Wagle <sw...@hortonworks.com>
Authored: Mon Dec 1 11:17:48 2014 -0800
Committer: Siddharth Wagle <sw...@hortonworks.com>
Committed: Mon Dec 1 11:17:48 2014 -0800

----------------------------------------------------------------------
 .../pom.xml                                     |  7 +-
 .../metrics/timeline/ConnectionProvider.java    |  3 +-
 .../timeline/DefaultPhoenixDataSource.java      | 11 +--
 .../timeline/HBaseTimelineMetricStore.java      |  2 +-
 .../MetricsInitializationException.java         | 42 +++++++++
 .../metrics/timeline/PhoenixHBaseAccessor.java  | 99 ++++++++++++++++++--
 .../timeline/TimelineMetricConfiguration.java   |  6 ++
 .../src/test/resources/log4j.properties         |  1 -
 .../src/test/resources/logging.properties       |  3 -
 9 files changed, 145 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/a5c06b22/ambari-metrics/ambari-metrics-hadoop-timelineservice/pom.xml
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-hadoop-timelineservice/pom.xml b/ambari-metrics/ambari-metrics-hadoop-timelineservice/pom.xml
index 6115d7b..9bf0f51 100644
--- a/ambari-metrics/ambari-metrics-hadoop-timelineservice/pom.xml
+++ b/ambari-metrics/ambari-metrics-hadoop-timelineservice/pom.xml
@@ -240,13 +240,8 @@
       <plugin>
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
+          <redirectTestOutputToFile>true</redirectTestOutputToFile>
           <forkMode>always</forkMode>
-          <systemProperties>
-            <property>
-              <name>java.util.logging.config.file</name>
-              <value>src/test/resources/logging.properties</value>
-            </property>
-          </systemProperties>
         </configuration>
       </plugin>
     </plugins>

http://git-wip-us.apache.org/repos/asf/ambari/blob/a5c06b22/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/ConnectionProvider.java
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/ConnectionProvider.java b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/ConnectionProvider.java
index 87650af..47435f4 100644
--- a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/ConnectionProvider.java
+++ b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/ConnectionProvider.java
@@ -20,10 +20,11 @@ package org.apache.hadoop.yarn.server.applicationhistoryservice.metrics
 
 
 import java.sql.Connection;
+import java.sql.SQLException;
 
 /**
  *
  */
 public interface ConnectionProvider {
-  public Connection getConnection();
+  public Connection getConnection() throws SQLException;
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/a5c06b22/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/DefaultPhoenixDataSource.java
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/DefaultPhoenixDataSource.java b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/DefaultPhoenixDataSource.java
index 679ee36..652c492 100644
--- a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/DefaultPhoenixDataSource.java
+++ b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/DefaultPhoenixDataSource.java
@@ -63,19 +63,16 @@ public class DefaultPhoenixDataSource implements ConnectionProvider {
    *
    * @return @java.sql.Connection
    */
-  public Connection getConnection() {
-    Connection connection = null;
+  public Connection getConnection() throws SQLException {
 
     LOG.debug("Metric store connection url: " + url);
     try {
-      // TODO: Exception is swallowed, it should be thrown - discuss it
-
-      connection = DriverManager.getConnection(url);
+      return DriverManager.getConnection(url);
     } catch (SQLException e) {
       LOG.warn("Unable to connect to HBase store using Phoenix.", e);
-    }
 
-    return connection;
+      throw e;
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/a5c06b22/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 a3eb731..9364187 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
@@ -88,7 +88,7 @@ public class HBaseTimelineMetricStore extends AbstractService
                                    Configuration metricsConf) {
     hBaseAccessor = new PhoenixHBaseAccessor(hbaseConf, metricsConf);
     hBaseAccessor.initMetricSchema();
-//...BUG...
+
     // Start the cluster aggregator
     TimelineMetricClusterAggregator minuteClusterAggregator =
       new TimelineMetricClusterAggregator(hBaseAccessor, metricsConf);

http://git-wip-us.apache.org/repos/asf/ambari/blob/a5c06b22/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/MetricsInitializationException.java
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/MetricsInitializationException.java b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/MetricsInitializationException.java
new file mode 100644
index 0000000..88a427a
--- /dev/null
+++ b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/MetricsInitializationException.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.applicationhistoryservice.metrics
+  .timeline;
+
+/**
+ * RuntimeException for initialization of metrics schema. It is RuntimeException
+ * since this is a not recoverable situation, and should be handled by main or
+ * service method followed by shutdown.
+ */
+public class MetricsInitializationException extends RuntimeException {
+  public MetricsInitializationException() {
+  }
+
+  public MetricsInitializationException(String msg) {
+    super(msg);
+  }
+
+  public MetricsInitializationException(Throwable t) {
+    super(t);
+  }
+
+  public MetricsInitializationException(String msg, Throwable t) {
+    super(msg, t);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/a5c06b22/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 269a2d7..d18c215 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
@@ -18,9 +18,12 @@
 package org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline;
 
 
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.hadoop.hbase.util.RetryCounterFactory;
 import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
 import org.apache.hadoop.metrics2.sink.timeline.TimelineMetrics;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
@@ -28,25 +31,74 @@ import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.type.TypeReference;
 
 import java.io.IOException;
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics
+  .timeline.PhoenixTransactSQL.CREATE_METRICS_AGGREGATE_HOURLY_TABLE_SQL;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics
+  .timeline.PhoenixTransactSQL.CREATE_METRICS_AGGREGATE_MINUTE_TABLE_SQL;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics
+  .timeline.PhoenixTransactSQL
+  .CREATE_METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_SQL;
+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_TABLE_NAME;
+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_CLUSTER_AGGREGATE_TIME_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.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.PhoenixTransactSQL.*;
+  .timeline.TimelineMetricConfiguration.GLOBAL_RESULT_LIMIT;
 import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics
-  .timeline.TimelineMetricConfiguration.*;
+  .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.GLOBAL_MAX_RETRIES;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics
+  .timeline.TimelineMetricConfiguration.GLOBAL_RETRY_INTERVAL;
 
 /**
  * Provides a facade over the Phoenix API to access HBase schema
  */
 public class PhoenixHBaseAccessor {
 
+  private static final Log LOG = LogFactory.getLog(PhoenixHBaseAccessor.class);
   private final Configuration hbaseConf;
   private final Configuration metricsConf;
-  static final Log LOG = LogFactory.getLog(PhoenixHBaseAccessor.class);
+  private final RetryCounterFactory retryCounterFactory;
 
   static final int PHOENIX_MAX_MUTATION_STATE_SIZE = 50000;
   /**
@@ -55,11 +107,8 @@ public class PhoenixHBaseAccessor {
   private static final int METRICS_PER_MINUTE = 4;
   public static int RESULTSET_LIMIT = (int)TimeUnit.DAYS.toMinutes(1) *
     METRICS_PER_MINUTE;
-  private static ObjectMapper mapper;
+  private static ObjectMapper mapper = new ObjectMapper();
 
-  static {
-    mapper = new ObjectMapper();
-  }
   private static TypeReference<Map<Long, Double>> metricValuesTypeRef =
     new TypeReference<Map<Long, Double>>() {};
   private final ConnectionProvider dataSource;
@@ -82,8 +131,30 @@ public class PhoenixHBaseAccessor {
       throw new IllegalStateException(e);
     }
     this.dataSource = dataSource;
+    this.retryCounterFactory = new RetryCounterFactory(
+      metricsConf.getInt(GLOBAL_MAX_RETRIES, 10),
+      (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
@@ -93,7 +164,7 @@ public class PhoenixHBaseAccessor {
    *
    * @return @java.sql.Connection
    */
-  public Connection getConnection() {
+  public Connection getConnection() throws SQLException {
     return dataSource.getConnection();
   }
 
@@ -169,7 +240,7 @@ public class PhoenixHBaseAccessor {
   }
 
   protected void initMetricSchema() {
-    Connection conn = getConnection();
+    Connection conn = null;
     Statement stmt = null;
 
     String encoding = metricsConf.get(HBASE_ENCODING_SCHEME, DEFAULT_ENCODING);
@@ -182,7 +253,9 @@ public class PhoenixHBaseAccessor {
 
     try {
       LOG.info("Initializing metrics schema...");
+      conn = getConnectionRetryingOnException();
       stmt = conn.createStatement();
+
       stmt.executeUpdate(String.format(CREATE_METRICS_TABLE_SQL,
         encoding, precisionTtl, compression));
       stmt.executeUpdate(String.format(CREATE_METRICS_AGGREGATE_HOURLY_TABLE_SQL,
@@ -196,6 +269,12 @@ public class PhoenixHBaseAccessor {
       conn.commit();
     } catch (SQLException sql) {
       LOG.warn("Error creating Metrics Schema in HBase using Phoenix.", sql);
+      throw new MetricsInitializationException(
+        "Error creating Metrics Schema in HBase using Phoenix.", sql);
+    } catch (InterruptedException e) {
+      LOG.warn("Error creating Metrics Schema in HBase using Phoenix.", e);
+      throw new MetricsInitializationException(
+        "Error creating Metrics Schema in HBase using Phoenix.", e);
     } finally {
       if (stmt != null) {
         try {

http://git-wip-us.apache.org/repos/asf/ambari/blob/a5c06b22/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
index d17b1e4..60833d0 100644
--- 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
@@ -92,6 +92,12 @@ public interface TimelineMetricConfiguration {
   public static final String GLOBAL_RESULT_LIMIT =
     "timeline.metrics.service.default.result.limit";
 
+  public static final String GLOBAL_MAX_RETRIES =
+    "timeline.metrics.service.default.max_retries";
+
+  public static final String GLOBAL_RETRY_INTERVAL =
+    "timeline.metrics.service.default.retryInterval";
+
   public static final String HOST_AGGREGATOR_MINUTE_DISABLED =
     "timeline.metrics.host.aggregator.minute.disabled";
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/a5c06b22/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/test/resources/log4j.properties b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/test/resources/log4j.properties
deleted file mode 100644
index 8520d48..0000000
--- a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/test/resources/log4j.properties
+++ /dev/null
@@ -1 +0,0 @@
-log4j.rootLogger=OFF

http://git-wip-us.apache.org/repos/asf/ambari/blob/a5c06b22/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/test/resources/logging.properties
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/test/resources/logging.properties b/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/test/resources/logging.properties
deleted file mode 100644
index f5edede..0000000
--- a/ambari-metrics/ambari-metrics-hadoop-timelineservice/src/test/resources/logging.properties
+++ /dev/null
@@ -1,3 +0,0 @@
-.level = OFF
-java.util.logging.ConsoleHandler.level=OFF
-java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter


[2/2] ambari git commit: AMBARI-8500. Metric Collector dies if HBase is not initialized. Multiline imports fixed.

Posted by sw...@apache.org.
AMBARI-8500. Metric Collector dies if HBase is not initialized. Multiline imports fixed.


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

Branch: refs/heads/branch-metrics-dev
Commit: 11b9c239280f5db4ad4ad38bc2eb7139f6d9f0cd
Parents: a5c06b2
Author: Siddharth Wagle <sw...@hortonworks.com>
Authored: Mon Dec 1 11:29:14 2014 -0800
Committer: Siddharth Wagle <sw...@hortonworks.com>
Committed: Mon Dec 1 11:29:14 2014 -0800

----------------------------------------------------------------------
 .../metrics/timeline/PhoenixHBaseAccessor.java  | 73 ++++++--------------
 1 file changed, 23 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/11b9c239/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 d18c215..4f248b7 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
@@ -18,7 +18,6 @@
 package org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline;
 
 
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -29,7 +28,6 @@ import org.apache.hadoop.metrics2.sink.timeline.TimelineMetrics;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.type.TypeReference;
-
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
@@ -40,55 +38,30 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
-
 import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics
-  .timeline.PhoenixTransactSQL.CREATE_METRICS_AGGREGATE_HOURLY_TABLE_SQL;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics
-  .timeline.PhoenixTransactSQL.CREATE_METRICS_AGGREGATE_MINUTE_TABLE_SQL;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics
-  .timeline.PhoenixTransactSQL
-  .CREATE_METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_SQL;
-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_TABLE_NAME;
-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_CLUSTER_AGGREGATE_TIME_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.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.GLOBAL_RESULT_LIMIT;
-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.GLOBAL_MAX_RETRIES;
-import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics
-  .timeline.TimelineMetricConfiguration.GLOBAL_RETRY_INTERVAL;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.CREATE_METRICS_AGGREGATE_HOURLY_TABLE_SQL;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.CREATE_METRICS_AGGREGATE_MINUTE_TABLE_SQL;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.PhoenixTransactSQL.CREATE_METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_SQL;
+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_TABLE_NAME;
+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_CLUSTER_AGGREGATE_TIME_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.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.GLOBAL_MAX_RETRIES;
+import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.GLOBAL_RESULT_LIMIT;
+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_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;
 
 /**
  * Provides a facade over the Phoenix API to access HBase schema