summaryrefslogtreecommitdiff
path: root/ambari-metrics/ambari-metrics-timelineservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/metrics/timeline/PhoenixHBaseAccessor.java
blob: 7fa1d2dc16897cf3e10b898b24a5bdf74a2616c8 (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
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
/**
 * 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.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.RetryCounterFactory;
import org.apache.hadoop.metrics2.sink.timeline.Precision;
import org.apache.hadoop.metrics2.sink.timeline.SingleValuedTimelineMetric;
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.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.AggregatorUtils;
import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.Function;
import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.MetricClusterAggregate;
import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.MetricHostAggregate;
import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineClusterMetric;
import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.aggregators.TimelineMetricReadHelper;
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.ConnectionProvider;
import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.DefaultPhoenixDataSource;
import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL;
import org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.SplitByMetricNamesCondition;
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
import org.apache.phoenix.exception.PhoenixIOException;
import org.apache.phoenix.exception.SQLExceptionCode;
import org.codehaus.jackson.map.ObjectMapper;
import org.codehaus.jackson.type.TypeReference;

import java.io.IOException;
import java.sql.Connection;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.TimeUnit;

import static java.util.concurrent.TimeUnit.SECONDS;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.AGGREGATE_TABLE_SPLIT_POINTS;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.AGGREGATORS_SKIP_BLOCK_CACHE;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.CLUSTER_DAILY_TABLE_TTL;
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.CLUSTER_SECOND_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_DAILY_TABLE_TTL;
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.OUT_OFF_BAND_DATA_TIME_ALLOWANCE;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.PRECISION_TABLE_SPLIT_POINTS;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.TimelineMetricConfiguration.PRECISION_TABLE_TTL;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.ALTER_SQL;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.CREATE_HOSTED_APPS_METADATA_TABLE_SQL;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.CREATE_METRICS_AGGREGATE_TABLE_SQL;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.CREATE_METRICS_CLUSTER_AGGREGATE_GROUPED_TABLE_SQL;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.CREATE_METRICS_CLUSTER_AGGREGATE_TABLE_SQL;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.CREATE_METRICS_METADATA_TABLE_SQL;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.CREATE_METRICS_TABLE_SQL;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.DEFAULT_ENCODING;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.DEFAULT_TABLE_COMPRESSION;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.GET_HOSTED_APPS_METADATA_SQL;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.GET_METRIC_METADATA_SQL;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_DAILY_TABLE_NAME;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_HOURLY_TABLE_NAME;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_AGGREGATE_MINUTE_TABLE_NAME;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_DAILY_TABLE_NAME;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_MINUTE_TABLE_NAME;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_CLUSTER_AGGREGATE_TABLE_NAME;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.METRICS_RECORD_TABLE_NAME;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.UPSERT_AGGREGATE_RECORD_SQL;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.UPSERT_CLUSTER_AGGREGATE_SQL;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.UPSERT_CLUSTER_AGGREGATE_TIME_SQL;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.UPSERT_HOSTED_APPS_METADATA_SQL;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.UPSERT_METADATA_SQL;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.metrics.timeline.query.PhoenixTransactSQL.UPSERT_METRICS_SQL;

/**
 * Provides a facade over the Phoenix API to access HBase schema
 */
public class PhoenixHBaseAccessor {
  private static final Log LOG = LogFactory.getLog(PhoenixHBaseAccessor.class);

  static final int PHOENIX_MAX_MUTATION_STATE_SIZE = 50000;
  // Default stale data allowance set to 3 minutes, 2 minutes more than time
  // it was collected. Also 2 minutes is the default aggregation interval at
  // cluster and host levels.
  static final long DEFAULT_OUT_OF_BAND_TIME_ALLOWANCE = 300000;
  /**
   * 22 metrics for 2hours in SECONDS (10 second data)
   * => Reasonable upper bound on the limit such that our Precision calculation for a given time range makes sense.
   */
  private static final int METRICS_PER_MINUTE = 22;
  private static final int POINTS_PER_MINUTE = 6;
  public static int RESULTSET_LIMIT = (int)TimeUnit.HOURS.toMinutes(2) * METRICS_PER_MINUTE * POINTS_PER_MINUTE ;

  private static final TimelineMetricReadHelper TIMELINE_METRIC_READ_HELPER = new TimelineMetricReadHelper();
  private static ObjectMapper mapper = new ObjectMapper();
  private static TypeReference<TreeMap<Long, Double>> metricValuesTypeRef = new TypeReference<TreeMap<Long, Double>>() {};

  private final Configuration hbaseConf;
  private final Configuration metricsConf;
  private final RetryCounterFactory retryCounterFactory;
  private final ConnectionProvider dataSource;
  private final long outOfBandTimeAllowance;
  private final boolean skipBlockCacheForAggregatorsEnabled;

  public PhoenixHBaseAccessor(Configuration hbaseConf,
                              Configuration metricsConf){
    this(hbaseConf, metricsConf, new DefaultPhoenixDataSource(hbaseConf));
  }

  PhoenixHBaseAccessor(Configuration hbaseConf,
                              Configuration metricsConf,
                              ConnectionProvider dataSource) {
    this.hbaseConf = hbaseConf;
    this.metricsConf = metricsConf;
    RESULTSET_LIMIT = metricsConf.getInt(GLOBAL_RESULT_LIMIT, RESULTSET_LIMIT);
    try {
      Class.forName("org.apache.phoenix.jdbc.PhoenixDriver");
    } catch (ClassNotFoundException e) {
      LOG.error("Phoenix client jar not found in the classpath.", e);
      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)));
    this.outOfBandTimeAllowance = metricsConf.getLong(OUT_OFF_BAND_DATA_TIME_ALLOWANCE,
      DEFAULT_OUT_OF_BAND_TIME_ALLOWANCE);
    this.skipBlockCacheForAggregatorsEnabled = metricsConf.getBoolean(AGGREGATORS_SKIP_BLOCK_CACHE, false);
  }

  private static TimelineMetric getLastTimelineMetricFromResultSet(ResultSet rs)
      throws SQLException, IOException {
    TimelineMetric metric = TIMELINE_METRIC_READ_HELPER.getTimelineMetricCommonsFromResultSet(rs);
    metric.setMetricValues(readLastMetricValueFromJSON(rs.getString("METRICS")));
    return metric;
  }

  private static TreeMap<Long, Double> readLastMetricValueFromJSON(String json)
      throws IOException {
    TreeMap<Long, Double> values = readMetricFromJSON(json);
    Long lastTimeStamp = values.lastKey();

    TreeMap<Long, Double> valueMap = new TreeMap<Long, Double>();
    valueMap.put(lastTimeStamp, values.get(lastTimeStamp));
    return valueMap;
  }

  @SuppressWarnings("unchecked")
  public static TreeMap<Long, Double>  readMetricFromJSON(String json) throws IOException {
    return mapper.readValue(json, metricValuesTypeRef);
  }

  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
   * the Configuration object.
   * Phoenix already caches the HConnection between the client and HBase
   * cluster.
   *
   * @return @java.sql.Connection
   */
  public Connection getConnection() throws SQLException {
    return dataSource.getConnection();
  }

  protected void initMetricSchema() {
    Connection conn = null;
    Statement stmt = null;

    String encoding = metricsConf.get(HBASE_ENCODING_SCHEME, DEFAULT_ENCODING);
    String compression = metricsConf.get(HBASE_COMPRESSION_SCHEME, DEFAULT_TABLE_COMPRESSION);
    String precisionTtl = metricsConf.get(PRECISION_TABLE_TTL, "86400");           //1 day
    String hostMinTtl = metricsConf.get(HOST_MINUTE_TABLE_TTL, "604800");          //7 days
    String hostHourTtl = metricsConf.get(HOST_HOUR_TABLE_TTL, "2592000");          //30 days
    String hostDailyTtl = metricsConf.get(HOST_DAILY_TABLE_TTL, "31536000");       //1 year
    String clusterSecTtl = metricsConf.get(CLUSTER_SECOND_TABLE_TTL, "2592000");     //7 days
    String clusterMinTtl = metricsConf.get(CLUSTER_MINUTE_TABLE_TTL, "7776000");   //30 days
    String clusterHourTtl = metricsConf.get(CLUSTER_HOUR_TABLE_TTL, "31536000");   //1 year
    String clusterDailyTtl = metricsConf.get(CLUSTER_DAILY_TABLE_TTL, "63072000"); //2 years

    try {
      LOG.info("Initializing metrics schema...");
      conn = getConnectionRetryingOnException();
      stmt = conn.createStatement();

      // Metadata
      String metadataSql = String.format(CREATE_METRICS_METADATA_TABLE_SQL,
        encoding, compression);
      stmt.executeUpdate(metadataSql);
      String hostedAppSql = String.format(CREATE_HOSTED_APPS_METADATA_TABLE_SQL,
        encoding, compression);
      stmt.executeUpdate(hostedAppSql);

      // Host level
      String precisionSql = String.format(CREATE_METRICS_TABLE_SQL,
        encoding, precisionTtl, compression);
      String splitPoints = metricsConf.get(PRECISION_TABLE_SPLIT_POINTS);
      if (!StringUtils.isEmpty(splitPoints)) {
        precisionSql += getSplitPointsStr(splitPoints);
      }
      stmt.executeUpdate(precisionSql);
      stmt.executeUpdate(String.format(CREATE_METRICS_AGGREGATE_TABLE_SQL,
        METRICS_AGGREGATE_MINUTE_TABLE_NAME, encoding, hostMinTtl, compression));
      stmt.executeUpdate(String.format(CREATE_METRICS_AGGREGATE_TABLE_SQL,
        METRICS_AGGREGATE_HOURLY_TABLE_NAME, encoding, hostHourTtl, compression));
      stmt.executeUpdate(String.format(CREATE_METRICS_AGGREGATE_TABLE_SQL,
        METRICS_AGGREGATE_DAILY_TABLE_NAME, encoding, hostDailyTtl, compression));

      // Cluster level
      String aggregateSql = String.format(CREATE_METRICS_CLUSTER_AGGREGATE_TABLE_SQL,
        METRICS_CLUSTER_AGGREGATE_TABLE_NAME, encoding, clusterMinTtl, compression);
      splitPoints = metricsConf.get(AGGREGATE_TABLE_SPLIT_POINTS);
      if (!StringUtils.isEmpty(splitPoints)) {
        aggregateSql += getSplitPointsStr(splitPoints);
      }
      stmt.executeUpdate(aggregateSql);
      stmt.executeUpdate(String.format(CREATE_METRICS_CLUSTER_AGGREGATE_GROUPED_TABLE_SQL,
        METRICS_CLUSTER_AGGREGATE_MINUTE_TABLE_NAME, encoding, clusterHourTtl, compression));
      stmt.executeUpdate(String.format(CREATE_METRICS_CLUSTER_AGGREGATE_GROUPED_TABLE_SQL,
        METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME, encoding, clusterHourTtl, compression));
      stmt.executeUpdate(String.format(CREATE_METRICS_CLUSTER_AGGREGATE_GROUPED_TABLE_SQL,
        METRICS_CLUSTER_AGGREGATE_DAILY_TABLE_NAME, encoding, clusterDailyTtl, compression));

      //alter TTL options to update tables
      stmt.executeUpdate(String.format(ALTER_SQL,
        METRICS_RECORD_TABLE_NAME,
        precisionTtl));
      stmt.executeUpdate(String.format(ALTER_SQL,
        METRICS_AGGREGATE_MINUTE_TABLE_NAME,
        hostMinTtl));
      stmt.executeUpdate(String.format(ALTER_SQL,
        METRICS_AGGREGATE_HOURLY_TABLE_NAME,
        hostHourTtl));
      stmt.executeUpdate(String.format(ALTER_SQL,
        METRICS_AGGREGATE_DAILY_TABLE_NAME,
        hostDailyTtl));
      stmt.executeUpdate(String.format(ALTER_SQL,
        METRICS_CLUSTER_AGGREGATE_TABLE_NAME,
        clusterSecTtl));
      stmt.executeUpdate(String.format(ALTER_SQL,
        METRICS_CLUSTER_AGGREGATE_MINUTE_TABLE_NAME,
        clusterMinTtl));
      stmt.executeUpdate(String.format(ALTER_SQL,
        METRICS_CLUSTER_AGGREGATE_HOURLY_TABLE_NAME,
        clusterHourTtl));
      stmt.executeUpdate(String.format(ALTER_SQL,
        METRICS_CLUSTER_AGGREGATE_DAILY_TABLE_NAME,
        clusterDailyTtl));

      conn.commit();

      LOG.info("Metrics schema initialized.");
    } catch (SQLException sql) {
      if (sql.getErrorCode() ==
        SQLExceptionCode.SET_UNSUPPORTED_PROP_ON_ALTER_TABLE.getErrorCode()) {
        LOG.warn("Cannot update TTL on tables. " + sql.getMessage());
      } else {
        LOG.error("Error creating Metrics Schema in HBase using Phoenix.", sql);
        throw new MetricsSystemInitializationException(
          "Error creating Metrics Schema in HBase using Phoenix.", sql);
      }
    } catch (InterruptedException e) {
      LOG.error("Error creating Metrics Schema in HBase using Phoenix.", e);
      throw new MetricsSystemInitializationException(
        "Error creating Metrics Schema in HBase using Phoenix.", e);
    } finally {
      if (stmt != null) {
        try {
          stmt.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (conn != null) {
        try {
          conn.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
    }
  }

  protected String getSplitPointsStr(String splitPoints) {
    if (StringUtils.isEmpty(splitPoints.trim())) {
      return "";
    }
    String[] points = splitPoints.split(",");
    if (points.length > 0) {
      StringBuilder sb = new StringBuilder(" SPLIT ON ");
      sb.append("(");
      for (String point : points) {
        sb.append("'");
        sb.append(point.trim());
        sb.append("'");
        sb.append(",");
      }
      sb.deleteCharAt(sb.length() - 1);
      sb.append(")");
      return sb.toString();
    }
    return "";
  }

  public void insertMetricRecordsWithMetadata(TimelineMetricMetadataManager metadataManager,
                                              TimelineMetrics metrics) throws SQLException, IOException {
    List<TimelineMetric> timelineMetrics = metrics.getMetrics();
    if (timelineMetrics == null || timelineMetrics.isEmpty()) {
      LOG.debug("Empty metrics insert request.");
      return;
    }

    Connection conn = getConnection();
    PreparedStatement metricRecordStmt = null;
    long currentTime = System.currentTimeMillis();

    try {
      metricRecordStmt = conn.prepareStatement(String.format(
        UPSERT_METRICS_SQL, METRICS_RECORD_TABLE_NAME));

      for (TimelineMetric metric : timelineMetrics) {
        if (Math.abs(currentTime - metric.getStartTime()) > outOfBandTimeAllowance) {
          // If timeseries start time is way in the past : discard
          LOG.debug("Discarding out of band timeseries, currentTime = "
            + currentTime + ", startTime = " + metric.getStartTime()
            + ", hostname = " + metric.getHostName());
          continue;
        }

        metricRecordStmt.clearParameters();

        if (LOG.isTraceEnabled()) {
          LOG.trace("host: " + metric.getHostName() + ", " +
            "metricName = " + metric.getMetricName() + ", " +
            "values: " + metric.getMetricValues());
        }
        double[] aggregates =  AggregatorUtils.calculateAggregates(
          metric.getMetricValues());

        metricRecordStmt.setString(1, metric.getMetricName());
        metricRecordStmt.setString(2, metric.getHostName());
        metricRecordStmt.setString(3, metric.getAppId());
        metricRecordStmt.setString(4, metric.getInstanceId());
        metricRecordStmt.setLong(5, currentTime);
        metricRecordStmt.setLong(6, metric.getStartTime());
        metricRecordStmt.setString(7, metric.getUnits());
        metricRecordStmt.setDouble(8, aggregates[0]);
        metricRecordStmt.setDouble(9, aggregates[1]);
        metricRecordStmt.setDouble(10, aggregates[2]);
        metricRecordStmt.setLong(11, (long) aggregates[3]);
        String json = TimelineUtils.dumpTimelineRecordtoJSON(metric.getMetricValues());
        metricRecordStmt.setString(12, json);

        try {
          metricRecordStmt.executeUpdate();

          // Write to metadata cache on successful write to store
          metadataManager.putIfModifiedTimelineMetricMetadata(
            metadataManager.getTimelineMetricMetadata(metric));

          metadataManager.putIfModifiedHostedAppsMetadata(
            metric.getHostName(), metric.getAppId());

        } catch (SQLException sql) {
          LOG.error("Failed on insert records to store.", sql);
        }
      }

      // commit() blocked if HBase unavailable
      conn.commit();

    } finally {
      if (metricRecordStmt != null) {
        try {
          metricRecordStmt.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (conn != null) {
        try {
          conn.close();
        } catch (SQLException sql) {
          // Ignore
        }
      }
    }
  }

  public void insertMetricRecords(TimelineMetrics metrics) throws SQLException, IOException {
    insertMetricRecordsWithMetadata(null, metrics);
  }

  @SuppressWarnings("unchecked")
  public TimelineMetrics getMetricRecords(
    final Condition condition, Map<String, List<Function>> metricFunctions)
    throws SQLException, IOException {

    validateConditionIsNotEmpty(condition);

    Connection conn = getConnection();
    PreparedStatement stmt = null;
    ResultSet rs = null;
    TimelineMetrics metrics = new TimelineMetrics();

    try {
      //get latest
      if (condition.isPointInTime()){
        getLatestMetricRecords(condition, conn, metrics);
      } else {
        if (condition.getEndTime() >= condition.getStartTime()) {
          stmt = PhoenixTransactSQL.prepareGetMetricsSqlStmt(conn, condition);
          rs = stmt.executeQuery();
          while (rs.next()) {
            appendMetricFromResultSet(metrics, condition, metricFunctions, rs);
          }
        } else {
          LOG.warn("Skipping metrics query because endTime < startTime");
        }
      }

    } catch (PhoenixIOException pioe) {
      Throwable pioe2 = pioe.getCause();
      // Need to find out if this is exception "Could not find hash cache
      // for joinId" or another PhoenixIOException
      if (pioe2 instanceof PhoenixIOException &&
        pioe2.getCause() instanceof DoNotRetryIOException) {
        String className = null;
        for (StackTraceElement ste : pioe2.getCause().getStackTrace()) {
          className = ste.getClassName();
        }

        if (className != null && className.equals("HashJoinRegionScanner")) {
          LOG.error("The cache might have expired and have been removed. Try to" +
            " increase the cache size by setting bigger value for " +
            "phoenix.coprocessor.maxMetaDataCacheSize in ams-hbase-site config." +
            " Falling back to sort-merge join algorithm.");
          PhoenixTransactSQL.setSortMergeJoinEnabled(true);
        }
      }
      throw pioe;
    } catch (RuntimeException ex) {
      // We need to find out if this is a real IO exception
      // or exception "maxStamp is smaller than minStamp"
      // which is thrown in hbase TimeRange.java
      Throwable io = ex.getCause();
      String className = null;
      if (io != null) {
        for (StackTraceElement ste : io.getStackTrace()) {
          className = ste.getClassName();
        }
      }
      if (className != null && className.equals("TimeRange")) {
        // This is "maxStamp is smaller than minStamp" exception
        // Log error and return empty metrics
        LOG.debug(io);
        return new TimelineMetrics();
      } else {
        throw ex;
      }

    } finally {
      if (rs != null) {
        try {
          rs.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (stmt != null) {
        try {
          stmt.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (conn != null) {
        try {
          conn.close();
        } catch (SQLException sql) {
          // Ignore
        }
      }
    }

    LOG.debug("Metrics records size: " + metrics.getMetrics().size());
    return metrics;
  }

  /**
   * Apply aggregate function to the result if supplied else get precision
   * or aggregate data with default function applied.
   */
  private void appendMetricFromResultSet(TimelineMetrics metrics, Condition condition,
                                         Map<String, List<Function>> metricFunctions,
                                         ResultSet rs) throws SQLException, IOException {
    String metricName = rs.getString("METRIC_NAME");
    List<Function> functions = metricFunctions.get(metricName);

    // Apply aggregation function if present
    if ((functions != null && !functions.isEmpty())) {
      if (functions.size() > 1) {
        throw new IllegalArgumentException("Multiple aggregate functions not supported.");
      }
      for (Function f : functions) {
        if (f.getReadFunction() == Function.ReadFunction.VALUE) {
          getTimelineMetricsFromResultSet(metrics, condition, rs);
        } else {
          SingleValuedTimelineMetric metric =
            TIMELINE_METRIC_READ_HELPER.getAggregatedTimelineMetricFromResultSet(rs, f);

          if (condition.isGrouped()) {
            metrics.addOrMergeTimelineMetric(metric);
          } else {
            metrics.getMetrics().add(metric.getTimelineMetric());
          }
        }
      }
    } else {
      // No aggregation requested
      // Execution never goes here, function always contain at least 1 element
      getTimelineMetricsFromResultSet(metrics, condition, rs);
    }
  }

  private void getTimelineMetricsFromResultSet(TimelineMetrics metrics, Condition condition, ResultSet rs) throws SQLException, IOException {
    if (condition.getPrecision().equals(Precision.SECONDS)) {
      TimelineMetric metric = TIMELINE_METRIC_READ_HELPER.getTimelineMetricFromResultSet(rs);
      if (condition.isGrouped()) {
        metrics.addOrMergeTimelineMetric(metric);
      } else {
        metrics.getMetrics().add(metric);
      }

    } else {
      SingleValuedTimelineMetric metric =
        TIMELINE_METRIC_READ_HELPER.getAggregatedTimelineMetricFromResultSet(rs,
          Function.DEFAULT_VALUE_FUNCTION);
      if (condition.isGrouped()) {
        metrics.addOrMergeTimelineMetric(metric);
      } else {
        metrics.getMetrics().add(metric.getTimelineMetric());
      }
    }
  }

  private void getLatestMetricRecords(Condition condition, Connection conn,
                                      TimelineMetrics metrics) throws SQLException, IOException {

    validateConditionIsNotEmpty(condition);

    PreparedStatement stmt = PhoenixTransactSQL.prepareGetLatestMetricSqlStmt(conn, condition);
    ResultSet rs = null;
    try {
      rs = stmt.executeQuery();
      while (rs.next()) {
        TimelineMetric metric = getLastTimelineMetricFromResultSet(rs);
        metrics.getMetrics().add(metric);
      }
    } finally {
      if (rs != null) {
        try {
          rs.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (stmt != null) {
        stmt.close();
      }
    }
  }

  /**
   * Get metrics aggregated across hosts.
   *
   * @param condition @Condition
   * @return @TimelineMetrics
   * @throws SQLException
   */
  public TimelineMetrics getAggregateMetricRecords(final Condition condition,
      Map<String, List<Function>> metricFunctions) throws SQLException {

    validateConditionIsNotEmpty(condition);

    Connection conn = getConnection();
    PreparedStatement stmt = null;
    ResultSet rs = null;
    TimelineMetrics metrics = new TimelineMetrics();

    try {
      //get latest
      if(condition.isPointInTime()) {
        getLatestAggregateMetricRecords(condition, conn, metrics, metricFunctions);
      } else {
        stmt = PhoenixTransactSQL.prepareGetAggregateSqlStmt(conn, condition);

        rs = stmt.executeQuery();
        while (rs.next()) {
          appendAggregateMetricFromResultSet(metrics, condition, metricFunctions, rs);
        }
      }
    } finally {
      if (rs != null) {
        try {
          rs.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (stmt != null) {
        try {
          stmt.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (conn != null) {
        try {
          conn.close();
        } catch (SQLException sql) {
          // Ignore
        }
      }
    }

    LOG.debug("Aggregate records size: " + metrics.getMetrics().size());
    return metrics;
  }

  private void appendAggregateMetricFromResultSet(TimelineMetrics metrics,
      Condition condition, Map<String, List<Function>> metricFunctions,
      ResultSet rs) throws SQLException {

    String metricName = rs.getString("METRIC_NAME");
    List<Function> functions = metricFunctions.get(metricName);

    for (Function aggregateFunction : functions) {
      SingleValuedTimelineMetric metric;

      if (condition.getPrecision() == Precision.MINUTES
          || condition.getPrecision() == Precision.HOURS
          || condition.getPrecision() == Precision.DAYS) {
        metric = getAggregateTimelineMetricFromResultSet(rs, aggregateFunction, false);
      } else {
        metric = getAggregateTimelineMetricFromResultSet(rs, aggregateFunction, true);
      }

      if (condition.isGrouped()) {
        metrics.addOrMergeTimelineMetric(metric);
      } else {
        metrics.getMetrics().add(metric.getTimelineMetric());
      }
    }
  }

  private void getLatestAggregateMetricRecords(Condition condition,
      Connection conn, TimelineMetrics metrics,
      Map<String, List<Function>> metricFunctions) throws SQLException {

    PreparedStatement stmt = null;
    SplitByMetricNamesCondition splitCondition =
      new SplitByMetricNamesCondition(condition);

    for (String metricName: splitCondition.getOriginalMetricNames()) {

      splitCondition.setCurrentMetric(metricName);
      stmt = PhoenixTransactSQL.prepareGetLatestAggregateMetricSqlStmt(conn, splitCondition);
      ResultSet rs = null;
      try {
        rs = stmt.executeQuery();
        while (rs.next()) {
          List<Function> functions = metricFunctions.get(metricName);
          if (functions != null) {
            for (Function f : functions) {
              SingleValuedTimelineMetric metric =
                getAggregateTimelineMetricFromResultSet(rs, f, true);

              if (condition.isGrouped()) {
                metrics.addOrMergeTimelineMetric(metric);
              } else {
                metrics.getMetrics().add(metric.getTimelineMetric());
              }
            }
          } else {
            SingleValuedTimelineMetric metric =
              getAggregateTimelineMetricFromResultSet(rs, new Function(), true);
            metrics.getMetrics().add(metric.getTimelineMetric());
          }
        }
      } finally {
        if (rs != null) {
          try {
            rs.close();
          } catch (SQLException e) {
            // Ignore
          }
        }
        if (stmt != null) {
          stmt.close();
        }        
      }
    }
  }

  private SingleValuedTimelineMetric getAggregateTimelineMetricFromResultSet(ResultSet rs,
      Function f, boolean useHostCount) throws SQLException {

    String countColumnName = "METRIC_COUNT";
    if (useHostCount) {
      countColumnName = "HOSTS_COUNT";
    }

    SingleValuedTimelineMetric metric = new SingleValuedTimelineMetric(
      rs.getString("METRIC_NAME") + f.getSuffix(),
      rs.getString("APP_ID"),
      rs.getString("INSTANCE_ID"),
      null,
      rs.getLong("SERVER_TIME"),
      rs.getLong("SERVER_TIME"),
      rs.getString("UNITS")
    );

    double value;
    switch(f.getReadFunction()){
      case AVG:
        value = rs.getDouble("METRIC_SUM") / rs.getInt(countColumnName);
        break;
      case MIN:
        value = rs.getDouble("METRIC_MIN");
        break;
      case MAX:
        value = rs.getDouble("METRIC_MAX");
        break;
      case SUM:
        value = rs.getDouble("METRIC_SUM");
        break;
      default:
        value = rs.getDouble("METRIC_SUM") / rs.getInt(countColumnName);
        break;
    }

    metric.setSingleTimeseriesValue(rs.getLong("SERVER_TIME"), value);

    return metric;
  }

  private void validateConditionIsNotEmpty(Condition condition) {
    if (condition.isEmpty()) {
      throw new IllegalArgumentException("No filter criteria specified.");
    }
  }

  public void saveHostAggregateRecords(Map<TimelineMetric, MetricHostAggregate> hostAggregateMap,
                                       String phoenixTableName) throws SQLException {

    if (hostAggregateMap == null || hostAggregateMap.isEmpty()) {
      LOG.debug("Empty aggregate records.");
      return;
    }

    Connection conn = getConnection();
    PreparedStatement stmt = null;

    long start = System.currentTimeMillis();
    int rowCount = 0;

    try {
      stmt = conn.prepareStatement(
        String.format(UPSERT_AGGREGATE_RECORD_SQL, phoenixTableName));

      for (Map.Entry<TimelineMetric, MetricHostAggregate> metricAggregate :
        hostAggregateMap.entrySet()) {

        TimelineMetric metric = metricAggregate.getKey();
        MetricHostAggregate hostAggregate = metricAggregate.getValue();

        rowCount++;
        stmt.clearParameters();
        stmt.setString(1, metric.getMetricName());
        stmt.setString(2, metric.getHostName());
        stmt.setString(3, metric.getAppId());
        stmt.setString(4, metric.getInstanceId());
        stmt.setLong(5, metric.getTimestamp());
        stmt.setString(6, metric.getType());
        stmt.setDouble(7, hostAggregate.getSum());
        stmt.setDouble(8, hostAggregate.getMax());
        stmt.setDouble(9, hostAggregate.getMin());
        stmt.setDouble(10, hostAggregate.getNumberOfSamples());

        try {
          stmt.executeUpdate();
        } catch (SQLException sql) {
          LOG.error(sql);
        }

        if (rowCount >= PHOENIX_MAX_MUTATION_STATE_SIZE - 1) {
          conn.commit();
          rowCount = 0;
        }

      }

      conn.commit();

    } finally {
      if (stmt != null) {
        try {
          stmt.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (conn != null) {
        try {
          conn.close();
        } catch (SQLException sql) {
          // Ignore
        }
      }
    }

    long end = System.currentTimeMillis();

    if ((end - start) > 60000l) {
      LOG.info("Time to save map: " + (end - start) + ", " +
        "thread = " + Thread.currentThread().getClass());
    }
  }

  /**
   * Save Metric aggregate records.
   *
   * @throws SQLException
   */
  public void saveClusterAggregateRecords(Map<TimelineClusterMetric, MetricClusterAggregate> records)
      throws SQLException {

    if (records == null || records.isEmpty()) {
      LOG.debug("Empty aggregate records.");
      return;
    }

    long start = System.currentTimeMillis();
    String sqlStr = String.format(UPSERT_CLUSTER_AGGREGATE_SQL, METRICS_CLUSTER_AGGREGATE_TABLE_NAME);
    Connection conn = getConnection();
    PreparedStatement stmt = null;
    try {
      stmt = conn.prepareStatement(sqlStr);
      int rowCount = 0;

      for (Map.Entry<TimelineClusterMetric, MetricClusterAggregate>
        aggregateEntry : records.entrySet()) {
        TimelineClusterMetric clusterMetric = aggregateEntry.getKey();
        MetricClusterAggregate aggregate = aggregateEntry.getValue();

        if (LOG.isTraceEnabled()) {
          LOG.trace("clusterMetric = " + clusterMetric + ", " +
            "aggregate = " + aggregate);
        }

        rowCount++;
        stmt.clearParameters();
        stmt.setString(1, clusterMetric.getMetricName());
        stmt.setString(2, clusterMetric.getAppId());
        stmt.setString(3, clusterMetric.getInstanceId());
        stmt.setLong(4, clusterMetric.getTimestamp());
        stmt.setString(5, clusterMetric.getType());
        stmt.setDouble(6, aggregate.getSum());
        stmt.setInt(7, aggregate.getNumberOfHosts());
        stmt.setDouble(8, aggregate.getMax());
        stmt.setDouble(9, aggregate.getMin());

        try {
          stmt.executeUpdate();
        } catch (SQLException sql) {
          // we have no way to verify it works!!!
          LOG.error(sql);
        }

        if (rowCount >= PHOENIX_MAX_MUTATION_STATE_SIZE - 1) {
          conn.commit();
          rowCount = 0;
        }
      }

      conn.commit();

    } finally {
      if (stmt != null) {
        try {
          stmt.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (conn != null) {
        try {
          conn.close();
        } catch (SQLException sql) {
          // Ignore
        }
      }
    }
    long end = System.currentTimeMillis();
    if ((end - start) > 60000l) {
      LOG.info("Time to save: " + (end - start) + ", " +
        "thread = " + Thread.currentThread().getName());
    }
  }


  /**
   * Save Metric aggregate records.
   *
   * @throws SQLException
   */
  public void saveClusterTimeAggregateRecords(Map<TimelineClusterMetric, MetricHostAggregate> records,
                                              String tableName) throws SQLException {
    if (records == null || records.isEmpty()) {
      LOG.debug("Empty aggregate records.");
      return;
    }

    long start = System.currentTimeMillis();

    Connection conn = getConnection();
    PreparedStatement stmt = null;
    try {
      stmt = conn.prepareStatement(String.format(UPSERT_CLUSTER_AGGREGATE_TIME_SQL, tableName));
      int rowCount = 0;

      for (Map.Entry<TimelineClusterMetric, MetricHostAggregate> aggregateEntry : records.entrySet()) {
        TimelineClusterMetric clusterMetric = aggregateEntry.getKey();
        MetricHostAggregate aggregate = aggregateEntry.getValue();

        if (LOG.isTraceEnabled()) {
          LOG.trace("clusterMetric = " + clusterMetric + ", " +
            "aggregate = " + aggregate);
        }

        rowCount++;
        stmt.clearParameters();
        stmt.setString(1, clusterMetric.getMetricName());
        stmt.setString(2, clusterMetric.getAppId());
        stmt.setString(3, clusterMetric.getInstanceId());
        stmt.setLong(4, clusterMetric.getTimestamp());
        stmt.setString(5, clusterMetric.getType());
        stmt.setDouble(6, aggregate.getSum());
        stmt.setLong(7, aggregate.getNumberOfSamples());
        stmt.setDouble(8, aggregate.getMax());
        stmt.setDouble(9, aggregate.getMin());

        try {
          stmt.executeUpdate();
        } catch (SQLException sql) {
          // we have no way to verify it works!!!
          LOG.error(sql);
        }

        if (rowCount >= PHOENIX_MAX_MUTATION_STATE_SIZE - 1) {
          conn.commit();
          rowCount = 0;
        }
      }

      conn.commit();

    } finally {
      if (stmt != null) {
        try {
          stmt.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (conn != null) {
        try {
          conn.close();
        } catch (SQLException sql) {
          // Ignore
        }
      }
    }
    long end = System.currentTimeMillis();
    if ((end - start) > 60000l) {
      LOG.info("Time to save: " + (end - start) + ", " +
        "thread = " + Thread.currentThread().getName());
    }
  }

  /**
   * Provide skip block cache hint for aggregator queries.
   */
  public boolean isSkipBlockCacheForAggregatorsEnabled() {
    return skipBlockCacheForAggregatorsEnabled;
  }

  /**
   * One time save of metadata when discovering topology during aggregation.
   * @throws SQLException
   */
  public void saveHostAppsMetadata(Map<String, Set<String>> hostedApps) throws SQLException {
    Connection conn = getConnection();
    PreparedStatement stmt = null;
    try {
      stmt = conn.prepareStatement(UPSERT_HOSTED_APPS_METADATA_SQL);
      int rowCount = 0;

      for (Map.Entry<String, Set<String>> hostedAppsEntry : hostedApps.entrySet()) {
        if (LOG.isTraceEnabled()) {
          LOG.trace("HostedAppsMetadata: " + hostedAppsEntry);
        }

        stmt.clearParameters();
        stmt.setString(1, hostedAppsEntry.getKey());
        stmt.setString(2, StringUtils.join(hostedAppsEntry.getValue(), ","));
        try {
          stmt.executeUpdate();
          rowCount++;
        } catch (SQLException sql) {
          LOG.error("Error saving hosted apps metadata.", sql);
        }
      }

      conn.commit();
      LOG.info("Saved " + rowCount + " hosted apps metadata records.");

    } finally {
      if (stmt != null) {
        try {
          stmt.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (conn != null) {
        try {
          conn.close();
        } catch (SQLException sql) {
          // Ignore
        }
      }
    }
  }

  /**
   * Save metdata on updates.
   * @param metricMetadata @Collection<@TimelineMetricMetadata>
   * @throws SQLException
   */
  public void saveMetricMetadata(Collection<TimelineMetricMetadata> metricMetadata) throws SQLException {
    if (metricMetadata.isEmpty()) {
      LOG.info("No metadata records to save.");
      return;
    }

    Connection conn = getConnection();
    PreparedStatement stmt = null;

    try {
      stmt = conn.prepareStatement(UPSERT_METADATA_SQL);
      int rowCount = 0;

      for (TimelineMetricMetadata metadata : metricMetadata) {
        if (LOG.isTraceEnabled()) {
          LOG.trace("TimelineMetricMetadata: metricName = " + metadata.getMetricName()
            + ", appId = " + metadata.getAppId()
            + ", seriesStartTime = " + metadata.getSeriesStartTime()
          );
        }

        stmt.clearParameters();
        stmt.setString(1, metadata.getMetricName());
        stmt.setString(2, metadata.getAppId());
        stmt.setString(3, metadata.getUnits());
        stmt.setString(4, metadata.getType());
        stmt.setLong(5, metadata.getSeriesStartTime());
        stmt.setBoolean(6, metadata.isSupportsAggregates());

        try {
          stmt.executeUpdate();
          rowCount++;
        } catch (SQLException sql) {
          LOG.error("Error saving metadata.", sql);
        }
      }

      conn.commit();
      LOG.info("Saved " + rowCount + " metadata records.");

    } finally {
      if (stmt != null) {
        try {
          stmt.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (conn != null) {
        try {
          conn.close();
        } catch (SQLException sql) {
          // Ignore
        }
      }
    }
  }

  public Map<String, Set<String>> getHostedAppsMetadata() throws SQLException {
    Map<String, Set<String>> hostedAppMap = new HashMap<>();
    Connection conn = getConnection();
    PreparedStatement stmt = null;
    ResultSet rs = null;

    try {
      stmt = conn.prepareStatement(GET_HOSTED_APPS_METADATA_SQL);
      rs = stmt.executeQuery();

      while (rs.next()) {
        hostedAppMap.put(rs.getString("HOSTNAME"),
          new HashSet<>(Arrays.asList(StringUtils.split(rs.getString("APP_IDS"), ","))));
      }

    } finally {
      if (rs != null) {
        try {
          rs.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (stmt != null) {
        try {
          stmt.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (conn != null) {
        try {
          conn.close();
        } catch (SQLException sql) {
          // Ignore
        }
      }
    }

    return hostedAppMap;
  }

  // No filter criteria support for now.
  public Map<TimelineMetricMetadataKey, TimelineMetricMetadata> getTimelineMetricMetadata() throws SQLException {
    Map<TimelineMetricMetadataKey, TimelineMetricMetadata> metadataMap = new HashMap<>();
    Connection conn = getConnection();
    PreparedStatement stmt = null;
    ResultSet rs = null;

    try {
      stmt = conn.prepareStatement(GET_METRIC_METADATA_SQL);
      rs = stmt.executeQuery();

      while (rs.next()) {
        String metricName = rs.getString("METRIC_NAME");
        String appId = rs.getString("APP_ID");
        TimelineMetricMetadata metadata = new TimelineMetricMetadata(
          metricName,
          appId,
          rs.getString("UNITS"),
          rs.getString("TYPE"),
          rs.getLong("START_TIME"),
          rs.getBoolean("SUPPORTS_AGGREGATION")
        );

        TimelineMetricMetadataKey key = new TimelineMetricMetadataKey(metricName, appId);
        metadata.setIsPersisted(true); // Always true on retrieval
        metadataMap.put(key, metadata);
      }

    } finally {
      if (rs != null) {
        try {
          rs.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (stmt != null) {
        try {
          stmt.close();
        } catch (SQLException e) {
          // Ignore
        }
      }
      if (conn != null) {
        try {
          conn.close();
        } catch (SQLException sql) {
          // Ignore
        }
      }
    }

    return metadataMap;
  }
}