You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by sr...@apache.org on 2015/01/23 04:39:36 UTC

ambari git commit: AMBARI-9282. Update Slider View to consume metrics from AMS (Ambari Metrics Service) (Gour Saha via srimanth)

Repository: ambari
Updated Branches:
  refs/heads/trunk ae49f7c96 -> a964003c7


AMBARI-9282. Update Slider View to consume metrics from AMS (Ambari Metrics Service) (Gour Saha via srimanth)


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

Branch: refs/heads/trunk
Commit: a964003c7f8c83b254d6279271fb3d1296091765
Parents: ae49f7c
Author: Srimanth Gunturi <sg...@hortonworks.com>
Authored: Thu Jan 22 19:38:00 2015 -0800
Committer: Srimanth Gunturi <sg...@hortonworks.com>
Committed: Thu Jan 22 19:38:00 2015 -0800

----------------------------------------------------------------------
 .../ambari/view/slider/GangliaMetric.java       | 257 -------------------
 .../ambari/view/slider/MetricsHolder.java       |  10 +-
 .../slider/SliderAppsViewControllerImpl.java    |  26 +-
 .../rest/client/SliderAppGangliaHelper.java     | 196 --------------
 .../rest/client/SliderAppMasterClient.java      |  38 +--
 .../rest/client/SliderAppMetricsHelper.java     | 151 +++++++++++
 .../view/slider/rest/client/TimelineMetric.java | 172 +++++++++++++
 .../slider/rest/client/TimelineMetrics.java     | 101 ++++++++
 8 files changed, 463 insertions(+), 488 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/a964003c/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/GangliaMetric.java
----------------------------------------------------------------------
diff --git a/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/GangliaMetric.java b/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/GangliaMetric.java
deleted file mode 100644
index cb51768..0000000
--- a/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/GangliaMetric.java
+++ /dev/null
@@ -1,257 +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.ambari.view.slider;
-
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-@JsonIgnoreProperties(ignoreUnknown = true)
-
-
-/**
- * Data structure for temporal data returned from Ganglia Web.
- */
-public class GangliaMetric {
-
-  // Note that the member names correspond to the names in the JSON returned from Ganglia Web.
-
-  /**
-   * The name.
-   */
-  private String ds_name;
-
-  /**
-   * The ganglia cluster name.
-   */
-  private String cluster_name;
-
-  /**
-   * The graph type.
-   */
-  private String graph_type;
-
-  /**
-   * The host name.
-   */
-  private String host_name;
-
-  /**
-   * The metric name.
-   */
-  private String metric_name;
-
-  /**
-   * The temporal data points.
-   */
-  private Number[][] datapoints;
-  
-  
-  private static final Set<String> PERCENTAGE_METRIC;
-
-  //BUG-3386 Cluster CPU Chart is off the charts
-  // Here can be added other percentage metrics
-  static {
-    Set<String> temp = new HashSet<String>();
-    temp.add("cpu_wio");
-    temp.add("cpu_idle");
-    temp.add("cpu_nice");
-    temp.add("cpu_aidle");
-    temp.add("cpu_system");
-    temp.add("cpu_user");
-    PERCENTAGE_METRIC = Collections.unmodifiableSet(temp);
-  }
-
-
-  // ----- GangliaMetric -----------------------------------------------------
-
-  public String getDs_name() {
-    return ds_name;
-  }
-
-  public void setDs_name(String ds_name) {
-    this.ds_name = ds_name;
-  }
-
-  public String getCluster_name() {
-    return cluster_name;
-  }
-
-  public void setCluster_name(String cluster_name) {
-    this.cluster_name = cluster_name;
-  }
-
-  public String getGraph_type() {
-    return graph_type;
-  }
-
-  public void setGraph_type(String graph_type) {
-    this.graph_type = graph_type;
-  }
-
-  public String getHost_name() {
-    return host_name;
-  }
-
-  public void setHost_name(String host_name) {
-    this.host_name = host_name;
-  }
-
-  public String getMetric_name() {
-    return metric_name;
-  }
-
-  public void setMetric_name(String metric_name) {
-    this.metric_name = metric_name;
-  }
-
-  public Number[][] getDatapoints() {
-    return datapoints;
-  }
-
-
-  public void setDatapoints(Number[][] datapoints) {
-    this.datapoints = datapoints;
-  } 
-  
-  public void setDatapointsFromList(List<GangliaMetric.TemporalMetric> listTemporalMetrics) { 
-    //this.datapoints = datapoints;
-    Number[][] datapointsArray = new Number[listTemporalMetrics.size()][2];
-    int cnt = 0;
-    if (PERCENTAGE_METRIC.contains(metric_name)) {
-      int firstIndex = 0;
-      int lastIndex = listTemporalMetrics.size() - 1;
-      for (int i = firstIndex; i <= lastIndex; ++i) {
-        GangliaMetric.TemporalMetric m = listTemporalMetrics.get(i);
-        Number val = m.getValue();
-        if (100.0 >= val.doubleValue()) {
-          datapointsArray[cnt][0] = val;
-          datapointsArray[cnt][1] = m.getTime();
-          cnt++;
-        }
-      }
-    } else {
-      int firstIndex = 0;
-      int lastIndex = listTemporalMetrics.size() - 1;
-      for (int i = firstIndex; i <= lastIndex; ++i) {
-        GangliaMetric.TemporalMetric m = listTemporalMetrics.get(i);
-        datapointsArray[i][0] = m.getValue();
-        datapointsArray[i][1] = m.getTime();
-        cnt++;
-      }
-    }
-
-    this.datapoints = new Number[cnt][2];
-    for (int i = 0; i < this.datapoints.length; i++) {
-      this.datapoints[i][0] = datapointsArray[i][0];
-      this.datapoints[i][1] = datapointsArray[i][1];
-    }
-
-  }
-
-  // ----- Object overrides --------------------------------------------------
-
-  @Override
-  public String toString() {
-    StringBuilder stringBuilder = new StringBuilder();
-
-    stringBuilder.append("\n");
-    stringBuilder.append("name=");
-    stringBuilder.append(ds_name);
-    stringBuilder.append("\n");
-    stringBuilder.append("cluster name=");
-    stringBuilder.append(cluster_name);
-    stringBuilder.append("\n");
-    stringBuilder.append("graph type=");
-    stringBuilder.append(graph_type);
-    stringBuilder.append("\n");
-    stringBuilder.append("host name=");
-    stringBuilder.append(host_name);
-    stringBuilder.append("\n");
-    stringBuilder.append("api name=");
-    stringBuilder.append(metric_name);
-    stringBuilder.append("\n");
-
-    stringBuilder.append("datapoints (value/timestamp):");
-    stringBuilder.append("\n");
-
-
-    boolean first = true;
-    stringBuilder.append("[");
-    for (Number[] m : datapoints) {
-      if (!first) {
-        stringBuilder.append(",");
-      }
-      stringBuilder.append("[");
-      stringBuilder.append(m[0]);
-      stringBuilder.append(",");
-      stringBuilder.append(m[1].longValue());
-      stringBuilder.append("]");
-      first = false;
-    }
-    stringBuilder.append("]");
-
-    return stringBuilder.toString();
-  }
-
-  public static class TemporalMetric {
-    private Number m_value;
-    private Number m_time;
-    private boolean valid;
-
-    public boolean isValid() {
-      return valid;
-    }
-
-    public TemporalMetric(String value, Number time) {
-      valid = true;
-      try{
-        m_value = convertToNumber(value);
-      } catch (NumberFormatException e) {
-        valid = false;
-      }
-      m_time = time;
-    }
-
-    public Number getValue() {
-      return m_value;
-    }
-
-    public Number getTime() {
-      return m_time;
-    }
-    
-    private Number convertToNumber(String s) throws NumberFormatException {
-      Number res;
-      if(s.contains(".")){
-        Double d = Double.parseDouble(s);
-        if(d.isNaN() || d.isInfinite()){
-          throw new NumberFormatException(s);
-        } else {
-          res = d;
-        } 
-      } else {
-        res = Long.parseLong(s);
-      }
-      return res;
-    }
-    
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/a964003c/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/MetricsHolder.java
----------------------------------------------------------------------
diff --git a/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/MetricsHolder.java b/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/MetricsHolder.java
index 29f35b0..212092e 100644
--- a/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/MetricsHolder.java
+++ b/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/MetricsHolder.java
@@ -24,7 +24,7 @@ import java.util.Map;
 
 public class MetricsHolder {
   private Map<String, Map<String, Map<String, Metric>>> jmxMetrics;
-  private Map<String, Map<String, Map<String, Metric>>> gangliaMetrics;
+  private Map<String, Map<String, Map<String, Metric>>> timelineMetrics;
 
   public Map<String, Map<String, Map<String, Metric>>> getJmxMetrics() {
     return jmxMetrics;
@@ -34,11 +34,11 @@ public class MetricsHolder {
     this.jmxMetrics = jmxMetrics;
   }
 
-  public Map<String, Map<String, Map<String, Metric>>> getGangliaMetrics() {
-    return gangliaMetrics;
+  public Map<String, Map<String, Map<String, Metric>>> getTimelineMetrics() {
+    return timelineMetrics;
   }
 
-  public void setGangliaMetrics(Map<String, Map<String, Map<String, Metric>>> gangliaMetrics) {
-    this.gangliaMetrics = gangliaMetrics;
+  public void setTimelineMetrics(Map<String, Map<String, Map<String, Metric>>> timelineMetrics) {
+    this.timelineMetrics = timelineMetrics;
   }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/a964003c/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/SliderAppsViewControllerImpl.java
----------------------------------------------------------------------
diff --git a/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/SliderAppsViewControllerImpl.java b/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/SliderAppsViewControllerImpl.java
index 2593096..f2b8b6c 100644
--- a/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/SliderAppsViewControllerImpl.java
+++ b/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/SliderAppsViewControllerImpl.java
@@ -649,7 +649,7 @@ public class SliderAppsViewControllerImpl implements SliderAppsViewController {
           : new SliderAppMasterClient(yarnApp.getTrackingUrl());
       SliderAppMasterData appMasterData = null;
       Map<String, String> quickLinks = new HashMap<String, String>();
-      Set<String> gangliaMetrics = new HashSet<String>();
+      Set<String> metrics = new HashSet<String>();
       for (String property : properties) {
         if ("RUNNING".equals(app.getState())) {
           if (sliderAppClient != null) {
@@ -762,7 +762,7 @@ public class SliderAppsViewControllerImpl implements SliderAppsViewController {
                 throw new RuntimeException(e.getMessage(), e);
               }
             } else if (property.startsWith(METRICS_PREFIX)) {
-              gangliaMetrics.add(property.substring(METRICS_PREFIX.length()));
+              metrics.add(property.substring(METRICS_PREFIX.length()));
             } else if ("supportedMetrics".equals(property)) {
               if (matchedAppType != null) {
                 app.setSupportedMetrics(matchedAppType.getSupportedMetrics());
@@ -771,7 +771,7 @@ public class SliderAppsViewControllerImpl implements SliderAppsViewController {
           }
         }
       }
-      if (gangliaMetrics.size() > 0) {
+      if (metrics.size() > 0) {
         if (quickLinks.isEmpty()) {
           quickLinks = sliderAppClient
               .getQuickLinks(appMasterData.publisherUrl);
@@ -780,8 +780,8 @@ public class SliderAppsViewControllerImpl implements SliderAppsViewController {
           String metricsUrl = quickLinks.get(METRICS_API_NAME);
           MetricsHolder metricsHolder = appMetrics.get(matchedAppType
               .uniqueName());
-          app.setMetrics(sliderAppClient.getGangliaMetrics(metricsUrl,
-              gangliaMetrics, null, viewContext, matchedAppType, metricsHolder));
+          app.setMetrics(sliderAppClient.getMetrics(metricsUrl,
+              metrics, null, viewContext, matchedAppType, metricsHolder));
         }
       }
     }
@@ -1081,10 +1081,10 @@ public class SliderAppsViewControllerImpl implements SliderAppsViewController {
               MetricsHolder metricsHolder = new MetricsHolder();
               metricsHolder.setJmxMetrics(readMetrics(zipFile,
                   "jmx_metrics.json"));
-              metricsHolder.setGangliaMetrics(readMetrics(zipFile,
-                  "ganglia_metrics.json"));
+              metricsHolder.setTimelineMetrics(readMetrics(zipFile,
+                  "timeline_metrics.json"));
               appType.setSupportedMetrics(getSupportedMetrics(metricsHolder
-                  .getGangliaMetrics()));
+                  .getTimelineMetrics()));
               appMetrics.put(appType.uniqueName(), metricsHolder);
 
               appType.setTypeComponents(appTypeComponentList);
@@ -1127,13 +1127,13 @@ public class SliderAppsViewControllerImpl implements SliderAppsViewController {
   }
 
   private List<String> getSupportedMetrics(
-      Map<String, Map<String, Map<String, Metric>>> gangliaMetrics) {
+      Map<String, Map<String, Map<String, Metric>>> metrics) {
     Set<String> supportedMetrics = new HashSet<String>();
-    if (gangliaMetrics != null && gangliaMetrics.size() > 0) {
-      for (Map<String, Map<String, Metric>> compMetrics : gangliaMetrics
+    if (metrics != null && metrics.size() > 0) {
+      for (Map<String, Map<String, Metric>> compMetrics : metrics
           .values()) {
-        for (Map<String, Metric> metrics : compMetrics.values()) {
-          supportedMetrics.addAll(metrics.keySet());
+        for (Map<String, Metric> compMetric : compMetrics.values()) {
+          supportedMetrics.addAll(compMetric.keySet());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/ambari/blob/a964003c/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/SliderAppGangliaHelper.java
----------------------------------------------------------------------
diff --git a/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/SliderAppGangliaHelper.java b/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/SliderAppGangliaHelper.java
deleted file mode 100644
index fba7d23..0000000
--- a/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/SliderAppGangliaHelper.java
+++ /dev/null
@@ -1,196 +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.ambari.view.slider.rest.client;
-
-import org.apache.ambari.view.SystemException;
-import org.apache.ambari.view.ViewContext;
-import org.apache.ambari.view.slider.GangliaMetric;
-import org.apache.ambari.view.slider.TemporalInfo;
-import org.apache.log4j.Logger;
-import org.apache.http.client.utils.URIBuilder;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-public class SliderAppGangliaHelper {
-
-  private static final Logger logger = Logger
-      .getLogger(SliderAppGangliaHelper.class);
-
-  private static String getSetString(Set<String> set, int limit) {
-    StringBuilder sb = new StringBuilder();
-
-    if (limit == -1 || set.size() <= limit) {
-      for (String cluster : set) {
-        if (sb.length() > 0) {
-          sb.append(",");
-        }
-        sb.append(cluster);
-      }
-    }
-    return sb.toString();
-  }
-
-  private static Number convertToNumber(String s) {
-    return s.contains(".") ? Double.parseDouble(s) : Long.parseLong(s);
-  }
-
-  public static Map<String, GangliaMetric> getGangliaMetrics(ViewContext context,
-                                                             String spec,
-                                                             String params) throws IOException {
-    Map<String, GangliaMetric> receivedMetrics = new HashMap<String, GangliaMetric>();
-    Map<String, String> headers = new HashMap<String, String>();
-    BufferedReader reader = new BufferedReader(new InputStreamReader(
-        context.getURLStreamProvider().readFrom(spec, "POST", params, headers)));
-
-    String feedStart = reader.readLine();
-    if (feedStart == null || feedStart.isEmpty()) {
-      logger.info("Empty feed while getting ganglia metrics for spec => " +
-                  spec);
-      return null;
-    }
-    int startTime = convertToNumber(feedStart).intValue();
-
-    String dsName = reader.readLine();
-    if (dsName == null || dsName.isEmpty()) {
-      logger.info("Feed without body while reading ganglia metrics for spec " +
-                  "=> " + spec);
-      return null;
-    }
-
-    while (!"[~EOF]".equals(dsName)) {
-      GangliaMetric metric = new GangliaMetric();
-      List<GangliaMetric.TemporalMetric> listTemporalMetrics =
-          new ArrayList<GangliaMetric.TemporalMetric>();
-
-      metric.setDs_name(dsName);
-      metric.setCluster_name(reader.readLine());
-      metric.setHost_name(reader.readLine());
-      metric.setMetric_name(reader.readLine());
-
-      String timeStr = reader.readLine();
-      String stepStr = reader.readLine();
-      if (timeStr == null || timeStr.isEmpty() || stepStr == null
-          || stepStr.isEmpty()) {
-        logger.info("Unexpected end of stream reached while getting ganglia " +
-                    "metrics for spec => " + spec);
-        return null;
-      }
-      int time = convertToNumber(timeStr).intValue();
-      int step = convertToNumber(stepStr).intValue();
-
-      String val = reader.readLine();
-      String lastVal = null;
-
-      while (val != null && !"[~EOM]".equals(val)) {
-        if (val.startsWith("[~r]")) {
-          Integer repeat = Integer.valueOf(val.substring(4)) - 1;
-          for (int i = 0; i < repeat; ++i) {
-            if (!"[~n]".equals(lastVal)) {
-              GangliaMetric.TemporalMetric tm = new GangliaMetric.TemporalMetric(lastVal, time);
-              if (tm.isValid()) listTemporalMetrics.add(tm);
-            }
-            time += step;
-          }
-        } else {
-          if (!"[~n]".equals(val)) {
-            GangliaMetric.TemporalMetric tm = new GangliaMetric.TemporalMetric(val, time);
-            if (tm.isValid()) listTemporalMetrics.add(tm);
-          }
-          time += step;
-        }
-        lastVal = val;
-        val = reader.readLine();
-      }
-
-      metric.setDatapointsFromList(listTemporalMetrics);
-      receivedMetrics.put(metric.getMetric_name(), metric);
-
-      dsName = reader.readLine();
-      if (dsName == null || dsName.isEmpty()) {
-        logger.info("Unexpected end of stream reached while getting ganglia " +
-                    "metrics for spec => " + spec);
-        return null;
-      }
-    }
-    String feedEnd = reader.readLine();
-    if (feedEnd == null || feedEnd.isEmpty()) {
-      logger.info("Error reading end of feed while getting ganglia metrics " +
-                  "for spec => " + spec);
-    } else {
-
-      int endTime = convertToNumber(feedEnd).intValue();
-      int totalTime = endTime - startTime;
-      if (logger.isInfoEnabled() && totalTime > 3) {
-        logger.info("Ganglia resource population time: " + totalTime);
-      }
-    }
-    return receivedMetrics;
-  }
-
-  public static String getSpec(String gangliaUrl,
-                               Set<String> metricSet,
-                               TemporalInfo temporalInfo) throws SystemException, URISyntaxException {
-
-    String metrics = getSetString(metricSet, -1);
-
-    URIBuilder uriBuilder = new URIBuilder(gangliaUrl);
-
-    uriBuilder.setParameter("h", "__SummaryInfo__");
-
-    if (metrics.length() > 0) {
-      uriBuilder.setParameter("m", metrics);
-    } else {
-      // get all metrics
-      uriBuilder.setParameter("m", ".*");
-    }
-
-    if (temporalInfo != null) {
-      long startTime = temporalInfo.getStartTime();
-      if (startTime != -1) {
-        uriBuilder.setParameter("s", String.valueOf(startTime));
-      }
-
-      long endTime = temporalInfo.getEndTime();
-      if (endTime != -1) {
-        uriBuilder.setParameter("e", String.valueOf(endTime));
-      }
-
-      long step = temporalInfo.getStep();
-      if (step != -1) {
-        uriBuilder.setParameter("r", String.valueOf(step));
-      }
-    } else {
-      long endTime = System.currentTimeMillis() / 1000;
-      long startTime = System.currentTimeMillis() / 1000 - 60 * 60;
-      uriBuilder.setParameter("e", String.valueOf(endTime));
-      uriBuilder.setParameter("s", String.valueOf(startTime));
-      uriBuilder.setParameter("r", "15");
-    }
-
-    return uriBuilder.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/a964003c/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/SliderAppMasterClient.java
----------------------------------------------------------------------
diff --git a/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/SliderAppMasterClient.java b/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/SliderAppMasterClient.java
index 4b6624f..001036c 100644
--- a/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/SliderAppMasterClient.java
+++ b/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/SliderAppMasterClient.java
@@ -30,7 +30,6 @@ import java.util.Set;
 
 import org.apache.ambari.view.URLStreamProvider;
 import org.apache.ambari.view.ViewContext;
-import org.apache.ambari.view.slider.GangliaMetric;
 import org.apache.ambari.view.slider.MetricsHolder;
 import org.apache.ambari.view.slider.SliderAppType;
 import org.apache.ambari.view.slider.SliderAppTypeComponent;
@@ -158,26 +157,29 @@ public class SliderAppMasterClient extends BaseHttpClient {
     return configsMap;
   }
 
-  public Map<String, Number[][]> getGangliaMetrics(String gangliaUrl,
-                                                   Set<String> metricsRequested,
-                                                   TemporalInfo temporalInfo,
-                                                   ViewContext context,
-                                                   SliderAppType appType,
-                                                   MetricsHolder metricsHolder) {
+  public Map<String, Number[][]> getMetrics(String metricsUrl,
+                                            Set<String> metricsRequested,
+                                            TemporalInfo temporalInfo,
+                                            ViewContext context,
+                                            SliderAppType appType,
+                                            MetricsHolder metricsHolder) {
     Map<String, Number[][]> retVal = new HashMap<String, Number[][]>();
 
-    if (appType == null || metricsHolder == null || metricsHolder.getGangliaMetrics() == null) {
-      logger.info("AppType must be provided and it must contain ganglia_metrics.json to extract jmx properties");
+    if (appType == null || metricsHolder == null
+        || metricsHolder.getTimelineMetrics() == null) {
+      logger.info("AppType must be provided and it must contain "
+          + "timeline_metrics.json to extract jmx properties");
       return retVal;
     }
 
-    Map<String, GangliaMetric> receivedMetrics = null;
+    Map<String, Number[][]> receivedMetrics = null;
     List<String> components = new ArrayList<String>();
     for (SliderAppTypeComponent appTypeComponent : appType.getTypeComponents()) {
       components.add(appTypeComponent.getName());
     }
 
-    Map<String, Map<String, Map<String, Metric>>> metrics = metricsHolder.getGangliaMetrics();
+    Map<String, Map<String, Map<String, Metric>>> metrics = metricsHolder
+        .getTimelineMetrics();
     Map<String, Metric> relevantMetrics = getRelevantMetrics(metrics, components);
     Set<String> metricsToRead = new HashSet<String>();
     Map<String, String> reverseNameLookup = new HashMap<String, String>();
@@ -191,7 +193,8 @@ public class SliderAppMasterClient extends BaseHttpClient {
 
     if (metricsToRead.size() != 0) {
       try {
-        String specWithParams = SliderAppGangliaHelper.getSpec(gangliaUrl, metricsToRead, temporalInfo);
+        String specWithParams = SliderAppMetricsHelper.getUrlWithParams(
+            metricsUrl, metricsToRead, temporalInfo);
         logger.info("Using spec: " + specWithParams);
         if (specWithParams != null) {
 
@@ -206,17 +209,18 @@ public class SliderAppMasterClient extends BaseHttpClient {
             logger.info(e.toString());
           }
 
-          receivedMetrics = SliderAppGangliaHelper.getGangliaMetrics(context, spec, params);
+          receivedMetrics = SliderAppMetricsHelper.getMetrics(context, spec,
+              params);
         }
       } catch (Exception e) {
-        logger.warn("Unable to retrieve ganglia metrics. " + e.getMessage());
+        logger.warn("Unable to retrieve metrics. " + e.getMessage());
       }
     }
 
     if (receivedMetrics != null) {
-      for (GangliaMetric metric : receivedMetrics.values()) {
-        if (reverseNameLookup.containsKey(metric.getMetric_name())) {
-          retVal.put(reverseNameLookup.get(metric.getMetric_name()), metric.getDatapoints());
+      for (Map.Entry<String, Number[][]> metric : receivedMetrics.entrySet()) {
+        if (reverseNameLookup.containsKey(metric.getKey())) {
+          retVal.put(reverseNameLookup.get(metric.getKey()), metric.getValue());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/ambari/blob/a964003c/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/SliderAppMetricsHelper.java
----------------------------------------------------------------------
diff --git a/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/SliderAppMetricsHelper.java b/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/SliderAppMetricsHelper.java
new file mode 100644
index 0000000..ee823a9
--- /dev/null
+++ b/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/SliderAppMetricsHelper.java
@@ -0,0 +1,151 @@
+/**
+ * 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.ambari.view.slider.rest.client;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.URISyntaxException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.ambari.view.SystemException;
+import org.apache.ambari.view.ViewContext;
+import org.apache.ambari.view.slider.TemporalInfo;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.Logger;
+import org.codehaus.jackson.map.AnnotationIntrospector;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.ObjectReader;
+import org.codehaus.jackson.map.annotate.JsonSerialize.Inclusion;
+import org.codehaus.jackson.xc.JaxbAnnotationIntrospector;
+
+public class SliderAppMetricsHelper {
+  private static final Logger logger = Logger
+      .getLogger(SliderAppMetricsHelper.class);
+  private static ObjectMapper mapper;
+  private final static ObjectReader timelineObjectReader;
+
+  static {
+    mapper = new ObjectMapper();
+    AnnotationIntrospector introspector = new JaxbAnnotationIntrospector();
+    mapper.setAnnotationIntrospector(introspector);
+    // no inspection deprecation
+    mapper.getSerializationConfig().setSerializationInclusion(
+        Inclusion.NON_NULL);
+    timelineObjectReader = mapper.reader(TimelineMetrics.class);
+  }
+
+  public static Map<String, Number[][]> getMetrics(ViewContext context,
+      String spec, String params) throws IOException {
+    Map<String, Number[][]> receivedMetrics = new HashMap<String, Number[][]>();
+    Map<String, String> headers = new HashMap<String, String>();
+
+    BufferedReader reader = null;
+    try {
+      String fullUrl = spec + "?" + params;
+      logger.debug("Metrics request url = " + fullUrl);
+      reader = new BufferedReader(new InputStreamReader(context
+          .getURLStreamProvider().readFrom(fullUrl, "GET", null, headers)));
+
+      TimelineMetrics timelineMetrics = timelineObjectReader.readValue(reader);
+      logger.debug("Timeline metrics response => " + timelineMetrics);
+
+      for (TimelineMetric tlMetric : timelineMetrics.getMetrics()) {
+        if (tlMetric.getMetricName() != null
+            && tlMetric.getMetricValues() != null) {
+          Map<Long, Double> tlMetricValues = tlMetric.getMetricValues();
+          Number[][] metricValues = transformMetricValues(tlMetricValues);
+          receivedMetrics.put(tlMetric.getMetricName(), metricValues);
+        }
+      }
+
+    } catch (IOException io) {
+      logger.warn("Error getting timeline metrics.", io);
+    } finally {
+      if (reader != null) {
+        try {
+          reader.close();
+        } catch (IOException e) {
+          if (logger.isDebugEnabled()) {
+            logger.warn("Unable to close http input steam : spec=" + spec, e);
+          }
+        }
+      }
+    }
+
+    return receivedMetrics;
+  }
+
+  private static Number[][] transformMetricValues(
+      Map<Long, Double> tlMetricValues) {
+    Number[][] metricValues = new Number[tlMetricValues.size()][2];
+    int i = 0;
+    for (Map.Entry<Long, Double> tlMetricValue : tlMetricValues.entrySet()) {
+      // value goes to column 0
+      metricValues[i][0] = tlMetricValue.getValue();
+      // timestamp goes to column 1 - convert it from millis to sec
+      metricValues[i][1] = tlMetricValue.getKey() / 1000;
+      i++;
+    }
+    return metricValues;
+  }
+
+  public static String getUrlWithParams(String metricUrl,
+      Set<String> metricSet, TemporalInfo temporalInfo) throws SystemException,
+      URISyntaxException {
+    String metrics = getSetString(metricSet, -1);
+    URIBuilder uriBuilder = new URIBuilder(metricUrl);
+
+    if (metrics.length() > 0) {
+      uriBuilder.setParameter("metricNames", metrics);
+    }
+
+    if (temporalInfo != null) {
+      long startTime = temporalInfo.getStartTime();
+      if (startTime != -1) {
+        uriBuilder.setParameter("startTime", String.valueOf(startTime));
+      }
+      long endTime = temporalInfo.getEndTime();
+      if (endTime != -1) {
+        uriBuilder.setParameter("endTime", String.valueOf(endTime));
+      }
+    } else {
+      long endTime = System.currentTimeMillis() / 1000;
+      long startTime = System.currentTimeMillis() / 1000 - 60 * 60;
+      uriBuilder.setParameter("endTime", String.valueOf(endTime));
+      uriBuilder.setParameter("startTime", String.valueOf(startTime));
+    }
+    return uriBuilder.toString();
+  }
+
+  private static String getSetString(Set<String> set, int limit) {
+    StringBuilder sb = new StringBuilder();
+    if (limit == -1 || set.size() <= limit) {
+      for (String cluster : set) {
+        if (sb.length() > 0) {
+          sb.append(",");
+        }
+        sb.append(cluster);
+      }
+    }
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/a964003c/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/TimelineMetric.java
----------------------------------------------------------------------
diff --git a/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/TimelineMetric.java b/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/TimelineMetric.java
new file mode 100644
index 0000000..e5e1e8f
--- /dev/null
+++ b/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/TimelineMetric.java
@@ -0,0 +1,172 @@
+/**
+ * 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.ambari.view.slider.rest.client;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.Map;
+import java.util.TreeMap;
+
+@XmlRootElement(name = "metric")
+@XmlAccessorType(XmlAccessType.NONE)
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class TimelineMetric implements Comparable<TimelineMetric> {
+
+  private String metricName;
+  private String appId;
+  private String instanceId;
+  private String hostName;
+  private long timestamp;
+  private long startTime;
+  private String type;
+  private Map<Long, Double> metricValues = new TreeMap<Long, Double>();
+
+  @XmlElement(name = "metricname")
+  public String getMetricName() {
+    return metricName;
+  }
+
+  public void setMetricName(String metricName) {
+    this.metricName = metricName;
+  }
+
+  @XmlElement(name = "appid")
+  public String getAppId() {
+    return appId;
+  }
+
+  public void setAppId(String appId) {
+    this.appId = appId;
+  }
+
+  @XmlElement(name = "instanceid")
+  public String getInstanceId() {
+    return instanceId;
+  }
+
+  public void setInstanceId(String instanceId) {
+    this.instanceId = instanceId;
+  }
+
+  @XmlElement(name = "hostname")
+  public String getHostName() {
+    return hostName;
+  }
+
+  public void setHostName(String hostName) {
+    this.hostName = hostName;
+  }
+
+  @XmlElement(name = "timestamp")
+  public long getTimestamp() {
+    return timestamp;
+  }
+
+  public void setTimestamp(long timestamp) {
+    this.timestamp = timestamp;
+  }
+
+  @XmlElement(name = "starttime")
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public void setStartTime(long startTime) {
+    this.startTime = startTime;
+  }
+
+  @XmlElement(name = "type")
+  public String getType() {
+    return type;
+  }
+
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  @XmlElement(name = "metrics")
+  public Map<Long, Double> getMetricValues() {
+    return metricValues;
+  }
+
+  public void setMetricValues(Map<Long, Double> metricValues) {
+    this.metricValues = metricValues;
+  }
+
+  public void addMetricValues(Map<Long, Double> metricValues) {
+    this.metricValues.putAll(metricValues);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    TimelineMetric metric = (TimelineMetric) o;
+
+    if (!metricName.equals(metric.metricName)) return false;
+    if (hostName != null ? !hostName.equals(metric.hostName) : metric.hostName != null)
+      return false;
+    if (appId != null ? !appId.equals(metric.appId) : metric.appId != null)
+      return false;
+    if (instanceId != null ? !instanceId.equals(metric.instanceId) : metric.instanceId != null)
+      return false;
+    if (timestamp != metric.timestamp) return false;
+    if (startTime != metric.startTime) return false;
+
+    return true;
+  }
+
+  public boolean equalsExceptTime(TimelineMetric metric) {
+    if (!metricName.equals(metric.metricName)) return false;
+    if (hostName != null ? !hostName.equals(metric.hostName) : metric.hostName != null)
+      return false;
+    if (appId != null ? !appId.equals(metric.appId) : metric.appId != null)
+      return false;
+    if (instanceId != null ? !instanceId.equals(metric.instanceId) : metric.instanceId != null)
+      return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = metricName.hashCode();
+    result = 31 * result + (appId != null ? appId.hashCode() : 0);
+    result = 31 * result + (instanceId != null ? instanceId.hashCode() : 0);
+    result = 31 * result + (hostName != null ? hostName.hashCode() : 0);
+    result = 31 * result + (int) (timestamp ^ (timestamp >>> 32));
+    return result;
+  }
+
+  @Override
+  public int compareTo(TimelineMetric other) {
+    if (timestamp > other.timestamp) {
+      return -1;
+    } else if (timestamp < other.timestamp) {
+      return 1;
+    } else {
+      return metricName.compareTo(other.metricName);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/a964003c/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/TimelineMetrics.java
----------------------------------------------------------------------
diff --git a/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/TimelineMetrics.java b/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/TimelineMetrics.java
new file mode 100644
index 0000000..bb125af
--- /dev/null
+++ b/contrib/views/slider/src/main/java/org/apache/ambari/view/slider/rest/client/TimelineMetrics.java
@@ -0,0 +1,101 @@
+/**
+ * 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.ambari.view.slider.rest.client;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * The class that hosts a list of timeline entities.
+ */
+@XmlRootElement(name = "metrics")
+@XmlAccessorType(XmlAccessType.NONE)
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class TimelineMetrics {
+
+  private List<TimelineMetric> allMetrics = new ArrayList<TimelineMetric>();
+
+  public TimelineMetrics() {}
+
+  @XmlElement(name = "metrics")
+  public List<TimelineMetric> getMetrics() {
+    return allMetrics;
+  }
+
+  public void setMetrics(List<TimelineMetric> allMetrics) {
+    this.allMetrics = allMetrics;
+  }
+
+  private boolean isEqualTimelineMetrics(TimelineMetric metric1,
+                                         TimelineMetric metric2) {
+
+    boolean isEqual = true;
+
+    if (!metric1.getMetricName().equals(metric2.getMetricName())) {
+      return false;
+    }
+
+    if (metric1.getHostName() != null) {
+      isEqual = metric1.getHostName().equals(metric2.getHostName());
+    }
+
+    if (metric1.getAppId() != null) {
+      isEqual = metric1.getAppId().equals(metric2.getAppId());
+    }
+
+    return isEqual;
+  }
+
+  /**
+   * Merge with existing TimelineMetric if everything except startTime is
+   * the same.
+   * @param metric {@link TimelineMetric}
+   */
+  public void addOrMergeTimelineMetric(TimelineMetric metric) {
+    TimelineMetric metricToMerge = null;
+
+    if (!allMetrics.isEmpty()) {
+      for (TimelineMetric timelineMetric : allMetrics) {
+        if (timelineMetric.equalsExceptTime(metric)) {
+          metricToMerge = timelineMetric;
+          break;
+        }
+      }
+    }
+
+    if (metricToMerge != null) {
+      metricToMerge.addMetricValues(metric.getMetricValues());
+      if (metricToMerge.getTimestamp() > metric.getTimestamp()) {
+        metricToMerge.setTimestamp(metric.getTimestamp());
+      }
+      if (metricToMerge.getStartTime() > metric.getStartTime()) {
+        metricToMerge.setStartTime(metric.getStartTime());
+      }
+    } else {
+      allMetrics.add(metric);
+    }
+  }
+}