summaryrefslogtreecommitdiff
path: root/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/HBaseTimelineMetricStore.java
blob: 5ee8b444801aab13ad068d52d6b655fffaa00b34 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
/**
 * 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.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.metrics2.sink.timeline.Precision;
import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
import org.apache.hadoop.metrics2.sink.timeline.TimelineMetricMetadata;
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.server.applicationhistoryservice.metrics.timeline.aggregators.Function;
import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricAggregator;
import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricAggregatorFactory;
import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.discovery.TimelineMetricMetadataKey;
import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.discovery.TimelineMetricMetadataManager;
import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.Condition;
import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.DefaultCondition;

import java.io.IOException;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;

import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.USE_GROUPBY_AGGREGATOR_QUERIES;

public class HBaseTimelineMetricStore extends AbstractService implements TimelineMetricStore {

  static final Log LOG = LogFactory.getLog(HBaseTimelineMetricStore.class);
  private final TimelineMetricConfiguration configuration;
  private PhoenixHBaseAccessor hBaseAccessor;
  private static volatile boolean isInitialized = false;
  private final ScheduledExecutorService executorService = Executors.newSingleThreadScheduledExecutor();
  private TimelineMetricMetadataManager metricMetadataManager;

  /**
   * Construct the service.
   *
   */
  public HBaseTimelineMetricStore(TimelineMetricConfiguration configuration) {
    super(HBaseTimelineMetricStore.class.getName());
    this.configuration = configuration;
  }

  @Override
  protected void serviceInit(Configuration conf) throws Exception {
    super.serviceInit(conf);
    initializeSubsystem(configuration.getHbaseConf(), configuration.getMetricsConf());
  }

  private synchronized void initializeSubsystem(Configuration hbaseConf,
                                                Configuration metricsConf) {
    if (!isInitialized) {
      hBaseAccessor = new PhoenixHBaseAccessor(hbaseConf, metricsConf);
      hBaseAccessor.initMetricSchema();
      // Initialize metadata from store
      metricMetadataManager = new TimelineMetricMetadataManager(hBaseAccessor, metricsConf);
      metricMetadataManager.initializeMetadata();

      if (Boolean.parseBoolean(metricsConf.get(USE_GROUPBY_AGGREGATOR_QUERIES, "true"))) {
        LOG.info("Using group by aggregators for aggregating host and cluster metrics.");
      }

      // Start the cluster aggregator second
      TimelineMetricAggregator secondClusterAggregator =
        TimelineMetricAggregatorFactory.createTimelineClusterAggregatorSecond(hBaseAccessor, metricsConf, metricMetadataManager);
      if (!secondClusterAggregator.isDisabled()) {
        Thread aggregatorThread = new Thread(secondClusterAggregator);
        aggregatorThread.start();
      }

      // Start the minute cluster aggregator
      TimelineMetricAggregator minuteClusterAggregator =
        TimelineMetricAggregatorFactory.createTimelineClusterAggregatorMinute(hBaseAccessor, metricsConf);
      if (!minuteClusterAggregator.isDisabled()) {
        Thread aggregatorThread = new Thread(minuteClusterAggregator);
        aggregatorThread.start();
      }

      // Start the hourly cluster aggregator
      TimelineMetricAggregator hourlyClusterAggregator =
        TimelineMetricAggregatorFactory.createTimelineClusterAggregatorHourly(hBaseAccessor, metricsConf);
      if (!hourlyClusterAggregator.isDisabled()) {
        Thread aggregatorThread = new Thread(hourlyClusterAggregator);
        aggregatorThread.start();
      }

      // Start the daily cluster aggregator
      TimelineMetricAggregator dailyClusterAggregator =
        TimelineMetricAggregatorFactory.createTimelineClusterAggregatorDaily(hBaseAccessor, metricsConf);
      if (!dailyClusterAggregator.isDisabled()) {
        Thread aggregatorThread = new Thread(dailyClusterAggregator);
        aggregatorThread.start();
      }

      // Start the minute host aggregator
      TimelineMetricAggregator minuteHostAggregator =
        TimelineMetricAggregatorFactory.createTimelineMetricAggregatorMinute(hBaseAccessor, metricsConf);
      if (!minuteHostAggregator.isDisabled()) {
        Thread minuteAggregatorThread = new Thread(minuteHostAggregator);
        minuteAggregatorThread.start();
      }

      // Start the hourly host aggregator
      TimelineMetricAggregator hourlyHostAggregator =
        TimelineMetricAggregatorFactory.createTimelineMetricAggregatorHourly(hBaseAccessor, metricsConf);
      if (!hourlyHostAggregator.isDisabled()) {
        Thread aggregatorHourlyThread = new Thread(hourlyHostAggregator);
        aggregatorHourlyThread.start();
      }

      // Start the daily host aggregator
      TimelineMetricAggregator dailyHostAggregator =
        TimelineMetricAggregatorFactory.createTimelineMetricAggregatorDaily(hBaseAccessor, metricsConf);
      if (!dailyHostAggregator.isDisabled()) {
        Thread aggregatorDailyThread = new Thread(dailyHostAggregator);
        aggregatorDailyThread.start();
      }

      int initDelay = configuration.getTimelineMetricsServiceWatcherInitDelay();
      int delay = configuration.getTimelineMetricsServiceWatcherDelay();
      // Start the watchdog
      executorService.scheduleWithFixedDelay(
        new TimelineMetricStoreWatcher(this, configuration), initDelay, delay,
        TimeUnit.SECONDS);
      LOG.info("Started watchdog for timeline metrics store with initial " +
        "delay = " + initDelay + ", delay = " + delay);

      isInitialized = true;
    }

  }

  @Override
  protected void serviceStop() throws Exception {
    super.serviceStop();
  }

  @Override
  public TimelineMetrics getTimelineMetrics(List<String> metricNames,
      List<String> hostnames, String applicationId, String instanceId,
      Long startTime, Long endTime, Precision precision, Integer limit,
      boolean groupedByHosts) throws SQLException, IOException {

    if (metricNames == null || metricNames.isEmpty()) {
      throw new IllegalArgumentException("No metric name filter specified.");
    }
    if ((startTime == null && endTime != null)
        || (startTime != null && endTime == null)) {
      throw new IllegalArgumentException("Open ended query not supported ");
    }
    if (limit != null && limit > PhoenixHBaseAccessor.RESULTSET_LIMIT){
      throw new IllegalArgumentException("Limit too big");
    }
    Map<String, List<Function>> metricFunctions =
      parseMetricNamesToAggregationFunctions(metricNames);

    Condition condition = new DefaultCondition(
      new ArrayList<String>(metricFunctions.keySet()),
      hostnames, applicationId, instanceId, startTime, endTime,
      precision, limit, groupedByHosts);

    TimelineMetrics metrics;

    if (hostnames == null || hostnames.isEmpty()) {
      metrics = hBaseAccessor.getAggregateMetricRecords(condition, metricFunctions);
    } else {
      metrics = hBaseAccessor.getMetricRecords(condition, metricFunctions);
    }
    return postProcessMetrics(metrics);
  }

  private TimelineMetrics postProcessMetrics(TimelineMetrics metrics) {
    List<TimelineMetric> metricsList = metrics.getMetrics();

    for (TimelineMetric metric : metricsList){
      String name = metric.getMetricName();
      if (name.contains("._rate")){
        updateValuesAsRate(metric.getMetricValues());
      }
    }

    return metrics;
  }

  static Map<Long, Double> updateValuesAsRate(Map<Long, Double> metricValues) {
    Long prevTime = null;
    Double prevVal = null;
    long step;
    Double diff;

    for (Map.Entry<Long, Double> timeValueEntry : metricValues.entrySet()) {
      Long currTime = timeValueEntry.getKey();
      Double currVal = timeValueEntry.getValue();

      if (prevTime != null) {
        step = currTime - prevTime;
        diff = currVal - prevVal;
        Double rate = diff / TimeUnit.MILLISECONDS.toSeconds(step);
        timeValueEntry.setValue(rate);
      } else {
        timeValueEntry.setValue(0.0);
      }

      prevTime = currTime;
      prevVal = currVal;
    }

    return metricValues;
  }

  static HashMap<String, List<Function>> parseMetricNamesToAggregationFunctions(List<String> metricNames) {
    HashMap<String, List<Function>> metricsFunctions = new HashMap<>();

    for (String metricName : metricNames){
      Function function = Function.DEFAULT_VALUE_FUNCTION;
      String cleanMetricName = metricName;

      try {
        function = Function.fromMetricName(metricName);
        int functionStartIndex = metricName.indexOf("._");
        if (functionStartIndex > 0) {
          cleanMetricName = metricName.substring(0, functionStartIndex);
        }
      } catch (Function.FunctionFormatException ffe){
        // unknown function so
        // fallback to VALUE, and fullMetricName
      }

      List<Function> functionsList = metricsFunctions.get(cleanMetricName);
      if (functionsList == null) {
        functionsList = new ArrayList<>(1);
      }
      functionsList.add(function);
      metricsFunctions.put(cleanMetricName, functionsList);
    }

    return metricsFunctions;
  }

  @Override
  public TimelineMetric getTimelineMetric(String metricName, List<String> hostnames,
      String applicationId, String instanceId, Long startTime,
      Long endTime, Precision precision, Integer limit)
      throws SQLException, IOException {

    if (metricName == null || metricName.isEmpty()) {
      throw new IllegalArgumentException("No metric name filter specified.");
    }
    if ((startTime == null && endTime != null)
        || (startTime != null && endTime == null)) {
      throw new IllegalArgumentException("Open ended query not supported ");
    }
    if (limit !=null && limit > PhoenixHBaseAccessor.RESULTSET_LIMIT){
      throw new IllegalArgumentException("Limit too big");
    }

    Map<String, List<Function>> metricFunctions =
      parseMetricNamesToAggregationFunctions(Collections.singletonList(metricName));

    Condition condition = new DefaultCondition(
      new ArrayList<String>(metricFunctions.keySet()), hostnames, applicationId,
      instanceId, startTime, endTime, precision, limit, true);
    TimelineMetrics metrics = hBaseAccessor.getMetricRecords(condition,
      metricFunctions);

    metrics = postProcessMetrics(metrics);

    TimelineMetric metric = new TimelineMetric();
    List<TimelineMetric> metricList = metrics.getMetrics();

    if (metricList != null && !metricList.isEmpty()) {
      metric.setMetricName(metricList.get(0).getMetricName());
      metric.setAppId(metricList.get(0).getAppId());
      metric.setInstanceId(metricList.get(0).getInstanceId());
      metric.setHostName(metricList.get(0).getHostName());
      // Assumption that metrics are ordered by start time
      metric.setStartTime(metricList.get(0).getStartTime());
      TreeMap<Long, Double> metricRecords = new TreeMap<Long, Double>();
      for (TimelineMetric timelineMetric : metricList) {
        metricRecords.putAll(timelineMetric.getMetricValues());
      }
      metric.setMetricValues(metricRecords);
    }

    return metric;
  }

  @Override
  public TimelinePutResponse putMetrics(TimelineMetrics metrics) throws SQLException, IOException {
    // Error indicated by the Sql exception
    TimelinePutResponse response = new TimelinePutResponse();

    hBaseAccessor.insertMetricRecordsWithMetadata(metricMetadataManager, metrics);

    return response;
  }

  @Override
  public Map<String, List<TimelineMetricMetadata>> getTimelineMetricMetadata() throws SQLException, IOException {
    Map<TimelineMetricMetadataKey, TimelineMetricMetadata> metadata =
      metricMetadataManager.getMetadataCache();

    // Group Metadata by AppId
    Map<String, List<TimelineMetricMetadata>> metadataByAppId = new HashMap<>();
    for (TimelineMetricMetadata metricMetadata : metadata.values()) {
      List<TimelineMetricMetadata> metadataList = metadataByAppId.get(metricMetadata.getAppId());
      if (metadataList == null) {
        metadataList = new ArrayList<>();
        metadataByAppId.put(metricMetadata.getAppId(), metadataList);
      }

      metadataList.add(metricMetadata);
    }

    return metadataByAppId;
  }

  @Override
  public Map<String, Set<String>> getHostAppsMetadata() throws SQLException, IOException {
    return metricMetadataManager.getHostedAppsCache();
  }
}