You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2017/02/07 20:59:30 UTC

[62/70] [abbrv] hive git commit: HIVE-14754: Track the queries execution lifecycle times (Barna Zsombor Klara via Szehon)

HIVE-14754: Track the queries execution lifecycle times (Barna Zsombor Klara via Szehon)


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

Branch: refs/heads/hive-14535
Commit: 3c230a62d5200a2e7f138d51746e09a8d028b3ca
Parents: 95d0ce7
Author: Szehon Ho <sz...@criteo.com>
Authored: Tue Feb 7 11:57:49 2017 +0100
Committer: Szehon Ho <sz...@criteo.com>
Committed: Tue Feb 7 11:57:49 2017 +0100

----------------------------------------------------------------------
 .../hive/common/metrics/LegacyMetrics.java      |   4 +
 .../hive/common/metrics/common/Metrics.java     |   6 +
 .../common/metrics/common/MetricsConstant.java  |   8 +
 .../metrics/metrics2/CodahaleMetrics.java       |  42 ++++-
 .../apache/hadoop/hive/ql/log/PerfLogger.java   |   8 +-
 .../hive/common/metrics/MetricsTestUtils.java   |   2 +
 .../metrics/metrics2/TestCodahaleMetrics.java   |  20 ++-
 .../hadoop/hive/metastore/HiveMetaStore.java    |   4 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  26 +++-
 .../hive/ql/hooks/MetricsQueryLifeTimeHook.java |  62 ++++++++
 .../ql/hooks/TestMetricsQueryLifeTimeHook.java  | 117 ++++++++++++++
 .../hive/service/cli/operation/Operation.java   |  23 +--
 .../service/cli/operation/SQLOperation.java     |  50 ++++--
 .../cli/operation/TestSQLOperationMetrics.java  | 156 +++++++++++++++++++
 14 files changed, 474 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3c230a62/common/src/java/org/apache/hadoop/hive/common/metrics/LegacyMetrics.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/LegacyMetrics.java b/common/src/java/org/apache/hadoop/hive/common/metrics/LegacyMetrics.java
index 0f082f6..090db3d 100644
--- a/common/src/java/org/apache/hadoop/hive/common/metrics/LegacyMetrics.java
+++ b/common/src/java/org/apache/hadoop/hive/common/metrics/LegacyMetrics.java
@@ -231,6 +231,10 @@ public class LegacyMetrics implements Metrics {
     //Not implemented
   }
 
+  public void markMeter(String name) {
+    //Not implemented.
+  }
+
   public void set(String name, Object value) {
     metrics.put(name,value);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/3c230a62/common/src/java/org/apache/hadoop/hive/common/metrics/common/Metrics.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/common/Metrics.java b/common/src/java/org/apache/hadoop/hive/common/metrics/common/Metrics.java
index 8fb7c5a..368dd4f 100644
--- a/common/src/java/org/apache/hadoop/hive/common/metrics/common/Metrics.java
+++ b/common/src/java/org/apache/hadoop/hive/common/metrics/common/Metrics.java
@@ -103,4 +103,10 @@ public interface Metrics {
   public void addRatio(String name, MetricsVariable<Integer> numerator,
                            MetricsVariable<Integer> denominator);
 
+  /**
+   * Mark an event occurance for a meter. Meters measure the rate of an event and track
+   * 1/5/15 minute moving averages
+   * @param name name of the meter
+   */
+  public void markMeter(String name);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/3c230a62/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java
index b4a7dcc..4c53297 100644
--- a/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java
+++ b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java
@@ -22,6 +22,9 @@ package org.apache.hadoop.hive.common.metrics.common;
  */
 public class MetricsConstant {
 
+  public static final String API_PREFIX = "api_";
+  public static final String ACTIVE_CALLS = "active_calls_";
+
   public static final String JVM_PAUSE_INFO = "jvm.pause.info-threshold";
   public static final String JVM_PAUSE_WARN = "jvm.pause.warn-threshold";
   public static final String JVM_EXTRA_SLEEP = "jvm.pause.extraSleepTime";
@@ -73,4 +76,9 @@ public class MetricsConstant {
   public static final String HS2_AVG_OPEN_SESSION_TIME = "hs2_avg_open_session_time";
   public static final String HS2_AVG_ACTIVE_SESSION_TIME = "hs2_avg_active_session_time";
 
+  public static final String HS2_SUBMITTED_QURIES = "hs2_submitted_queries";
+  public static final String HS2_COMPILING_QUERIES = "hs2_compiling_queries";
+  public static final String HS2_EXECUTING_QUERIES = "hs2_executing_queries";
+  public static final String HS2_FAILED_QUERIES = "hs2_failed_queries";
+  public static final String HS2_SUCEEDED_QUERIES = "hs2_suceeded_queries";
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/3c230a62/common/src/java/org/apache/hadoop/hive/common/metrics/metrics2/CodahaleMetrics.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/metrics2/CodahaleMetrics.java b/common/src/java/org/apache/hadoop/hive/common/metrics/metrics2/CodahaleMetrics.java
index cd3d627..e8abf6c 100644
--- a/common/src/java/org/apache/hadoop/hive/common/metrics/metrics2/CodahaleMetrics.java
+++ b/common/src/java/org/apache/hadoop/hive/common/metrics/metrics2/CodahaleMetrics.java
@@ -23,6 +23,7 @@ import com.codahale.metrics.Counter;
 import com.codahale.metrics.ExponentiallyDecayingReservoir;
 import com.codahale.metrics.Gauge;
 import com.codahale.metrics.JmxReporter;
+import com.codahale.metrics.Meter;
 import com.codahale.metrics.Metric;
 import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.MetricSet;
@@ -46,6 +47,7 @@ import com.google.common.collect.Lists;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
 import org.apache.hadoop.hive.common.metrics.common.MetricsScope;
 import org.apache.hadoop.hive.common.metrics.common.MetricsVariable;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -76,17 +78,17 @@ import java.util.concurrent.locks.ReentrantLock;
  */
 public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.common.Metrics {
 
-  public static final String API_PREFIX = "api_";
-  public static final String ACTIVE_CALLS = "active_calls_";
   public static final Logger LOGGER = LoggerFactory.getLogger(CodahaleMetrics.class);
 
   public final MetricRegistry metricRegistry = new MetricRegistry();
   private final Lock timersLock = new ReentrantLock();
   private final Lock countersLock = new ReentrantLock();
   private final Lock gaugesLock = new ReentrantLock();
+  private final Lock metersLock = new ReentrantLock();
 
   private LoadingCache<String, Timer> timers;
   private LoadingCache<String, Counter> counters;
+  private LoadingCache<String, Meter> meters;
   private ConcurrentHashMap<String, Gauge> gauges;
 
   private HiveConf conf;
@@ -126,7 +128,7 @@ public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.co
       if (!isOpen) {
         isOpen = true;
         this.timerContext = timer.time();
-        CodahaleMetrics.this.incrementCounter(ACTIVE_CALLS + name);
+        CodahaleMetrics.this.incrementCounter(MetricsConstant.ACTIVE_CALLS + name);
       } else {
         LOGGER.warn("Scope named " + name + " is not closed, cannot be opened.");
       }
@@ -138,7 +140,7 @@ public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.co
     public void close() {
       if (isOpen) {
         timerContext.close();
-        CodahaleMetrics.this.decrementCounter(ACTIVE_CALLS + name);
+        CodahaleMetrics.this.decrementCounter(MetricsConstant.ACTIVE_CALLS + name);
       } else {
         LOGGER.warn("Scope named " + name + " is not open, cannot be closed.");
       }
@@ -169,6 +171,16 @@ public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.co
         }
       }
     );
+    meters = CacheBuilder.newBuilder().build(
+        new CacheLoader<String, Meter>() {
+          @Override
+          public Meter load(String key) {
+            Meter meter = new Meter();
+            metricRegistry.register(key, meter);
+            return meter;
+          }
+        }
+    );
     gauges = new ConcurrentHashMap<String, Gauge>();
 
     //register JVM metrics
@@ -209,11 +221,11 @@ public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.co
     }
     timers.invalidateAll();
     counters.invalidateAll();
+    meters.invalidateAll();
   }
 
   @Override
   public void startStoredScope(String name) {
-    name = API_PREFIX + name;
     if (threadLocalScopes.get().containsKey(name)) {
       threadLocalScopes.get().get(name).open();
     } else {
@@ -223,7 +235,6 @@ public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.co
 
   @Override
   public void endStoredScope(String name) {
-    name = API_PREFIX + name;
     if (threadLocalScopes.get().containsKey(name)) {
       threadLocalScopes.get().get(name).close();
       threadLocalScopes.get().remove(name);
@@ -239,7 +250,6 @@ public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.co
   }
 
   public MetricsScope createScope(String name) {
-    name = API_PREFIX + name;
     return new CodahaleMetricsScope(name);
   }
 
@@ -322,6 +332,21 @@ public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.co
     }
   }
 
+  @Override
+  public void markMeter(String name) {
+    String key = name;
+    try {
+      metersLock.lock();
+      Meter meter = meters.get(name);
+      meter.mark();
+    } catch (ExecutionException e) {
+      throw new IllegalStateException("Error retrieving meter " + name
+          + " from the metric registry ", e);
+    } finally {
+      metersLock.unlock();
+    }
+  }
+
   // This method is necessary to synchronize lazy-creation to the timers.
   private Timer getTimer(String name) {
     String key = name;
@@ -330,7 +355,8 @@ public class CodahaleMetrics implements org.apache.hadoop.hive.common.metrics.co
       Timer timer = timers.get(key);
       return timer;
     } catch (ExecutionException e) {
-      throw new IllegalStateException("Error retrieving timer from the metric registry ", e);
+      throw new IllegalStateException("Error retrieving timer " + name
+          + " from the metric registry ", e);
     } finally {
       timersLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/3c230a62/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java
index 7658f1c..7f3c8b3 100644
--- a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java
+++ b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.log;
 
 import com.google.common.collect.ImmutableMap;
 import org.apache.hadoop.hive.common.metrics.common.Metrics;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
 import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
 import org.apache.hadoop.hive.common.metrics.common.MetricsScope;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -27,13 +28,8 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.Map;
-import java.util.Set;
 
 /**
  * PerfLogger.
@@ -225,7 +221,7 @@ public class PerfLogger {
   private void beginMetrics(String method) {
     Metrics metrics = MetricsFactory.getInstance();
     if (metrics != null) {
-      MetricsScope scope = metrics.createScope(method);
+      MetricsScope scope = metrics.createScope(MetricsConstant.API_PREFIX + method);
       openScopes.put(method, scope);
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3c230a62/common/src/test/org/apache/hadoop/hive/common/metrics/MetricsTestUtils.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/metrics/MetricsTestUtils.java b/common/src/test/org/apache/hadoop/hive/common/metrics/MetricsTestUtils.java
index 3bb7a1e..5c38780 100644
--- a/common/src/test/org/apache/hadoop/hive/common/metrics/MetricsTestUtils.java
+++ b/common/src/test/org/apache/hadoop/hive/common/metrics/MetricsTestUtils.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.common.metrics;
 
+import com.codahale.metrics.Meter;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.junit.Assert;
@@ -33,6 +34,7 @@ public class MetricsTestUtils {
   public static final MetricsCategory COUNTER = new MetricsCategory("counters", "count");
   public static final MetricsCategory TIMER = new MetricsCategory("timers", "count");
   public static final MetricsCategory GAUGE = new MetricsCategory("gauges", "value");
+  public static final MetricsCategory METER = new MetricsCategory("meters", "count");
 
   static class MetricsCategory {
     String category;

http://git-wip-us.apache.org/repos/asf/hive/blob/3c230a62/common/src/test/org/apache/hadoop/hive/common/metrics/metrics2/TestCodahaleMetrics.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/metrics/metrics2/TestCodahaleMetrics.java b/common/src/test/org/apache/hadoop/hive/common/metrics/metrics2/TestCodahaleMetrics.java
index 6ee6245..aa4e75f 100644
--- a/common/src/test/org/apache/hadoop/hive/common/metrics/metrics2/TestCodahaleMetrics.java
+++ b/common/src/test/org/apache/hadoop/hive/common/metrics/metrics2/TestCodahaleMetrics.java
@@ -80,7 +80,7 @@ public class TestCodahaleMetrics {
       MetricsFactory.getInstance().endStoredScope("method1");
     }
 
-    Timer timer = metricRegistry.getTimers().get("api_method1");
+    Timer timer = metricRegistry.getTimers().get("method1");
     Assert.assertEquals(5, timer.getCount());
     Assert.assertTrue(timer.getMeanRate() > 0);
   }
@@ -113,7 +113,7 @@ public class TestCodahaleMetrics {
     }
     executorService.shutdown();
     assertTrue(executorService.awaitTermination(10000, TimeUnit.MILLISECONDS));
-    Timer timer = metricRegistry.getTimers().get("api_method2");
+    Timer timer = metricRegistry.getTimers().get("method2");
     Assert.assertEquals(4, timer.getCount());
     Assert.assertTrue(timer.getMeanRate() > 0);
   }
@@ -161,4 +161,20 @@ public class TestCodahaleMetrics {
     json = ((CodahaleMetrics) MetricsFactory.getInstance()).dumpJson();
     MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.GAUGE, "gauge1", testVar.getValue());
   }
+
+  @Test
+  public void testMeter() throws Exception {
+
+    String json = ((CodahaleMetrics) MetricsFactory.getInstance()).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER, "meter", "");
+
+    MetricsFactory.getInstance().markMeter("meter");
+    json = ((CodahaleMetrics) MetricsFactory.getInstance()).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER, "meter", "1");
+
+    MetricsFactory.getInstance().markMeter("meter");
+    json = ((CodahaleMetrics) MetricsFactory.getInstance()).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER, "meter", "2");
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/3c230a62/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index f8c3c4e..12485a9 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -780,7 +780,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       logInfo((getThreadLocalIpAddress() == null ? "" : "source:" + getThreadLocalIpAddress() + " ") +
           function + extraLogInfo);
       if (MetricsFactory.getInstance() != null) {
-        MetricsFactory.getInstance().startStoredScope(function);
+        MetricsFactory.getInstance().startStoredScope(MetricsConstant.API_PREFIX + function);
       }
       return function;
     }
@@ -819,7 +819,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
     private void endFunction(String function, MetaStoreEndFunctionContext context) {
       if (MetricsFactory.getInstance() != null) {
-        MetricsFactory.getInstance().endStoredScope(function);
+        MetricsFactory.getInstance().endStoredScope(MetricsConstant.API_PREFIX + function);
       }
 
       for (MetaStoreEndFunctionListener listener : endFunctionListeners) {

http://git-wip-us.apache.org/repos/asf/hive/blob/3c230a62/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index d6b7f08..2423471 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -37,6 +37,7 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantLock;
 
+import com.google.common.collect.Iterables;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hive.common.ValidTxnList;
@@ -65,6 +66,7 @@ import org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext;
 import org.apache.hadoop.hive.ql.hooks.Hook;
 import org.apache.hadoop.hive.ql.hooks.HookContext;
 import org.apache.hadoop.hive.ql.hooks.HookUtils;
+import org.apache.hadoop.hive.ql.hooks.MetricsQueryLifeTimeHook;
 import org.apache.hadoop.hive.ql.hooks.PostExecute;
 import org.apache.hadoop.hive.ql.hooks.PreExecute;
 import org.apache.hadoop.hive.ql.hooks.QueryLifeTimeHook;
@@ -88,7 +90,6 @@ import org.apache.hadoop.hive.ql.optimizer.ppr.PartitionPruner;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.ColumnAccessInfo;
-import org.apache.hadoop.hive.ql.parse.ExplainSemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHook;
 import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContext;
 import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContextImpl;
@@ -178,7 +179,7 @@ public class Driver implements CommandProcessor {
   private QueryState queryState;
 
   // Query hooks that execute before compilation and after execution
-  List<QueryLifeTimeHook> queryHooks;
+  private List<QueryLifeTimeHook> queryHooks;
 
   public enum DriverState {
     INITIALIZED,
@@ -432,8 +433,8 @@ public class Driver implements CommandProcessor {
 
     // Whether any error occurred during query compilation. Used for query lifetime hook.
     boolean compileError = false;
-
     try {
+
       // Initialize the transaction manager.  This must be done before analyze is called.
       final HiveTxnManager txnManager = SessionState.get().initTxnMgr(conf);
       // In case when user Ctrl-C twice to kill Hive CLI JVM, we want to release locks
@@ -471,7 +472,7 @@ public class Driver implements CommandProcessor {
       perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PARSE);
 
       // Trigger query hook before compilation
-      queryHooks = getHooks(ConfVars.HIVE_QUERY_LIFETIME_HOOKS, QueryLifeTimeHook.class);
+      queryHooks = loadQueryHooks();
       if (queryHooks != null && !queryHooks.isEmpty()) {
         QueryLifeTimeHookContext qhc = new QueryLifeTimeHookContextImpl();
         qhc.setHiveConf(conf);
@@ -666,6 +667,19 @@ public class Driver implements CommandProcessor {
     }
   }
 
+  private List<QueryLifeTimeHook> loadQueryHooks() throws Exception {
+    List<QueryLifeTimeHook> hooks = new ArrayList<>();
+
+    if (conf.getBoolVar(ConfVars.HIVE_SERVER2_METRICS_ENABLED)) {
+      hooks.add(new MetricsQueryLifeTimeHook());
+    }
+    List<QueryLifeTimeHook> propertyDefinedHoooks = getHooks(ConfVars.HIVE_QUERY_LIFETIME_HOOKS, QueryLifeTimeHook.class);
+    if (propertyDefinedHoooks != null) {
+      Iterables.addAll(hooks, propertyDefinedHoooks);
+    }
+    return hooks;
+  }
+
   private ImmutableMap<String, Long> dumpMetaCallTimingWithoutEx(String phase) {
     try {
       return Hive.get().dumpAndClearMetaCallTiming(phase);
@@ -1687,6 +1701,7 @@ public class Driver implements CommandProcessor {
 
     boolean noName = StringUtils.isEmpty(conf.get(MRJobConfig.JOB_NAME));
     int maxlen = conf.getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH);
+    Metrics metrics = MetricsFactory.getInstance();
 
     String queryId = conf.getVar(HiveConf.ConfVars.HIVEQUERYID);
     // Get the query string from the conf file as the compileInternal() method might
@@ -1810,7 +1825,6 @@ public class Driver implements CommandProcessor {
         assert tsk.getParentTasks() == null || tsk.getParentTasks().isEmpty();
         driverCxt.addToRunnable(tsk);
 
-        Metrics metrics = MetricsFactory.getInstance();
         if (metrics != null) {
           tsk.updateTaskMetrics(metrics);
         }
@@ -2434,7 +2448,7 @@ public class Driver implements CommandProcessor {
     this.operationId = opId;
   }
 
-  /** 
+  /**
    * Resets QueryState to get new queryId on Driver reuse.
    */
   public void resetQueryState() {

http://git-wip-us.apache.org/repos/asf/hive/blob/3c230a62/ql/src/java/org/apache/hadoop/hive/ql/hooks/MetricsQueryLifeTimeHook.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/MetricsQueryLifeTimeHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/MetricsQueryLifeTimeHook.java
new file mode 100644
index 0000000..246ce0e
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/MetricsQueryLifeTimeHook.java
@@ -0,0 +1,62 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hive.ql.hooks;
+
+import org.apache.hadoop.hive.common.metrics.common.Metrics;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
+import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
+import org.apache.hadoop.hive.common.metrics.common.MetricsScope;
+
+/**
+ * LifeTimeHook gathering metrics for the query lifecycle if the
+ * metrics are enabled
+ */
+public class MetricsQueryLifeTimeHook implements QueryLifeTimeHook {
+
+  private Metrics metrics = MetricsFactory.getInstance();
+  private MetricsScope compilingQryScp;
+  private MetricsScope executingQryScp;
+
+  @Override
+  public void beforeCompile(QueryLifeTimeHookContext ctx) {
+    if (metrics != null) {
+      compilingQryScp = metrics.createScope(MetricsConstant.HS2_COMPILING_QUERIES);
+    }
+  }
+
+  @Override
+  public void afterCompile(QueryLifeTimeHookContext ctx, boolean hasError) {
+    if (metrics != null && compilingQryScp != null) {
+      metrics.endScope(compilingQryScp);
+    }
+  }
+
+  @Override
+  public void beforeExecution(QueryLifeTimeHookContext ctx) {
+    if (metrics != null) {
+      executingQryScp = metrics.createScope(MetricsConstant.HS2_EXECUTING_QUERIES);
+    }
+  }
+
+  @Override
+  public void afterExecution(QueryLifeTimeHookContext ctx, boolean hasError) {
+    if (metrics != null && executingQryScp != null) {
+      metrics.endScope(executingQryScp);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/3c230a62/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestMetricsQueryLifeTimeHook.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestMetricsQueryLifeTimeHook.java b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestMetricsQueryLifeTimeHook.java
new file mode 100644
index 0000000..0fb7e3e
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestMetricsQueryLifeTimeHook.java
@@ -0,0 +1,117 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hive.ql.hooks;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
+import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
+import org.apache.hadoop.hive.common.metrics.metrics2.CodahaleMetrics;
+import org.apache.hadoop.hive.common.metrics.metrics2.MetricsReporting;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestMetricsQueryLifeTimeHook {
+
+  private MetricsQueryLifeTimeHook hook;
+  private QueryLifeTimeHookContext ctx;
+  private MetricRegistry metricRegistry;
+
+  @Before
+  public void before() throws Exception {
+    HiveConf conf = new HiveConf();
+
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "local");
+    conf.setVar(HiveConf.ConfVars.HIVE_METRICS_CLASS, CodahaleMetrics.class.getCanonicalName());
+    conf.setVar(HiveConf.ConfVars.HIVE_METRICS_REPORTER, MetricsReporting.JSON_FILE.name()
+        + "," + MetricsReporting.JMX.name());
+    conf.setVar(HiveConf.ConfVars.HIVE_METRICS_JSON_FILE_INTERVAL, "100000s");
+
+    MetricsFactory.init(conf);
+    metricRegistry = ((CodahaleMetrics) MetricsFactory.getInstance()).getMetricRegistry();
+
+    hook = new MetricsQueryLifeTimeHook();
+    ctx = new QueryLifeTimeHookContextImpl();
+  }
+
+  @Test
+  public void testCompilationQueryMetric() {
+    Timer timer = metricRegistry.getTimers().get(MetricsConstant.HS2_COMPILING_QUERIES);
+    Counter counter = metricRegistry.getCounters()
+        .get(MetricsConstant.ACTIVE_CALLS + MetricsConstant.HS2_COMPILING_QUERIES);
+    assertThat(timer, nullValue());
+    assertThat(counter, nullValue());
+
+    hook.beforeCompile(ctx);
+    timer = metricRegistry.getTimers().get(MetricsConstant.HS2_COMPILING_QUERIES);
+    counter = metricRegistry.getCounters()
+        .get(MetricsConstant.ACTIVE_CALLS + MetricsConstant.HS2_COMPILING_QUERIES);
+    assertThat(timer.getCount(), equalTo(0l));
+    assertThat(counter.getCount(), equalTo(1l));
+
+    hook.afterCompile(ctx, false);
+    timer = metricRegistry.getTimers().get(MetricsConstant.HS2_COMPILING_QUERIES);
+    counter = metricRegistry.getCounters()
+        .get(MetricsConstant.ACTIVE_CALLS + MetricsConstant.HS2_COMPILING_QUERIES);
+    assertThat(timer.getCount(), equalTo(1l));
+    assertThat(counter.getCount(), equalTo(0l));
+  }
+
+  @Test
+  public void testExecutionQueryMetric() {
+    Timer timer = metricRegistry.getTimers().get(MetricsConstant.HS2_EXECUTING_QUERIES);
+    Counter counter = metricRegistry.getCounters()
+        .get(MetricsConstant.ACTIVE_CALLS + MetricsConstant.HS2_EXECUTING_QUERIES);
+    assertThat(timer, nullValue());
+    assertThat(counter, nullValue());
+
+    hook.beforeExecution(ctx);
+    timer = metricRegistry.getTimers().get(MetricsConstant.HS2_EXECUTING_QUERIES);
+    counter = metricRegistry.getCounters()
+        .get(MetricsConstant.ACTIVE_CALLS + MetricsConstant.HS2_EXECUTING_QUERIES);
+    assertThat(timer.getCount(), equalTo(0l));
+    assertThat(counter.getCount(), equalTo(1l));
+
+    hook.afterExecution(ctx, false);
+    timer = metricRegistry.getTimers().get(MetricsConstant.HS2_EXECUTING_QUERIES);
+    counter = metricRegistry.getCounters()
+        .get(MetricsConstant.ACTIVE_CALLS + MetricsConstant.HS2_EXECUTING_QUERIES);
+    assertThat(timer.getCount(), equalTo(1l));
+    assertThat(counter.getCount(), equalTo(0l));
+  }
+
+  @Test
+  public void testNoErrorOnDisabledMetrics() throws Exception {
+    MetricsFactory.close();
+    MetricsQueryLifeTimeHook emptyhook = new MetricsQueryLifeTimeHook();
+
+    assertThat(MetricsFactory.getInstance(), nullValue());
+
+    emptyhook.beforeCompile(ctx);
+    emptyhook.afterCompile(ctx, false);
+    emptyhook.beforeExecution(ctx);
+    emptyhook.afterExecution(ctx, false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/3c230a62/service/src/java/org/apache/hive/service/cli/operation/Operation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/Operation.java b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
index 28ca41a..2039946 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/Operation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
@@ -19,7 +19,6 @@ package org.apache.hive.service.cli.operation;
 
 import java.io.File;
 import java.io.FileNotFoundException;
-import java.io.IOException;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
@@ -104,7 +103,9 @@ public abstract class Operation {
     lastAccessTime = beginTime;
     operationTimeout = HiveConf.getTimeVar(parentSession.getHiveConf(),
         HiveConf.ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT, TimeUnit.MILLISECONDS);
-    setMetrics(state);
+
+    currentStateScope = updateOperationStateMetrics(null, MetricsConstant.OPERATION_PREFIX,
+        MetricsConstant.COMPLETED_OPERATION_PREFIX, state);
     queryState = new QueryState(parentSession.getHiveConf(), confOverlay, isAsyncQueryState);
   }
 
@@ -163,7 +164,8 @@ public abstract class Operation {
     state.validateTransition(newState);
     OperationState prevState = state;
     this.state = newState;
-    setMetrics(state);
+    currentStateScope = updateOperationStateMetrics(currentStateScope, MetricsConstant.OPERATION_PREFIX,
+        MetricsConstant.COMPLETED_OPERATION_PREFIX, state);
     onNewState(state, prevState);
     this.lastAccessTime = System.currentTimeMillis();
     return this.state;
@@ -325,11 +327,7 @@ public abstract class Operation {
     try {
       Metrics metrics = MetricsFactory.getInstance();
       if (metrics != null) {
-        try {
-          metrics.incrementCounter(MetricsConstant.OPEN_OPERATIONS);
-        } catch (Exception e) {
-          LOG.warn("Error Reporting open operation to Metrics system", e);
-        }
+        metrics.incrementCounter(MetricsConstant.OPEN_OPERATIONS);
       }
       runInternal();
     } finally {
@@ -414,12 +412,7 @@ public abstract class Operation {
     OperationState.UNKNOWN
   );
 
-  private void setMetrics(OperationState state) {
-    currentStateScope = setMetrics(currentStateScope, MetricsConstant.OPERATION_PREFIX,
-      MetricsConstant.COMPLETED_OPERATION_PREFIX, state);
-  }
-
-  protected static MetricsScope setMetrics(MetricsScope stateScope, String operationPrefix,
+  protected final MetricsScope updateOperationStateMetrics(MetricsScope stateScope, String operationPrefix,
       String completedOperationPrefix, OperationState state) {
     Metrics metrics = MetricsFactory.getInstance();
     if (metrics != null) {
@@ -428,7 +421,7 @@ public abstract class Operation {
         stateScope = null;
       }
       if (scopeStates.contains(state)) {
-        stateScope = metrics.createScope(operationPrefix + state);
+        stateScope = metrics.createScope(MetricsConstant.API_PREFIX + operationPrefix + state);
       }
       if (terminalStates.contains(state)) {
         metrics.incrementCounter(completedOperationPrefix + state);

http://git-wip-us.apache.org/repos/asf/hive/blob/3c230a62/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
index a652756..668b4b7 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -107,6 +107,7 @@ public class SQLOperation extends ExecuteStatementOperation {
    */
   private static Map<String, AtomicInteger> userQueries = new HashMap<String, AtomicInteger>();
   private static final String ACTIVE_SQL_USER = MetricsConstant.SQL_OPERATION_PREFIX + "active_user";
+  private MetricsScope submittedQryScp;
 
   public SQLOperation(HiveSession parentSession, String statement, Map<String, String> confOverlay,
       boolean runInBackground, long queryTimeout) {
@@ -126,6 +127,11 @@ public class SQLOperation extends ExecuteStatementOperation {
     } catch (HiveSQLException e) {
       LOG.warn("Error calcluating SQL Operation Display for webui", e);
     }
+
+    Metrics metrics = MetricsFactory.getInstance();
+    if (metrics != null) {
+      submittedQryScp = metrics.createScope(MetricsConstant.HS2_SUBMITTED_QURIES);
+    }
   }
 
   @Override
@@ -623,29 +629,30 @@ public class SQLOperation extends ExecuteStatementOperation {
 
   @Override
   protected void onNewState(OperationState state, OperationState prevState) {
+
     super.onNewState(state, prevState);
-    currentSQLStateScope = setMetrics(currentSQLStateScope, MetricsConstant.SQL_OPERATION_PREFIX,
-      MetricsConstant.COMPLETED_SQL_OPERATION_PREFIX, state);
+    currentSQLStateScope = updateOperationStateMetrics(currentSQLStateScope,
+        MetricsConstant.SQL_OPERATION_PREFIX,
+        MetricsConstant.COMPLETED_SQL_OPERATION_PREFIX, state);
 
     Metrics metrics = MetricsFactory.getInstance();
     if (metrics != null) {
-      try {
-        // New state is changed to running from something else (user is active)
-        if (state == OperationState.RUNNING && prevState != state) {
-          incrementUserQueries(metrics);
-        }
-        // New state is not running (user not active) any more
-        if (prevState == OperationState.RUNNING && prevState != state) {
-          decrementUserQueries(metrics);
-        }
-      } catch (IOException e) {
-        LOG.warn("Error metrics", e);
+      // New state is changed to running from something else (user is active)
+      if (state == OperationState.RUNNING && prevState != state) {
+        incrementUserQueries(metrics);
+      }
+      // New state is not running (user not active) any more
+      if (prevState == OperationState.RUNNING && prevState != state) {
+        decrementUserQueries(metrics);
       }
     }
 
     if (state == OperationState.FINISHED || state == OperationState.CANCELED || state == OperationState.ERROR) {
       //update runtime
       sqlOpDisplay.setRuntime(getOperationComplete() - getOperationStart());
+      if (metrics != null && submittedQryScp != null) {
+        metrics.endScope(submittedQryScp);
+      }
     }
 
     if (state == OperationState.CLOSED) {
@@ -654,9 +661,16 @@ public class SQLOperation extends ExecuteStatementOperation {
       //CLOSED state not interesting, state before (FINISHED, ERROR) is.
       sqlOpDisplay.updateState(state);
     }
+
+    if (state == OperationState.ERROR) {
+      markQueryMetric(MetricsFactory.getInstance(), MetricsConstant.HS2_FAILED_QUERIES);
+    }
+    if (state == OperationState.FINISHED) {
+      markQueryMetric(MetricsFactory.getInstance(), MetricsConstant.HS2_SUCEEDED_QUERIES);
+    }
   }
 
-  private void incrementUserQueries(Metrics metrics) throws IOException {
+  private void incrementUserQueries(Metrics metrics) {
     String username = parentSession.getUserName();
     if (username != null) {
       synchronized (userQueries) {
@@ -675,7 +689,7 @@ public class SQLOperation extends ExecuteStatementOperation {
     }
   }
 
-  private void decrementUserQueries(Metrics metrics) throws IOException {
+  private void decrementUserQueries(Metrics metrics) {
     String username = parentSession.getUserName();
     if (username != null) {
       synchronized (userQueries) {
@@ -688,6 +702,12 @@ public class SQLOperation extends ExecuteStatementOperation {
     }
   }
 
+  private void markQueryMetric(Metrics metric, String name) {
+    if(metric != null) {
+      metric.markMeter(name);
+    }
+  }
+
   public String getExecutionEngine() {
     return queryState.getConf().getVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/3c230a62/service/src/test/org/apache/hive/service/cli/operation/TestSQLOperationMetrics.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/cli/operation/TestSQLOperationMetrics.java b/service/src/test/org/apache/hive/service/cli/operation/TestSQLOperationMetrics.java
new file mode 100644
index 0000000..5a648c0
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/cli/operation/TestSQLOperationMetrics.java
@@ -0,0 +1,156 @@
+/**
+ * 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.hive.service.cli.operation;
+
+import com.google.common.collect.Maps;
+import org.apache.hadoop.hive.common.metrics.MetricsTestUtils;
+import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
+import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
+import org.apache.hadoop.hive.common.metrics.metrics2.CodahaleMetrics;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hive.service.cli.OperationState;
+import org.apache.hive.service.cli.session.HiveSession;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * TestSQLOperationMetrics
+ */
+public class TestSQLOperationMetrics {
+
+  private SQLOperation operation;
+  private CodahaleMetrics metrics;
+
+  @Before
+  public void setup() throws Exception {
+    HiveConf conf = new HiveConf();
+    conf.setBoolVar(HiveConf.ConfVars.HIVE_SERVER2_METRICS_ENABLED, true);
+    MetricsFactory.init(conf);
+
+    HiveSession session = mock(HiveSession.class);
+    when(session.getHiveConf()).thenReturn(conf);
+    when(session.getSessionState()).thenReturn(mock(SessionState.class));
+    when(session.getUserName()).thenReturn("userName");
+
+    operation = new SQLOperation(session, "select * from dummy",
+        Maps.<String, String>newHashMap(), false, 0L);
+
+    metrics = (CodahaleMetrics) MetricsFactory.getInstance();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    MetricsFactory.getInstance().close();
+  }
+
+  @Test
+  public void testSubmittedQueryCount() throws Exception {
+    String json = ((CodahaleMetrics) metrics).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.TIMER,
+        MetricsConstant.HS2_SUBMITTED_QURIES, "0");
+
+    operation.onNewState(OperationState.FINISHED, OperationState.RUNNING);
+
+    json = ((CodahaleMetrics) metrics).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.TIMER,
+        MetricsConstant.HS2_SUBMITTED_QURIES, "1");
+  }
+
+  @Test
+  public void testActiveUserQueriesCount() throws Exception {
+    String name = MetricsConstant.SQL_OPERATION_PREFIX + "active_user";
+    String json = ((CodahaleMetrics) metrics).dumpJson();
+
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.COUNTER, name, "");
+
+    operation.onNewState(OperationState.RUNNING, OperationState.INITIALIZED);
+    json = ((CodahaleMetrics) metrics).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.COUNTER, name, "1");
+
+    operation.onNewState(OperationState.RUNNING, OperationState.RUNNING);
+    json = ((CodahaleMetrics) metrics).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.COUNTER, name, "1");
+
+    operation.onNewState(OperationState.FINISHED, OperationState.RUNNING);
+    json = ((CodahaleMetrics) metrics).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.COUNTER, name, "0");
+  }
+
+  @Test
+  public void testSucceededQueriesCount() throws Exception {
+    String json = ((CodahaleMetrics) metrics).dumpJson();
+
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER,
+        MetricsConstant.HS2_SUCEEDED_QUERIES, "");
+
+    operation.onNewState(OperationState.FINISHED, OperationState.RUNNING);
+    json = ((CodahaleMetrics) metrics).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER,
+        MetricsConstant.HS2_SUCEEDED_QUERIES, "1");
+
+    operation.onNewState(OperationState.ERROR, OperationState.RUNNING);
+    json = ((CodahaleMetrics) metrics).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER,
+        MetricsConstant.HS2_SUCEEDED_QUERIES, "1");
+
+    operation.onNewState(OperationState.CANCELED, OperationState.RUNNING);
+    json = ((CodahaleMetrics) metrics).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER,
+        MetricsConstant.HS2_SUCEEDED_QUERIES, "1");
+
+    operation.onNewState(OperationState.FINISHED, OperationState.RUNNING);
+    json = ((CodahaleMetrics) metrics).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER,
+        MetricsConstant.HS2_SUCEEDED_QUERIES, "2");
+  }
+
+  @Test
+  public void testFailedQueriesCount() throws Exception {
+    String json = ((CodahaleMetrics) metrics).dumpJson();
+
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER,
+        MetricsConstant.HS2_FAILED_QUERIES, "");
+
+    operation.onNewState(OperationState.ERROR, OperationState.RUNNING);
+    json = ((CodahaleMetrics) metrics).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER,
+        MetricsConstant.HS2_FAILED_QUERIES, "1");
+
+    operation.onNewState(OperationState.FINISHED, OperationState.RUNNING);
+    json = ((CodahaleMetrics) metrics).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER,
+        MetricsConstant.HS2_FAILED_QUERIES, "1");
+
+    operation.onNewState(OperationState.CANCELED, OperationState.RUNNING);
+    json = ((CodahaleMetrics) metrics).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER,
+        MetricsConstant.HS2_FAILED_QUERIES, "1");
+
+    operation.onNewState(OperationState.ERROR, OperationState.RUNNING);
+    json = ((CodahaleMetrics) metrics).dumpJson();
+    MetricsTestUtils.verifyMetricsJson(json, MetricsTestUtils.METER,
+        MetricsConstant.HS2_FAILED_QUERIES, "2");
+  }
+
+
+}