You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by ds...@apache.org on 2015/08/11 18:06:32 UTC

ambari git commit: AMBARI-12718 AMS is doing a (Select *) for certain point in time metrics from Dashboard page (dsen)

Repository: ambari
Updated Branches:
  refs/heads/trunk 09f610171 -> 18c6130f2


AMBARI-12718 AMS is doing a (Select *) for certain point in time metrics from Dashboard page (dsen)


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

Branch: refs/heads/trunk
Commit: 18c6130f2682dffb1c34f10b0b18a9a2ca89404e
Parents: 09f6101
Author: Dmytro Sen <ds...@apache.org>
Authored: Tue Aug 11 19:06:24 2015 +0300
Committer: Dmytro Sen <ds...@apache.org>
Committed: Tue Aug 11 19:06:24 2015 +0300

----------------------------------------------------------------------
 .../timeline/query/PhoenixTransactSQL.java      | 127 +++++++++++--------
 .../timeline/TestPhoenixTransactSQL.java        |  52 ++++++++
 2 files changed, 128 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/18c6130f/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/query/PhoenixTransactSQL.java
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/query/PhoenixTransactSQL.java b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/query/PhoenixTransactSQL.java
index 4ca1b4b..ed80474 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/query/PhoenixTransactSQL.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/query/PhoenixTransactSQL.java
@@ -161,28 +161,49 @@ public class PhoenixTransactSQL {
 
   /**
    * Get latest metrics for a number of hosts
+   *
+   * Different queries for a number and a single hosts are used due to bug
+   * in Apache Phoenix
    */
   public static final String GET_LATEST_METRIC_SQL = "SELECT " +
-      "E.METRIC_NAME AS METRIC_NAME, E.HOSTNAME AS HOSTNAME, " +
-      "E.APP_ID AS APP_ID, E.INSTANCE_ID AS INSTANCE_ID, " +
-      "E.SERVER_TIME AS SERVER_TIME, E.START_TIME AS START_TIME, " +
-      "E.UNITS AS UNITS, E.METRIC_SUM AS METRIC_SUM, " +
-      "E.METRIC_MAX AS METRIC_MAX, E.METRIC_MIN AS METRIC_MIN, " +
-      "E.METRIC_COUNT AS METRIC_COUNT, E.METRICS AS METRICS " +
-      "FROM %s AS E " +
-      "INNER JOIN " +
-      "(SELECT METRIC_NAME, HOSTNAME, MAX(SERVER_TIME) AS MAX_SERVER_TIME, " +
-      "APP_ID, INSTANCE_ID " +
-      "FROM %s " +
-      "WHERE " +
-      "%s " +
-      "GROUP BY METRIC_NAME, HOSTNAME, APP_ID, INSTANCE_ID) " +
-      "AS I " +
-      "ON E.METRIC_NAME=I.METRIC_NAME " +
-      "AND E.HOSTNAME=I.HOSTNAME " +
-      "AND E.SERVER_TIME=I.MAX_SERVER_TIME " +
-      "AND E.APP_ID=I.APP_ID " +
-      "AND E.INSTANCE_ID=I.INSTANCE_ID";
+    "E.METRIC_NAME AS METRIC_NAME, E.HOSTNAME AS HOSTNAME, " +
+    "E.APP_ID AS APP_ID, E.INSTANCE_ID AS INSTANCE_ID, " +
+    "E.SERVER_TIME AS SERVER_TIME, E.START_TIME AS START_TIME, " +
+    "E.UNITS AS UNITS, E.METRIC_SUM AS METRIC_SUM, " +
+    "E.METRIC_MAX AS METRIC_MAX, E.METRIC_MIN AS METRIC_MIN, " +
+    "E.METRIC_COUNT AS METRIC_COUNT, E.METRICS AS METRICS " +
+    "FROM %s AS E " +
+    "INNER JOIN " +
+    "(SELECT METRIC_NAME, HOSTNAME, MAX(SERVER_TIME) AS MAX_SERVER_TIME, " +
+    "APP_ID, INSTANCE_ID " +
+    "FROM %s " +
+    "WHERE " +
+    "%s " +
+    "GROUP BY METRIC_NAME, HOSTNAME, APP_ID, INSTANCE_ID) " +
+    "AS I " +
+    "ON E.METRIC_NAME=I.METRIC_NAME " +
+    "AND E.HOSTNAME=I.HOSTNAME " +
+    "AND E.SERVER_TIME=I.MAX_SERVER_TIME " +
+    "AND E.APP_ID=I.APP_ID " +
+    "AND E.INSTANCE_ID=I.INSTANCE_ID";
+
+  /**
+   * Get latest metrics for a single host
+   *
+   * Different queries for a number of hosts and a single host are used due to
+   * bug in Apache Phoenix
+   */
+  public static final String GET_LATEST_METRIC_SQL_SINGLE_HOST = "SELECT " +
+    "METRIC_NAME, HOSTNAME, APP_ID, INSTANCE_ID, SERVER_TIME, START_TIME, " +
+    "UNITS, METRIC_SUM, METRIC_MAX, METRIC_MIN, METRIC_COUNT, METRICS " +
+    "FROM %s " +
+    "WHERE %s " +
+    "AND (METRIC_NAME, HOSTNAME, SERVER_TIME, APP_ID, INSTANCE_ID) = ANY " +
+    "(SELECT " +
+    "METRIC_NAME, HOSTNAME, MAX(SERVER_TIME) AS SERVER_TIME, APP_ID, INSTANCE_ID " +
+    "FROM %s " +
+    "WHERE %s " +
+    "GROUP BY METRIC_NAME, HOSTNAME, APP_ID, INSTANCE_ID)";
 
   public static final String GET_METRIC_AGGREGATE_ONLY_SQL = "SELECT %s " +
     "METRIC_NAME, HOSTNAME, APP_ID, INSTANCE_ID, SERVER_TIME, " +
@@ -412,9 +433,9 @@ public class PhoenixTransactSQL {
     validateConditionIsNotEmpty(condition);
 
     if (condition.getMetricNames() == null
-            || condition.getMetricNames().isEmpty()) {
+      || condition.getMetricNames().isEmpty()) {
       throw new IllegalArgumentException("Point in time query without "
-              + "metric names not supported ");
+        + "metric names not supported ");
     }
 
     String stmtStr;
@@ -423,24 +444,17 @@ public class PhoenixTransactSQL {
     } else {
       //if not a single metric for a single host
       if (condition.getHostnames().size() > 1
-              && condition.getMetricNames().size() > 1) {
+        && condition.getMetricNames().size() > 1) {
         stmtStr = String.format(GET_LATEST_METRIC_SQL,
-                METRICS_RECORD_TABLE_NAME,
-                METRICS_RECORD_TABLE_NAME,
-                condition.getConditionClause());
+          METRICS_RECORD_TABLE_NAME,
+          METRICS_RECORD_TABLE_NAME,
+          condition.getConditionClause());
       } else {
-        StringBuilder sb = new StringBuilder(String.format(GET_METRIC_SQL,
-                "",
-                METRICS_RECORD_TABLE_NAME));
-        sb.append(" WHERE ");
-        sb.append(condition.getConditionClause());
-        String orderByClause = condition.getOrderByClause(false);
-        if (orderByClause != null) {
-          sb.append(orderByClause);
-        } else {
-          sb.append(" ORDER BY METRIC_NAME DESC, HOSTNAME DESC, SERVER_TIME DESC ");
-        }
-        stmtStr = sb.toString();
+        stmtStr = String.format(GET_LATEST_METRIC_SQL_SINGLE_HOST,
+          METRICS_RECORD_TABLE_NAME,
+          condition.getConditionClause(),
+          METRICS_RECORD_TABLE_NAME,
+          condition.getConditionClause());
       }
     }
 
@@ -450,14 +464,28 @@ public class PhoenixTransactSQL {
     PreparedStatement stmt = null;
     try {
       stmt = connection.prepareStatement(stmtStr);
-      int pos = 1;
+      setQueryParameters(stmt, condition);
+    } catch (SQLException e) {
+      if (stmt != null) {
+        stmt.close();
+      }
+      throw e;
+    }
+
+    return stmt;
+  }
+  private static PreparedStatement setQueryParameters(PreparedStatement stmt,
+                                                      Condition condition)
+    throws SQLException {
+    int pos = 1;
+    //For GET_LATEST_METRIC_SQL_SINGLE_HOST parameters should be set 2 times
+    do {
       if (condition.getMetricNames() != null) {
-        //IGNORE condition limit, set one based on number of metric names
-        for (; pos <= condition.getMetricNames().size(); pos++) {
+        for (String metricName: condition.getMetricNames()) {
           if (LOG.isDebugEnabled()) {
-            LOG.debug("Setting pos: " + pos + ", value = " + condition.getMetricNames().get(pos - 1));
+            LOG.debug("Setting pos: " + pos + ", value = " + metricName);
           }
-          stmt.setString(pos, condition.getMetricNames().get(pos - 1));
+          stmt.setString(pos++, metricName);
         }
       }
       if (condition.getHostnames() != null) {
@@ -476,20 +504,17 @@ public class PhoenixTransactSQL {
       }
       if (condition.getInstanceId() != null) {
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Setting pos: " + pos + ", value: " + condition.getInstanceId());
+          LOG.debug("Setting pos: " + pos +
+            ", value: " + condition.getInstanceId());
         }
-        stmt.setString(pos, condition.getInstanceId());
+        stmt.setString(pos++, condition.getInstanceId());
       }
 
       if (condition.getFetchSize() != null) {
         stmt.setFetchSize(condition.getFetchSize());
+        pos++;
       }
-    } catch (SQLException e) {
-      if (stmt != null) {
-        stmt.close();
-      }
-      throw e;
-    }
+    } while (pos < stmt.getParameterMetaData().getParameterCount());
 
     return stmt;
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/18c6130f/ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TestPhoenixTransactSQL.java
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TestPhoenixTransactSQL.java b/ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TestPhoenixTransactSQL.java
index ee7a468..98738af 100644
--- a/ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TestPhoenixTransactSQL.java
+++ b/ambari-metrics/ambari-metrics-timelineservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/TestPhoenixTransactSQL.java
@@ -25,6 +25,7 @@ import org.easymock.Capture;
 import org.junit.Assert;
 import org.junit.Test;
 import java.sql.Connection;
+import java.sql.ParameterMetaData;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.util.Arrays;
@@ -222,6 +223,57 @@ public class TestPhoenixTransactSQL {
   }
 
   @Test
+  public void testPrepareGetLatestMetricSqlStmtSingleHostName() throws SQLException {
+    Condition condition = new DefaultCondition(
+      Arrays.asList("cpu_user", "mem_free"), Collections.singletonList("h1"),
+      "a1", "i1", null, null, null, null, false);
+    Connection connection = createNiceMock(Connection.class);
+    PreparedStatement preparedStatement = createNiceMock(PreparedStatement.class);
+    ParameterMetaData parameterMetaData = createNiceMock(ParameterMetaData.class);
+    Capture<String> stmtCapture = new Capture<String>();
+    expect(connection.prepareStatement(EasyMock.and(EasyMock.anyString(), EasyMock.capture(stmtCapture))))
+        .andReturn(preparedStatement);
+    expect(preparedStatement.getParameterMetaData())
+      .andReturn(parameterMetaData).times(2);
+    // 10 = (1 instance_id + 1 appd_id + 1 hostname + 2 metric names) * 2,
+    // For GET_LATEST_METRIC_SQL_SINGLE_HOST parameters should be set 2 times
+    expect(parameterMetaData.getParameterCount())
+      .andReturn(10).times(2);
+
+    replay(connection, preparedStatement, parameterMetaData);
+    PhoenixTransactSQL.prepareGetLatestMetricSqlStmt(connection, condition);
+    String stmt = stmtCapture.getValue();
+    Assert.assertTrue(stmt.contains("FROM METRIC_RECORD"));
+    Assert.assertTrue(stmt.contains("ANY"));
+    verify(connection, preparedStatement, parameterMetaData);
+  }
+
+  @Test
+  public void testPrepareGetLatestMetricSqlStmtMultipleHostNames() throws SQLException {
+    Condition condition = new DefaultCondition(
+      Arrays.asList("cpu_user", "mem_free"), Arrays.asList("h1", "h2"),
+      "a1", "i1", null, null, null, null, false);
+    Connection connection = createNiceMock(Connection.class);
+    PreparedStatement preparedStatement = createNiceMock(PreparedStatement.class);
+    ParameterMetaData parameterMetaData = createNiceMock(ParameterMetaData.class);
+    Capture<String> stmtCapture = new Capture<String>();
+    expect(connection.prepareStatement(EasyMock.and(EasyMock.anyString(), EasyMock.capture(stmtCapture))))
+        .andReturn(preparedStatement);
+    expect(preparedStatement.getParameterMetaData())
+      .andReturn(parameterMetaData).once();
+    // 6 = 1 instance_id + 1 appd_id + 2 hostnames + 2 metric names
+    expect(parameterMetaData.getParameterCount())
+      .andReturn(6).once();
+
+    replay(connection, preparedStatement, parameterMetaData);
+    PhoenixTransactSQL.prepareGetLatestMetricSqlStmt(connection, condition);
+    String stmt = stmtCapture.getValue();
+    Assert.assertTrue(stmt.contains("FROM METRIC_RECORD"));
+    Assert.assertTrue(stmt.contains("JOIN"));
+    verify(connection, preparedStatement, parameterMetaData);
+  }
+
+  @Test
   public void testPrepareGetMetricsPrecisionHours() throws SQLException {
     Condition condition = new DefaultCondition(
       Arrays.asList("cpu_user", "mem_free"), Collections.singletonList("h1"),