You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by am...@apache.org on 2015/03/09 07:49:07 UTC

[2/2] incubator-lens git commit: LENS-283 : Add query level metrics instrumentation, especially around estimate api (amareshwari)

LENS-283 : Add query level metrics instrumentation, especially around estimate api (amareshwari)


Project: http://git-wip-us.apache.org/repos/asf/incubator-lens/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-lens/commit/683516c9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-lens/tree/683516c9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-lens/diff/683516c9

Branch: refs/heads/master
Commit: 683516c9555ada888de375de26490be9a9c2da02
Parents: 0393d7b
Author: Amareshwari Sriramadasu <am...@apache.org>
Authored: Mon Mar 9 12:18:47 2015 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Mon Mar 9 12:18:47 2015 +0530

----------------------------------------------------------------------
 .../lens/cube/parse/CubeQueryRewriter.java      |  14 ++
 .../apache/lens/driver/cube/RewriteUtil.java    |  15 +-
 .../lens/cube/parse/TestQueryMetrics.java       |  68 +++++++
 .../lens/driver/cube/TestMinCostSelector.java   |   1 +
 .../apache/lens/driver/cube/TestRewriting.java  |  13 ++
 .../apache/lens/driver/hive/TestHiveDriver.java |   1 +
 .../org/apache/lens/driver/jdbc/JDBCDriver.java |  14 ++
 .../apache/lens/driver/jdbc/TestJdbcDriver.java |  26 +++
 lens-server-api/pom.xml                         |   8 +
 .../lens/server/api/LensConfConstants.java      |  24 +++
 .../server/api/metrics/LensMetricsRegistry.java |  51 +++++
 .../server/api/metrics/MethodMetricGauge.java   |  71 +++++++
 .../lens/server/api/metrics/MethodMetrics.java  |  87 +++++++++
 .../api/metrics/MethodMetricsFactory.java       | 189 +++++++++++++++++++
 .../server/api/query/AbstractQueryContext.java  |  13 +-
 .../api/query/DriverSelectorQueryContext.java   |   7 +
 .../lens/server/model/LensContainerRequest.java |  82 ++++++++
 .../lens/server/model/LensResourceMethod.java   |  92 +++++++++
 .../server/api/driver/MockQueryContext.java     |  34 ----
 .../api/metrics/TestMethodMetricsFactory.java   |  65 +++++++
 .../lens/server/api/query/MockQueryContext.java |  36 ++++
 .../api/query/TestAbstractQueryContext.java     |  91 +++++++++
 .../lens/server/metrics/MethodMetrics.java      |  88 ---------
 .../server/metrics/MethodMetricsFactory.java    | 158 ----------------
 .../lens/server/metrics/MetricsServiceImpl.java |   4 +-
 .../lens/server/model/LensContainerRequest.java |  81 --------
 .../lens/server/model/LensResourceMethod.java   |  92 ---------
 .../server/query/QueryExecutionServiceImpl.java |  15 ++
 .../src/main/resources/lenssession-default.xml  |   9 +
 .../org/apache/lens/server/LensJerseyTest.java  |   2 +
 .../server/api/metrics/LensMetricsUtil.java     |  34 ++++
 .../metrics/TestResourceMethodMetrics.java      |   1 +
 .../lens/server/query/TestQueryService.java     |  30 +++
 src/site/apt/admin/session-config.apt           |  48 ++---
 34 files changed, 1085 insertions(+), 479 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryRewriter.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryRewriter.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryRewriter.java
index ba19125..3e66157 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryRewriter.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryRewriter.java
@@ -22,6 +22,9 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.lens.server.api.metrics.MethodMetricsContext;
+import org.apache.lens.server.api.metrics.MethodMetricsFactory;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.Context;
@@ -176,9 +179,20 @@ public class CubeQueryRewriter {
     return rewrite(tree);
   }
 
+  private static final String ITER_STR = "-ITER-";
+
   private void rewrite(List<ContextRewriter> rewriters, CubeQueryContext ctx) throws SemanticException {
+    int i = 0;
     for (ContextRewriter rewriter : rewriters) {
+      /*
+       * Adding iteration number as part of gauge name since some rewriters are have more than one phase, and having
+       * iter number gives the idea which iteration the rewriter was run
+       */
+      MethodMetricsContext mgauge = MethodMetricsFactory.createMethodGauge(ctx.getHiveConf(), true,
+        rewriter.getClass().getCanonicalName() + ITER_STR + i);
       rewriter.rewriteContext(ctx);
+      mgauge.markSuccess();
+      i++;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-cube/src/main/java/org/apache/lens/driver/cube/RewriteUtil.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/driver/cube/RewriteUtil.java b/lens-cube/src/main/java/org/apache/lens/driver/cube/RewriteUtil.java
index 0e9ca25..b905f70 100644
--- a/lens-cube/src/main/java/org/apache/lens/driver/cube/RewriteUtil.java
+++ b/lens-cube/src/main/java/org/apache/lens/driver/cube/RewriteUtil.java
@@ -26,9 +26,12 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.lens.api.LensException;
+import org.apache.lens.cube.parse.CubeQueryContext;
 import org.apache.lens.cube.parse.CubeQueryRewriter;
 import org.apache.lens.cube.parse.HQLParser;
 import org.apache.lens.server.api.driver.LensDriver;
+import org.apache.lens.server.api.metrics.MethodMetricsContext;
+import org.apache.lens.server.api.metrics.MethodMetricsFactory;
 import org.apache.lens.server.api.query.AbstractQueryContext;
 
 import org.apache.hadoop.conf.Configuration;
@@ -200,6 +203,9 @@ public final class RewriteUtil {
     return query.replaceAll("[\\n\\r]", " ").replaceAll("&&", " AND ").replaceAll("\\|\\|", " OR ").trim();
   }
 
+  private static final String REWRITE_QUERY_GAUGE = RewriteUtil.class.getCanonicalName() + "-rewriteQuery";
+  private static final String TOHQL_GAUGE = RewriteUtil.class.getCanonicalName() + "-rewriteQuery-toHQL";
+
   /**
    * Rewrite query.
    *
@@ -222,6 +228,8 @@ public final class RewriteUtil {
       } else {
         List<RewriteUtil.CubeQueryInfo> cubeQueries = findCubePositions(replacedQuery);
         for (LensDriver driver : ctx.getDriverContext().getDrivers()) {
+          MethodMetricsContext rewriteGauge = MethodMetricsFactory.createMethodGauge(ctx.getDriverConf(driver), true,
+            REWRITE_QUERY_GAUGE);
           StringBuilder builder = new StringBuilder();
           int start = 0;
           try {
@@ -236,7 +244,11 @@ public final class RewriteUtil {
               if (start != cqi.startPos) {
                 builder.append(replacedQuery.substring(start, cqi.startPos));
               }
-              String hqlQuery = rewriter.rewrite(cqi.query).toHQL();
+              CubeQueryContext cqc = rewriter.rewrite(cqi.query);
+              MethodMetricsContext toHQLGauge = MethodMetricsFactory.createMethodGauge(ctx.getDriverConf(driver), true,
+                TOHQL_GAUGE);
+              String hqlQuery = cqc.toHQL();
+              toHQLGauge.markSuccess();
               LOG.debug("Rewritten query:" + hqlQuery);
               builder.append(hqlQuery);
               start = cqi.endPos;
@@ -259,6 +271,7 @@ public final class RewriteUtil {
               failureCause = e.getLocalizedMessage();
             }
           }
+          rewriteGauge.markSuccess();
         }
       }
       if (driverQueries.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-cube/src/test/java/org/apache/lens/cube/parse/TestQueryMetrics.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestQueryMetrics.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestQueryMetrics.java
new file mode 100644
index 0000000..aec0828
--- /dev/null
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestQueryMetrics.java
@@ -0,0 +1,68 @@
+/**
+ * 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.lens.cube.parse;
+
+import static org.apache.lens.cube.parse.CubeTestSetup.TWO_DAYS_RANGE;
+
+import java.util.Arrays;
+
+import org.apache.lens.server.api.LensConfConstants;
+import org.apache.lens.server.api.metrics.LensMetricsRegistry;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.codahale.metrics.MetricRegistry;
+
+public class TestQueryMetrics extends TestQueryRewrite {
+
+  @Test
+  public void testMethodGauges() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY, TestQueryMetrics.class.getSimpleName());
+    conf.set(LensConfConstants.QUERY_METRIC_DRIVER_STACK_NAME, "testCubeRewriteStackName");
+
+    rewriteCtx("cube select" + " SUM(msr2) from testCube where " + TWO_DAYS_RANGE, conf);
+    MetricRegistry reg = LensMetricsRegistry.getStaticRegistry();
+
+    Assert.assertTrue(reg.getGauges().keySet().containsAll(Arrays.asList(
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.ExpressionResolver-ITER-0",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.ColumnResolver-ITER-1",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.AliasReplacer-ITER-2",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.DenormalizationResolver-ITER-3",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.CandidateTableResolver-ITER-4",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.JoinResolver-ITER-5",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.TimerangeResolver-ITER-6",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.CandidateTableResolver-ITER-7",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.AggregateResolver-ITER-8",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.GroupbyResolver-ITER-9",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.StorageTableResolver-ITER-10",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.StorageTableResolver-ITER-11",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.LightestFactResolver-ITER-12",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.StorageTableResolver-ITER-13",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.DenormalizationResolver-ITER-14",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.LeastPartitionResolver-ITER-15",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.LightestFactResolver-ITER-16",
+      "lens.MethodMetricGauge.testCubeRewriteStackName-org.apache.lens.cube.parse.LightestDimensionResolver-ITER-17"
+    )));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-cube/src/test/java/org/apache/lens/driver/cube/TestMinCostSelector.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/driver/cube/TestMinCostSelector.java b/lens-cube/src/test/java/org/apache/lens/driver/cube/TestMinCostSelector.java
index df81054..68d61d0 100644
--- a/lens-cube/src/test/java/org/apache/lens/driver/cube/TestMinCostSelector.java
+++ b/lens-cube/src/test/java/org/apache/lens/driver/cube/TestMinCostSelector.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import org.apache.lens.api.LensConf;
 import org.apache.lens.api.LensException;
 import org.apache.lens.server.api.driver.*;
+import org.apache.lens.server.api.query.MockQueryContext;
 
 import org.apache.hadoop.conf.Configuration;
 

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-cube/src/test/java/org/apache/lens/driver/cube/TestRewriting.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/driver/cube/TestRewriting.java b/lens-cube/src/test/java/org/apache/lens/driver/cube/TestRewriting.java
index 98aeb25..6e758a6 100644
--- a/lens-cube/src/test/java/org/apache/lens/driver/cube/TestRewriting.java
+++ b/lens-cube/src/test/java/org/apache/lens/driver/cube/TestRewriting.java
@@ -21,6 +21,7 @@ package org.apache.lens.driver.cube;
 import static org.mockito.Matchers.any;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
@@ -29,8 +30,10 @@ import org.apache.lens.api.LensException;
 import org.apache.lens.cube.parse.CubeQueryContext;
 import org.apache.lens.cube.parse.CubeQueryRewriter;
 import org.apache.lens.cube.parse.HQLParser;
+import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.driver.LensDriver;
 import org.apache.lens.server.api.driver.MockDriver;
+import org.apache.lens.server.api.metrics.LensMetricsRegistry;
 import org.apache.lens.server.api.query.QueryContext;
 
 import org.apache.hadoop.conf.Configuration;
@@ -52,6 +55,8 @@ import org.testng.IObjectFactory;
 import org.testng.annotations.ObjectFactory;
 import org.testng.annotations.Test;
 
+import com.codahale.metrics.MetricRegistry;
+
 /**
  * The Class TestRewriting.
  */
@@ -167,6 +172,8 @@ public class TestRewriting {
     QueryContext ctx = new QueryContext(q1, null, lensConf, conf, drivers);
     RewriteUtil.rewriteQuery(ctx);
 
+    conf.set(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY, TestRewriting.class.getSimpleName());
+    driver.configure(conf);
     String q2 = "cube select name from table";
     Assert.assertTrue(RewriteUtil.isCubeQuery(q2));
     cubeQueries = RewriteUtil.findCubePositions(q2);
@@ -174,6 +181,12 @@ public class TestRewriting {
     Assert.assertEquals(cubeQueries.get(0).query, "cube select name from table");
     ctx = new QueryContext(q2, null, lensConf, conf, drivers);
     RewriteUtil.rewriteQuery(ctx);
+    MetricRegistry reg = LensMetricsRegistry.getStaticRegistry();
+
+    Assert.assertTrue(reg.getGauges().keySet().containsAll(Arrays.asList(
+      "lens.MethodMetricGauge.TestRewriting-MockDriver-org.apache.lens.driver.cube.RewriteUtil-rewriteQuery",
+      "lens.MethodMetricGauge.TestRewriting-MockDriver-org.apache.lens.driver.cube.RewriteUtil-rewriteQuery-toHQL")));
+    conf.unset(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY);
 
     q2 = "insert overwrite directory '/tmp/rewrite' cube select name from table";
     Assert.assertTrue(RewriteUtil.isCubeQuery(q2));

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-driver-hive/src/test/java/org/apache/lens/driver/hive/TestHiveDriver.java
----------------------------------------------------------------------
diff --git a/lens-driver-hive/src/test/java/org/apache/lens/driver/hive/TestHiveDriver.java b/lens-driver-hive/src/test/java/org/apache/lens/driver/hive/TestHiveDriver.java
index 774f43f..8a5839b 100644
--- a/lens-driver-hive/src/test/java/org/apache/lens/driver/hive/TestHiveDriver.java
+++ b/lens-driver-hive/src/test/java/org/apache/lens/driver/hive/TestHiveDriver.java
@@ -34,6 +34,7 @@ import org.apache.lens.server.api.driver.*;
 import org.apache.lens.server.api.driver.DriverQueryStatus.DriverQueryState;
 import org.apache.lens.server.api.query.AbstractQueryContext;
 import org.apache.lens.server.api.query.ExplainQueryContext;
+import org.apache.lens.server.api.query.MockQueryContext;
 import org.apache.lens.server.api.query.PreparedQueryContext;
 import org.apache.lens.server.api.query.QueryContext;
 import org.apache.lens.server.api.util.LensUtil;

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-driver-jdbc/src/main/java/org/apache/lens/driver/jdbc/JDBCDriver.java
----------------------------------------------------------------------
diff --git a/lens-driver-jdbc/src/main/java/org/apache/lens/driver/jdbc/JDBCDriver.java b/lens-driver-jdbc/src/main/java/org/apache/lens/driver/jdbc/JDBCDriver.java
index 1257092..d67e1da 100644
--- a/lens-driver-jdbc/src/main/java/org/apache/lens/driver/jdbc/JDBCDriver.java
+++ b/lens-driver-jdbc/src/main/java/org/apache/lens/driver/jdbc/JDBCDriver.java
@@ -42,6 +42,8 @@ import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.driver.*;
 import org.apache.lens.server.api.driver.DriverQueryStatus.DriverQueryState;
 import org.apache.lens.server.api.events.LensEventListener;
+import org.apache.lens.server.api.metrics.MethodMetricsContext;
+import org.apache.lens.server.api.metrics.MethodMetricsFactory;
 import org.apache.lens.server.api.query.AbstractQueryContext;
 import org.apache.lens.server.api.query.PreparedQueryContext;
 import org.apache.lens.server.api.query.QueryContext;
@@ -543,9 +545,15 @@ public class JDBCDriver implements LensDriver {
     }
   }
 
+  private static final String VALIDATE_GAUGE = "validate-thru-prepare";
+  private static final String COLUMNAR_SQL_REWRITE_GAUGE = "columnar-sql-rewrite";
+  private static final String JDBC_PREPARE_GAUGE = "jdbc-prepare-statement";
   @Override
   public QueryCost estimate(AbstractQueryContext qctx) throws LensException {
+    MethodMetricsContext validateGauge = MethodMetricsFactory.createMethodGauge(qctx.getDriverConf(this), true,
+      VALIDATE_GAUGE);
     validate(qctx);
+    validateGauge.markSuccess();
     return JDBC_DRIVER_COST;
   }
 
@@ -641,7 +649,12 @@ public class JDBCDriver implements LensDriver {
     }
     checkConfigured();
     // Only create a prepared statement and then close it
+    MethodMetricsContext sqlRewriteGauge = MethodMetricsFactory.createMethodGauge(pContext.getDriverConf(this), true,
+      COLUMNAR_SQL_REWRITE_GAUGE);
     String rewrittenQuery = rewriteQuery(pContext.getDriverQuery(this), pContext.getDriverConf(this));
+    sqlRewriteGauge.markSuccess();
+    MethodMetricsContext jdbcPrepareGauge = MethodMetricsFactory.createMethodGauge(pContext.getDriverConf(this), true,
+      JDBC_PREPARE_GAUGE);
     Connection conn = null;
     PreparedStatement stmt = null;
     try {
@@ -660,6 +673,7 @@ public class JDBCDriver implements LensDriver {
           LOG.error("Error closing connection: " + rewrittenQuery, e);
         }
       }
+      jdbcPrepareGauge.markSuccess();
     }
     LOG.info("Prepared: " + rewrittenQuery);
     return stmt;

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-driver-jdbc/src/test/java/org/apache/lens/driver/jdbc/TestJdbcDriver.java
----------------------------------------------------------------------
diff --git a/lens-driver-jdbc/src/test/java/org/apache/lens/driver/jdbc/TestJdbcDriver.java b/lens-driver-jdbc/src/test/java/org/apache/lens/driver/jdbc/TestJdbcDriver.java
index 68bd19b..6c1d77e 100644
--- a/lens-driver-jdbc/src/test/java/org/apache/lens/driver/jdbc/TestJdbcDriver.java
+++ b/lens-driver-jdbc/src/test/java/org/apache/lens/driver/jdbc/TestJdbcDriver.java
@@ -24,6 +24,7 @@ import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.Statement;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
@@ -34,8 +35,10 @@ import org.apache.lens.api.LensException;
 import org.apache.lens.api.query.QueryCost;
 import org.apache.lens.api.query.QueryHandle;
 import org.apache.lens.api.query.ResultRow;
+import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.driver.*;
 import org.apache.lens.server.api.driver.DriverQueryStatus.DriverQueryState;
+import org.apache.lens.server.api.metrics.LensMetricsRegistry;
 import org.apache.lens.server.api.query.ExplainQueryContext;
 import org.apache.lens.server.api.query.PreparedQueryContext;
 import org.apache.lens.server.api.query.QueryContext;
@@ -52,6 +55,8 @@ import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.BeforeTest;
 import org.testng.annotations.Test;
 
+import com.codahale.metrics.MetricRegistry;
+
 /**
  * The Class TestJdbcDriver.
  */
@@ -254,6 +259,27 @@ public class TestJdbcDriver {
   }
 
   /**
+   * Test estimate failing
+   *
+   * @throws Exception the exception
+   */
+  @Test
+  public void testEstimateGauges() throws Exception {
+    createTable("estimate_test_gauge"); // Create table
+    insertData("estimate_test_gauge"); // Insert some data into table
+    String query1 = "SELECT * FROM estimate_test_gauge"; // Select query against existing table
+    Configuration metricConf = new Configuration(baseConf);
+    metricConf.set(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY, TestJdbcDriver.class.getSimpleName());
+    driver.estimate(createExplainContext(query1, metricConf));
+    MetricRegistry reg = LensMetricsRegistry.getStaticRegistry();
+
+    Assert.assertTrue(reg.getGauges().keySet().containsAll(Arrays.asList(
+      "lens.MethodMetricGauge.TestJdbcDriver-JDBCDriver-columnar-sql-rewrite",
+      "lens.MethodMetricGauge.TestJdbcDriver-JDBCDriver-jdbc-prepare-statement",
+      "lens.MethodMetricGauge.TestJdbcDriver-JDBCDriver-validate-thru-prepare")));
+  }
+
+  /**
    * Test explain.
    *
    * @throws Exception the exception

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server-api/pom.xml
----------------------------------------------------------------------
diff --git a/lens-server-api/pom.xml b/lens-server-api/pom.xml
index 8472840..aa2ef59 100644
--- a/lens-server-api/pom.xml
+++ b/lens-server-api/pom.xml
@@ -59,5 +59,13 @@
       <groupId>org.glassfish.jersey.core</groupId>
       <artifactId>jersey-server</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.media</groupId>
+      <artifactId>jersey-media-multipart</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.codahale.metrics</groupId>
+      <artifactId>metrics-core</artifactId>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server-api/src/main/java/org/apache/lens/server/api/LensConfConstants.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/LensConfConstants.java b/lens-server-api/src/main/java/org/apache/lens/server/api/LensConfConstants.java
index ff25fa2..0bf78f0 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/LensConfConstants.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/LensConfConstants.java
@@ -823,4 +823,28 @@ public final class LensConfConstants {
    * Default value of top level dir for database specific resources
    */
   public static final String DEFAULT_DATABASE_RESOURCE_DIR = "/tmp/lens/resources";
+
+  /**
+   * Key for enabling metrics for each query to be different
+   */
+  public static final String ENABLE_QUERY_METRICS = QUERY_PFX + "enable.metrics.per.query";
+
+  /**
+   * Default value for query wise metrics
+   */
+  public static final boolean DEFAULT_ENABLE_QUERY_METRICS = false;
+
+  /**
+   * Key used to hold value of unique id for query metrics. This wont be passed by user, will be generated and set.
+   * This is to pass unique id for query across the code flow.
+   */
+  public static final String QUERY_METRIC_UNIQUE_ID_CONF_KEY = QUERY_PFX + "metric.unique.id";
+
+  /**
+   * Key used to hold value query metric name in the stack. This wont be passed by user, will be generated and set.
+   * When each query looked at by driver, the metric needs to be different for each driver. This name capture the stack
+   * from which driver the code reached there.
+   */
+  public static final String QUERY_METRIC_DRIVER_STACK_NAME = QUERY_PFX + "metric.driver.stack.name";
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/LensMetricsRegistry.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/LensMetricsRegistry.java b/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/LensMetricsRegistry.java
new file mode 100644
index 0000000..0427658
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/LensMetricsRegistry.java
@@ -0,0 +1,51 @@
+/**
+ * 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.lens.server.api.metrics;
+
+import com.codahale.metrics.MetricRegistry;
+
+/**
+ * Lens MetricsRegistry.
+ */
+public class LensMetricsRegistry {
+  private LensMetricsRegistry() {
+  }
+
+  private static MetricRegistry metricRegistry;
+
+  /**
+   * Static instance of metrics registry. This instance is registered with reporters. For all the use of publishing
+   * metrics to reporter, this should be used.
+   *
+   * @return LensMetricsRegistry
+   */
+  public static synchronized MetricRegistry getStaticRegistry() {
+    if (metricRegistry == null) {
+      metricRegistry = new MetricRegistry();
+    }
+    return metricRegistry;
+  }
+
+  /**
+   * This clears the registry, would called on server stop, included only for restart tests.
+   */
+  static synchronized void clearRegistry() {
+    metricRegistry = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MethodMetricGauge.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MethodMetricGauge.java b/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MethodMetricGauge.java
new file mode 100644
index 0000000..73bacad
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MethodMetricGauge.java
@@ -0,0 +1,71 @@
+/**
+ * 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.lens.server.api.metrics;
+
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricRegistry;
+
+import lombok.NonNull;
+
+/**
+ * Holds the gauge value indicating the time taken for the method.
+ *
+ * When we have methods which can take variable time with respect to the parameters passed, the timers available in
+ * {@link MethodMetrics} (which aggregated values over all calls) does not provide information on how each call
+ * performed. Having gauge for each call will solve the purpose, which resulted in this class.
+ *
+ * The gauge added here should be created with unique name for each call so that the gauges are not lost to
+ * the latest calls.
+ */
+public class MethodMetricGauge implements MethodMetricsContext {
+  private final long startTime;
+  private long totalTime;
+  private final String gaugeName;
+  private final MetricRegistry metricRegistry;
+
+  /**
+   * The gauge for method time.
+   *
+   * @param metricRegistry The metric registry
+   * @param gaugeName Gauge name.
+   *  It should be unique for each creation. Callers have to take care of passing unique name
+   */
+  public MethodMetricGauge(@NonNull MetricRegistry metricRegistry, @NonNull String gaugeName) {
+    this.startTime = System.nanoTime();
+    this.gaugeName = gaugeName;
+    this.metricRegistry = metricRegistry;
+  }
+
+  @Override
+  public void markError() {
+  }
+
+  @Override
+  public void markSuccess() {
+    this.totalTime = System.nanoTime() - startTime;
+    metricRegistry.register(MetricRegistry.name("lens", MethodMetricGauge.class.getSimpleName(), gaugeName),
+      new Gauge<Long>() {
+        @Override
+        public Long getValue() {
+          return totalTime;
+        }
+      });
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MethodMetrics.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MethodMetrics.java b/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MethodMetrics.java
new file mode 100644
index 0000000..2374ae8
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MethodMetrics.java
@@ -0,0 +1,87 @@
+/**
+ * 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.lens.server.api.metrics;
+
+
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.Timer;
+import lombok.NonNull;
+
+/**
+ * metrics measuring for resource methods. Every resource method has one timer and two meters associated with it. One
+ * meter is marking number of invocations of the method, another is marking number of invocations where exception
+ * occurred.
+ */
+
+public class MethodMetrics {
+  // Corresponding meter and timer counts are same, but they are marked at different times.
+  // meter is marked at the start of event, timer is marked at the end of the event.
+  private final Meter meter;
+  private final Timer successTimer;
+  private final Timer exceptionTimer;
+
+  public MethodMetrics(
+    @NonNull Meter meter, @NonNull Timer successTimer, @NonNull Timer exceptionTimer) {
+    this.meter = meter;
+    this.successTimer = successTimer;
+    this.exceptionTimer = exceptionTimer;
+  }
+
+  /**
+   * Inner class instances handle contexts for the same instance of MethodMetrics. This is useful since one method can
+   * be executing multiple times in parallel. But the contexts will be different.
+   */
+  public class Context implements MethodMetricsContext {
+    private Timer.Context successTimerContext;
+    private Timer.Context exceptionTimerContext;
+
+    private Context() {
+      meter.mark();
+      successTimerContext = successTimer.time();
+      exceptionTimerContext = exceptionTimer.time();
+    }
+
+    @Override
+    public void markError() {
+      exceptionTimerContext.close();
+    }
+
+    @Override
+    public void markSuccess() {
+      successTimerContext.close();
+    }
+  }
+
+  /** Create and return new context since an execution of this method has started */
+  public MethodMetricsContext newContext() {
+    return new Context();
+  }
+
+  public long getCount() {
+    return meter.getCount();
+  }
+
+  public long getSuccessCount() {
+    return successTimer.getCount();
+  }
+
+  public long getErrorCount() {
+    return exceptionTimer.getCount();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MethodMetricsFactory.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MethodMetricsFactory.java b/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MethodMetricsFactory.java
new file mode 100644
index 0000000..3227d20
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/metrics/MethodMetricsFactory.java
@@ -0,0 +1,189 @@
+/**
+ * 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.lens.server.api.metrics;
+
+
+import static com.codahale.metrics.MetricRegistry.name;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lens.server.api.LensConfConstants;
+import org.apache.lens.server.model.LensContainerRequest;
+import org.apache.lens.server.model.LensResourceMethod;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.log4j.Logger;
+
+import org.glassfish.jersey.server.ContainerRequest;
+import org.glassfish.jersey.server.model.ResourceMethod;
+
+import com.codahale.metrics.MetricRegistry;
+import com.google.common.base.Optional;
+import com.google.common.base.Supplier;
+import lombok.Getter;
+import lombok.NonNull;
+
+/**
+ * Factory for creating MethodMetrics objects on demand.
+ */
+public class MethodMetricsFactory {
+  public static final Logger LOG = Logger.getLogger(MethodMetricsFactory.class);
+
+  private final MetricRegistry metricRegistry;
+
+  public MethodMetricsFactory(@NonNull MetricRegistry metricRegistry) {
+    this.metricRegistry = metricRegistry;
+  }
+
+  /** factory helper */
+  @Getter
+  private volatile Map<String, MethodMetrics> methodMetricsMap = new HashMap<String, MethodMetrics>();
+
+  /**
+   * This is a factory method for getting a MethodMetrics instance. First a unique name is determined for the arguments
+   * and then function with same name is called with that unique name.
+   *
+   * @param method
+   * @param containerRequest
+   * @return
+   * @see #get(String)
+   * @see #getUniqueName(org.glassfish.jersey.server.model.ResourceMethod, org.glassfish.jersey.server.ContainerRequest)
+   */
+  public MethodMetrics get(@NonNull ResourceMethod method, @NonNull ContainerRequest containerRequest) {
+    return get(getUniqueName(method, containerRequest));
+  }
+
+  /**
+   * Returns MethodMetrics object corresponding to the given name. If doesn't exist yet, one will be created.
+   *
+   * @param name
+   * @return
+   */
+  public MethodMetrics get(@NonNull final String name) {
+    //SUSPEND CHECKSTYLE CHECK DoubleCheckedLockingCheck
+    MethodMetrics result = methodMetricsMap.get(name);
+    if (result == null) {
+      synchronized (this) {
+        result = methodMetricsMap.get(name);
+        LOG.info("Creating MethodMetrics of name: " + name);
+        result = new MethodMetrics(
+          metricRegistry.meter(name(name, "meter")),
+          metricRegistry.timer(name(name, "timer")),
+          metricRegistry.timer(name(name, "exception.timer")));
+        methodMetricsMap.put(name, result);
+      }
+    }
+    //RESUME CHECKSTYLE CHECK DoubleCheckedLockingCheck
+    return result;
+  }
+
+  /**
+   * Wrapper method. Creates Lens model wrappers: objects of {@link org.apache.lens.server.model.LensResourceMethod} and
+   * {@link org.apache.lens.server.model.LensContainerRequest}. And calls function of same name with those objects as
+   * arguments.
+   *
+   * @param method
+   * @param containerRequest
+   * @return unique name of MethodMetrics object to be returned by #get
+   * @see #getUniqueName
+   * @see #get(org.glassfish.jersey.server.model.ResourceMethod, org.glassfish.jersey.server.ContainerRequest)
+   */
+  private String getUniqueName(ResourceMethod method, ContainerRequest containerRequest) {
+    return getUniqueName(new LensResourceMethod(method), new LensContainerRequest(containerRequest));
+  }
+
+  /**
+   * Extracts base name from lensResourceMethod#name. Checks if the resource method was annotated with {@link
+   * org.apache.lens.server.api.annotations.MultiPurposeResource}. If not, returns the base name If yes, extracts the
+   * multi purpose form param from the resource method, gets the value of that param from lensContainerRequest object
+   * and appends that to baseName. if value is not passed, gets the default value of that argument from
+   * lensContainerRequest and appends that to baseName. Returns the final string constructed
+   *
+   * @param lensResourceMethod
+   * @param lensContainerRequest
+   * @return Unique name of the MethodMetrics object associated with given arguments.
+   * @see org.apache.lens.server.model.LensResourceMethod#name()
+   * @see org.apache.lens.server.api.annotations.MultiPurposeResource
+   */
+  private String getUniqueName(final LensResourceMethod lensResourceMethod, LensContainerRequest lensContainerRequest) {
+    StringBuilder sb = new StringBuilder();
+    sb.append(lensResourceMethod.name());
+    final Optional<String> multiPurposeFormParam = lensResourceMethod.getMultiPurposeFormParam();
+    if (multiPurposeFormParam.isPresent()) {
+      sb.append(".");
+      String value = lensContainerRequest.getFormDataFieldValue(multiPurposeFormParam.get()).or(
+        new Supplier<String>() {
+          @Override
+          public String get() {
+            return lensResourceMethod.getDefaultValueForParam(multiPurposeFormParam.get());
+          }
+        });
+      sb.append(value.toUpperCase());
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Remove all meters/timers/... created when MethodMetrics objects were constructed using this factory.
+   */
+  public void clear() {
+    synchronized (this) {
+      LOG.info("clearing factory");
+      for (Map.Entry<String, MethodMetrics> entry : methodMetricsMap.entrySet()) {
+        metricRegistry.remove(name(entry.getKey(), "meter"));
+        metricRegistry.remove(name(entry.getKey(), "timer"));
+        metricRegistry.remove(name(entry.getKey(), "exception.timer"));
+      }
+      methodMetricsMap.clear();
+    }
+  }
+
+  /**
+   * Get query metric gauge name.
+   *
+   * @param conf
+   * @param appendToStackName
+   * @param gaugeSuffix
+   * @return
+   */
+  public static MethodMetricsContext createMethodGauge(@NonNull Configuration conf, boolean appendToStackName,
+    String gaugeSuffix) {
+    String uid = conf.get(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY);
+    if (StringUtils.isBlank(uid)) {
+      return DisabledMethodMetricsContext.getInstance();
+    }
+    LOG.info("query metricid:" + uid);
+    StringBuilder metricName = new StringBuilder();
+    if (appendToStackName) {
+      String stackName = conf.get(LensConfConstants.QUERY_METRIC_DRIVER_STACK_NAME);
+      LOG.info("query metric stackname:" + stackName);
+      metricName.append(stackName);
+      metricName.append("-");
+    } else {
+      metricName.append(uid);
+      metricName.append("-");
+    }
+    metricName.append(gaugeSuffix);
+    String metricGaugeName = metricName.toString();
+    MethodMetricGauge mg = new MethodMetricGauge(LensMetricsRegistry.getStaticRegistry(), metricGaugeName);
+    return mg;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server-api/src/main/java/org/apache/lens/server/api/query/AbstractQueryContext.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/AbstractQueryContext.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/AbstractQueryContext.java
index f141428..432c863 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/AbstractQueryContext.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/AbstractQueryContext.java
@@ -21,12 +21,16 @@ package org.apache.lens.server.api.query;
 import java.io.Serializable;
 import java.util.Collection;
 import java.util.Map;
+import java.util.UUID;
 
 import org.apache.lens.api.LensConf;
 import org.apache.lens.api.LensException;
 import org.apache.lens.api.query.QueryCost;
+import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.driver.DriverQueryPlan;
 import org.apache.lens.server.api.driver.LensDriver;
+import org.apache.lens.server.api.metrics.MethodMetricsContext;
+import org.apache.lens.server.api.metrics.MethodMetricsFactory;
 import org.apache.lens.server.api.query.DriverSelectorQueryContext.DriverQueryContext;
 import org.apache.lens.server.api.util.LensUtil;
 
@@ -104,6 +108,11 @@ public abstract class AbstractQueryContext implements Serializable {
 
   protected AbstractQueryContext(final String query, final String user, final LensConf qconf, final Configuration conf,
     final Collection<LensDriver> drivers) {
+    if (conf.getBoolean(LensConfConstants.ENABLE_QUERY_METRICS, LensConfConstants.DEFAULT_ENABLE_QUERY_METRICS)) {
+      UUID metricId = UUID.randomUUID();
+      conf.set(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY, metricId.toString());
+      LOG.info("Generated metric id: " + metricId + " for query: " + query);
+    }
     driverContext = new DriverSelectorQueryContext(query, conf, drivers);
     userQuery = query;
     this.lensConf = qconf;
@@ -140,6 +149,8 @@ public abstract class AbstractQueryContext implements Serializable {
     boolean succeededOnAtleastOneDriver = false;
     for (LensDriver driver : driverContext.getDrivers()) {
       final DriverQueryContext driverQueryContext = driverContext.driverQueryContextMap.get(driver);
+      MethodMetricsContext estimateGauge = MethodMetricsFactory.createMethodGauge(getDriverConf(driver), true,
+        "driverEstimate");
       if (driverQueryContext.getDriverQueryRewriteError() != null) {
         // skip estimate
         continue;
@@ -160,6 +171,7 @@ public abstract class AbstractQueryContext implements Serializable {
           failureCause = expMsg;
         }
       }
+      estimateGauge.markSuccess();
     }
     if (!succeededOnAtleastOneDriver) {
       throw new LensException(useBuilder ? detailedFailureCause.toString() : failureCause);
@@ -305,5 +317,4 @@ public abstract class AbstractQueryContext implements Serializable {
   public Exception getDriverRewriteError(LensDriver driver) {
     return driverContext.driverQueryContextMap.get(driver).getDriverQueryRewriteError();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server-api/src/main/java/org/apache/lens/server/api/query/DriverSelectorQueryContext.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/DriverSelectorQueryContext.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/DriverSelectorQueryContext.java
index 1c04952..94b5daf 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/DriverSelectorQueryContext.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/DriverSelectorQueryContext.java
@@ -22,10 +22,12 @@ import java.util.*;
 
 import org.apache.lens.api.LensException;
 import org.apache.lens.api.query.QueryCost;
+import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.driver.DriverQueryPlan;
 import org.apache.lens.server.api.driver.LensDriver;
 import org.apache.lens.server.api.util.LensUtil;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -61,6 +63,11 @@ public class DriverSelectorQueryContext {
     for (LensDriver driver : drivers) {
       DriverQueryContext ctx = new DriverQueryContext(driver);
       ctx.setDriverSpecificConf(mergeConf(driver, queryConf));
+      String metricId = ctx.driverSpecificConf.get(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY);
+      if (!StringUtils.isBlank(metricId)) {
+        ctx.driverSpecificConf.set(LensConfConstants.QUERY_METRIC_DRIVER_STACK_NAME,
+          metricId + "-" + driver.getClass().getSimpleName());
+      }
       ctx.setQuery(userQuery);
       driverQueryContextMap.put(driver, ctx);
     }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server-api/src/main/java/org/apache/lens/server/model/LensContainerRequest.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/model/LensContainerRequest.java b/lens-server-api/src/main/java/org/apache/lens/server/model/LensContainerRequest.java
new file mode 100644
index 0000000..fc1da23
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/model/LensContainerRequest.java
@@ -0,0 +1,82 @@
+/**
+ * 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.lens.server.model;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.glassfish.jersey.media.multipart.FormDataBodyPart;
+import org.glassfish.jersey.media.multipart.FormDataMultiPart;
+import org.glassfish.jersey.message.internal.ReaderWriter;
+import org.glassfish.jersey.server.ContainerRequest;
+
+import com.google.common.base.Optional;
+import lombok.AllArgsConstructor;
+import lombok.NonNull;
+
+/**
+ * Wrapper class for ContainerRequest.
+ *
+ * Has helper methods to give form data values for the request. More helper methods can be included.
+ */
+@AllArgsConstructor
+public class LensContainerRequest {
+  @NonNull
+  private ContainerRequest containerRequest;
+
+  /**
+   * parses form data as multipart form data and extracts the value of given fieldName from it.
+   *
+   * @param fieldName
+   * @return Optional value of field passed as multipart post data
+   */
+  public Optional<String> getFormDataFieldValue(final String fieldName) {
+    FormDataBodyPart field = getFormData(FormDataMultiPart.class).getField(fieldName);
+    return field == null ? Optional.<String>absent() : Optional.of(field.getValue());
+  }
+
+  /**
+   * Utility method for reading form/multipart-form data from container request.
+   *
+   * @param clz Either Form.class or FormDataMultiPart.class
+   * @param <T> clz type
+   * @return an instance of T
+   */
+  public <T> T getFormData(Class<T> clz) {
+
+    InputStream in = containerRequest.getEntityStream();
+    if (in.getClass() != ByteArrayInputStream.class) {
+      // Buffer input
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      try {
+        ReaderWriter.writeTo(in, baos);
+      } catch (IOException e) {
+        throw new IllegalArgumentException(e);
+      }
+      in = new ByteArrayInputStream(baos.toByteArray());
+      containerRequest.setEntityStream(in);
+    }
+    ByteArrayInputStream bais = (ByteArrayInputStream) in;
+    T f = containerRequest.readEntity(clz);
+    bais.reset();
+    return f;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server-api/src/main/java/org/apache/lens/server/model/LensResourceMethod.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/model/LensResourceMethod.java b/lens-server-api/src/main/java/org/apache/lens/server/model/LensResourceMethod.java
new file mode 100644
index 0000000..a3dca46
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/model/LensResourceMethod.java
@@ -0,0 +1,92 @@
+/**
+ * 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.lens.server.model;
+
+import java.lang.annotation.Annotation;
+
+import org.apache.lens.server.api.annotations.MultiPurposeResource;
+
+import org.glassfish.jersey.server.model.Parameter;
+import org.glassfish.jersey.server.model.ResourceMethod;
+
+import com.google.common.base.Optional;
+import lombok.AllArgsConstructor;
+import lombok.NonNull;
+
+/**
+ * Model object wrapping ResourceMethod. Adds utility methods.
+ */
+@AllArgsConstructor
+public class LensResourceMethod {
+  @NonNull
+  private ResourceMethod resourceMethod;
+
+  /**
+   * if handler method is annotated with {@link org.apache.lens.server.api.annotations.MultiPurposeResource}, extract
+   * that and return {@link org.apache.lens.server.api.annotations.MultiPurposeResource#formParamName()}
+   *
+   * @return
+   */
+  public Optional<String> getMultiPurposeFormParam() {
+    return getHandlerAnnotation(MultiPurposeResource.class).isPresent()
+      ? Optional.of(getHandlerAnnotation(MultiPurposeResource.class).get().formParamName()) : Optional.<String>absent();
+  }
+
+  /**
+   * className.methodname.httpmethod
+   *
+   * @return qualified name of the handler method
+   */
+  public String name() {
+    return new StringBuilder()
+      .append(resourceMethod.getInvocable().getHandlingMethod().getDeclaringClass().getCanonicalName())
+      .append(".")
+      .append(resourceMethod.getInvocable().getHandlingMethod().getName())
+      .append(".")
+      .append(resourceMethod.getHttpMethod())
+      .toString();
+  }
+
+  /**
+   * Get annotation of class clz on the handler method
+   *
+   * @param <T>
+   * @param clz annotation class
+   * @return
+   */
+  private <T extends Annotation> Optional<T> getHandlerAnnotation(Class<T> clz) {
+    return Optional.fromNullable(resourceMethod.getInvocable().getHandlingMethod().getAnnotation(clz));
+  }
+
+  /**
+   * Extract default value of a parameter from the parameter annotations of the handler method
+   *
+   * @param paramName value of {@link org.glassfish.jersey.media.multipart.FormDataParam} annotation on the handler
+   *                  method. Which will be the name of the argument in the request parameters
+   * @return value of {@link javax.ws.rs.DefaultValue} annotation on the handler method
+   */
+  public String getDefaultValueForParam(final String paramName) {
+    for (Parameter param : resourceMethod.getInvocable().getParameters()) {
+      if (param.getSourceName().equals(paramName)) {
+        return param.getDefaultValue();
+      }
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server-api/src/test/java/org/apache/lens/server/api/driver/MockQueryContext.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/test/java/org/apache/lens/server/api/driver/MockQueryContext.java b/lens-server-api/src/test/java/org/apache/lens/server/api/driver/MockQueryContext.java
deleted file mode 100644
index 175eab3..0000000
--- a/lens-server-api/src/test/java/org/apache/lens/server/api/driver/MockQueryContext.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.lens.server.api.driver;
-
-import java.util.Collection;
-
-import org.apache.lens.api.LensConf;
-import org.apache.lens.server.api.query.AbstractQueryContext;
-
-import org.apache.hadoop.conf.Configuration;
-
-public class MockQueryContext extends AbstractQueryContext {
-
-  public MockQueryContext(final String query, final LensConf qconf,
-    final Configuration conf, final Collection<LensDriver> drivers) {
-    super(query, "testuser", qconf, conf, drivers);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server-api/src/test/java/org/apache/lens/server/api/metrics/TestMethodMetricsFactory.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/test/java/org/apache/lens/server/api/metrics/TestMethodMetricsFactory.java b/lens-server-api/src/test/java/org/apache/lens/server/api/metrics/TestMethodMetricsFactory.java
new file mode 100644
index 0000000..1975f14
--- /dev/null
+++ b/lens-server-api/src/test/java/org/apache/lens/server/api/metrics/TestMethodMetricsFactory.java
@@ -0,0 +1,65 @@
+/**
+ * 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.lens.server.api.metrics;
+
+import org.apache.lens.server.api.LensConfConstants;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.codahale.metrics.MetricRegistry;
+
+/**
+ * Tests for method metrics factory
+ */
+public class TestMethodMetricsFactory {
+
+  @Test
+  public void testMetricGauge() throws InterruptedException {
+    Configuration conf = new Configuration();
+    MethodMetricsContext mg = MethodMetricsFactory.createMethodGauge(conf, false, "nogauge");
+    Assert.assertNotNull(mg);
+    Assert.assertTrue(mg instanceof DisabledMethodMetricsContext);
+    conf.set(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY, "TestMethodMetricsFactory");
+    conf.set(LensConfConstants.QUERY_METRIC_DRIVER_STACK_NAME, "TestMethodMetricsFactoryStackName");
+    mg = MethodMetricsFactory.createMethodGauge(conf, false, "nostackgauge");
+    Assert.assertNotNull(mg);
+    Assert.assertTrue(mg instanceof MethodMetricGauge);
+    Thread.sleep(1);
+    mg.markSuccess();
+    MetricRegistry reg = LensMetricsRegistry.getStaticRegistry();
+
+    Assert.assertTrue(reg.getGauges().keySet().contains(
+      "lens.MethodMetricGauge.TestMethodMetricsFactory-nostackgauge"));
+    // assert gauge value. It will be in nano seconds
+    Assert.assertTrue(((Long)reg.getGauges().get(
+      "lens.MethodMetricGauge.TestMethodMetricsFactory-nostackgauge").getValue()) > 1000000);
+
+    mg = MethodMetricsFactory.createMethodGauge(conf, true, "stackgauge");
+    Assert.assertNotNull(mg);
+    Thread.sleep(1);
+    mg.markSuccess();
+    Assert.assertTrue(reg.getGauges().keySet().contains(
+      "lens.MethodMetricGauge.TestMethodMetricsFactoryStackName-stackgauge"));
+    Assert.assertTrue(((Long)reg.getGauges().get(
+      "lens.MethodMetricGauge.TestMethodMetricsFactoryStackName-stackgauge").getValue()) > 1000000);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server-api/src/test/java/org/apache/lens/server/api/query/MockQueryContext.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/test/java/org/apache/lens/server/api/query/MockQueryContext.java b/lens-server-api/src/test/java/org/apache/lens/server/api/query/MockQueryContext.java
new file mode 100644
index 0000000..078988b
--- /dev/null
+++ b/lens-server-api/src/test/java/org/apache/lens/server/api/query/MockQueryContext.java
@@ -0,0 +1,36 @@
+/**
+ * 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.lens.server.api.query;
+
+import java.util.Collection;
+
+import org.apache.lens.api.LensConf;
+import org.apache.lens.server.api.driver.LensDriver;
+
+import org.apache.hadoop.conf.Configuration;
+
+public class MockQueryContext extends AbstractQueryContext {
+
+  private static final long serialVersionUID = 1L;
+
+  public MockQueryContext(final String query, final LensConf qconf,
+    final Configuration conf, final Collection<LensDriver> drivers) {
+    super(query, "testuser", qconf, conf, drivers);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server-api/src/test/java/org/apache/lens/server/api/query/TestAbstractQueryContext.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/test/java/org/apache/lens/server/api/query/TestAbstractQueryContext.java b/lens-server-api/src/test/java/org/apache/lens/server/api/query/TestAbstractQueryContext.java
new file mode 100644
index 0000000..06ce751
--- /dev/null
+++ b/lens-server-api/src/test/java/org/apache/lens/server/api/query/TestAbstractQueryContext.java
@@ -0,0 +1,91 @@
+/**
+ * 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.lens.server.api.query;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.lens.api.LensConf;
+import org.apache.lens.api.LensException;
+import org.apache.lens.server.api.LensConfConstants;
+import org.apache.lens.server.api.driver.LensDriver;
+import org.apache.lens.server.api.driver.MockDriver;
+import org.apache.lens.server.api.metrics.LensMetricsRegistry;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.codahale.metrics.MetricRegistry;
+
+/**
+ * Tests for abstract query context
+ */
+public class TestAbstractQueryContext {
+
+  @Test
+  public void testMetricsConfigEnabled() throws LensException {
+    Configuration conf = new Configuration();
+    List<LensDriver> testDrivers = new ArrayList<LensDriver>();
+    MockDriver mdriver = new MockDriver();
+    mdriver.configure(conf);
+    testDrivers.add(mdriver);
+    conf.setBoolean(LensConfConstants.ENABLE_QUERY_METRICS, true);
+    MockQueryContext ctx = new MockQueryContext("mock query", new LensConf(), conf, testDrivers);
+    String uniqueMetridId = ctx.getConf().get(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY);
+    Assert.assertNotNull(uniqueMetridId);
+    UUID.fromString(uniqueMetridId);
+    StringBuilder expectedStackName = new StringBuilder();
+    expectedStackName.append(uniqueMetridId).append("-").append(MockDriver.class.getSimpleName());
+    Assert.assertEquals(ctx.getDriverConf(mdriver).get(LensConfConstants.QUERY_METRIC_DRIVER_STACK_NAME),
+      expectedStackName.toString());
+  }
+
+  @Test
+  public void testMetricsConfigDisabled() throws LensException {
+    Configuration conf = new Configuration();
+    List<LensDriver> testDrivers = new ArrayList<LensDriver>();
+    MockDriver mdriver = new MockDriver();
+    mdriver.configure(conf);
+    testDrivers.add(mdriver);
+    conf.setBoolean(LensConfConstants.ENABLE_QUERY_METRICS, false);
+    MockQueryContext ctx = new MockQueryContext("mock query", new LensConf(), conf, testDrivers);
+    Assert.assertNull(ctx.getConf().get(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY));
+    Assert.assertNull(ctx.getDriverConf(mdriver).get(LensConfConstants.QUERY_METRIC_DRIVER_STACK_NAME));
+  }
+
+  @Test
+  public void testEstimateGauges() throws LensException {
+    Configuration conf = new Configuration();
+    List<LensDriver> testDrivers = new ArrayList<LensDriver>();
+    MockDriver mdriver = new MockDriver();
+    mdriver.configure(conf);
+    testDrivers.add(mdriver);
+    conf.set(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY, TestAbstractQueryContext.class.getSimpleName());
+    MockQueryContext ctx = new MockQueryContext("mock query", new LensConf(), conf, testDrivers);
+    ctx.estimateCostForDrivers();
+    MetricRegistry reg = LensMetricsRegistry.getStaticRegistry();
+
+    Assert.assertTrue(reg.getGauges().keySet().containsAll(Arrays.asList(
+      "lens.MethodMetricGauge.TestAbstractQueryContext-MockDriver-driverEstimate")));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server/src/main/java/org/apache/lens/server/metrics/MethodMetrics.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/metrics/MethodMetrics.java b/lens-server/src/main/java/org/apache/lens/server/metrics/MethodMetrics.java
deleted file mode 100644
index 6a60421..0000000
--- a/lens-server/src/main/java/org/apache/lens/server/metrics/MethodMetrics.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.lens.server.metrics;
-
-import org.apache.lens.server.api.metrics.MethodMetricsContext;
-
-import com.codahale.metrics.Meter;
-import com.codahale.metrics.Timer;
-import lombok.NonNull;
-
-/**
- * metrics measuring for resource methods. Every resource method has one timer and two meters associated with it. One
- * meter is marking number of invocations of the method, another is marking number of invocations where exception
- * occurred.
- */
-
-public class MethodMetrics {
-  // Corresponding meter and timer counts are same, but they are marked at different times.
-  // meter is marked at the start of event, timer is marked at the end of the event.
-  private final Meter meter;
-  private final Timer successTimer;
-  private final Timer exceptionTimer;
-
-  public MethodMetrics(
-    @NonNull Meter meter, @NonNull Timer successTimer, @NonNull Timer exceptionTimer) {
-    this.meter = meter;
-    this.successTimer = successTimer;
-    this.exceptionTimer = exceptionTimer;
-  }
-
-  /**
-   * Inner class instances handle contexts for the same instance of MethodMetrics. This is useful since one method can
-   * be executing multiple times in parallel. But the contexts will be different.
-   */
-  public class Context implements MethodMetricsContext {
-    private Timer.Context successTimerContext;
-    private Timer.Context exceptionTimerContext;
-
-    private Context() {
-      meter.mark();
-      successTimerContext = successTimer.time();
-      exceptionTimerContext = exceptionTimer.time();
-    }
-
-    @Override
-    public void markError() {
-      exceptionTimerContext.close();
-    }
-
-    @Override
-    public void markSuccess() {
-      successTimerContext.close();
-    }
-  }
-
-  /** Create and return new context since an execution of this method has started */
-  public MethodMetricsContext newContext() {
-    return new Context();
-  }
-
-  public long getCount() {
-    return meter.getCount();
-  }
-
-  public long getSuccessCount() {
-    return successTimer.getCount();
-  }
-
-  public long getErrorCount() {
-    return exceptionTimer.getCount();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server/src/main/java/org/apache/lens/server/metrics/MethodMetricsFactory.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/metrics/MethodMetricsFactory.java b/lens-server/src/main/java/org/apache/lens/server/metrics/MethodMetricsFactory.java
deleted file mode 100644
index f035848..0000000
--- a/lens-server/src/main/java/org/apache/lens/server/metrics/MethodMetricsFactory.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.lens.server.metrics;
-
-
-import static com.codahale.metrics.MetricRegistry.name;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.lens.server.model.LensContainerRequest;
-import org.apache.lens.server.model.LensResourceMethod;
-
-import org.apache.log4j.Logger;
-
-import org.glassfish.jersey.server.ContainerRequest;
-import org.glassfish.jersey.server.model.ResourceMethod;
-
-import com.codahale.metrics.MetricRegistry;
-import com.google.common.base.Optional;
-import com.google.common.base.Supplier;
-import lombok.Getter;
-import lombok.NonNull;
-
-/**
- * Factory for creating MethodMetrics objects on demand.
- */
-public class MethodMetricsFactory {
-  public static final Logger LOG = Logger.getLogger(MethodMetricsFactory.class);
-
-  private final MetricRegistry metricRegistry;
-
-  public MethodMetricsFactory(@NonNull MetricRegistry metricRegistry) {
-    this.metricRegistry = metricRegistry;
-  }
-
-  /** factory helper */
-  @Getter
-  private volatile Map<String, MethodMetrics> methodMetricsMap = new HashMap<String, MethodMetrics>();
-
-  /**
-   * This is a factory method for getting a MethodMetrics instance. First a unique name is determined for the arguments
-   * and then function with same name is called with that unique name.
-   *
-   * @param method
-   * @param containerRequest
-   * @return
-   * @see #get(String)
-   * @see #getUniqueName(org.glassfish.jersey.server.model.ResourceMethod, org.glassfish.jersey.server.ContainerRequest)
-   */
-  public MethodMetrics get(@NonNull ResourceMethod method, @NonNull ContainerRequest containerRequest) {
-    return get(getUniqueName(method, containerRequest));
-  }
-
-  /**
-   * Returns MethodMetrics object corresponding to the given name. If doesn't exist yet, one will be created.
-   *
-   * @param name
-   * @return
-   */
-  public MethodMetrics get(@NonNull final String name) {
-    //SUSPEND CHECKSTYLE CHECK DoubleCheckedLockingCheck
-    MethodMetrics result = methodMetricsMap.get(name);
-    if (result == null) {
-      synchronized (this) {
-        result = methodMetricsMap.get(name);
-        if (result == null) {
-          LOG.info("Creating MethodMetrics of name: " + name);
-          result = new MethodMetrics(
-            metricRegistry.meter(name(name, "meter")),
-            metricRegistry.timer(name(name, "timer")),
-            metricRegistry.timer(name(name, "exception.timer"))
-          );
-          methodMetricsMap.put(name, result);
-        }
-      }
-    }
-    //RESUME CHECKSTYLE CHECK DoubleCheckedLockingCheck
-    return result;
-  }
-
-  /**
-   * Wrapper method. Creates Lens model wrappers: objects of {@link org.apache.lens.server.model.LensResourceMethod} and
-   * {@link org.apache.lens.server.model.LensContainerRequest}. And calls function of same name with those objects as
-   * arguments.
-   *
-   * @param method
-   * @param containerRequest
-   * @return unique name of MethodMetrics object to be returned by #get
-   * @see #getUniqueName
-   * @see #get(org.glassfish.jersey.server.model.ResourceMethod, org.glassfish.jersey.server.ContainerRequest)
-   */
-  private String getUniqueName(ResourceMethod method, ContainerRequest containerRequest) {
-    return getUniqueName(new LensResourceMethod(method), new LensContainerRequest(containerRequest));
-  }
-
-  /**
-   * Extracts base name from lensResourceMethod#name. Checks if the resource method was annotated with {@link
-   * org.apache.lens.server.api.annotations.MultiPurposeResource}. If not, returns the base name If yes, extracts the
-   * multi purpose form param from the resource method, gets the value of that param from lensContainerRequest object
-   * and appends that to baseName. if value is not passed, gets the default value of that argument from
-   * lensContainerRequest and appends that to baseName. Returns the final string constructed
-   *
-   * @param lensResourceMethod
-   * @param lensContainerRequest
-   * @return Unique name of the MethodMetrics object associated with given arguments.
-   * @see org.apache.lens.server.model.LensResourceMethod#name()
-   * @see org.apache.lens.server.api.annotations.MultiPurposeResource
-   */
-  private String getUniqueName(final LensResourceMethod lensResourceMethod, LensContainerRequest lensContainerRequest) {
-    StringBuilder sb = new StringBuilder();
-    sb.append(lensResourceMethod.name());
-    final Optional<String> multiPurposeFormParam = lensResourceMethod.getMultiPurposeFormParam();
-    if (multiPurposeFormParam.isPresent()) {
-      sb.append(".");
-      String value = lensContainerRequest.getFormDataFieldValue(multiPurposeFormParam.get()).or(
-        new Supplier<String>() {
-          @Override
-          public String get() {
-            return lensResourceMethod.getDefaultValueForParam(multiPurposeFormParam.get());
-          }
-        });
-      sb.append(value.toUpperCase());
-    }
-    return sb.toString();
-  }
-
-  /**
-   * Remove all meters/timers/... created when MethodMetrics objects were constructed using this factory.
-   */
-  public void clear() {
-    synchronized (this) {
-      LOG.info("clearing factory");
-      for (Map.Entry<String, MethodMetrics> entry : methodMetricsMap.entrySet()) {
-        metricRegistry.remove(name(entry.getKey(), "meter"));
-        metricRegistry.remove(name(entry.getKey(), "timer"));
-        metricRegistry.remove(name(entry.getKey(), "exception.timer"));
-      }
-      methodMetricsMap.clear();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server/src/main/java/org/apache/lens/server/metrics/MetricsServiceImpl.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/metrics/MetricsServiceImpl.java b/lens-server/src/main/java/org/apache/lens/server/metrics/MetricsServiceImpl.java
index 3f20bfd..90137e1 100644
--- a/lens-server/src/main/java/org/apache/lens/server/metrics/MetricsServiceImpl.java
+++ b/lens-server/src/main/java/org/apache/lens/server/metrics/MetricsServiceImpl.java
@@ -34,7 +34,9 @@ import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.events.AsyncEventListener;
 import org.apache.lens.server.api.events.LensEventService;
 import org.apache.lens.server.api.metrics.DisabledMethodMetricsContext;
+import org.apache.lens.server.api.metrics.LensMetricsRegistry;
 import org.apache.lens.server.api.metrics.MethodMetricsContext;
+import org.apache.lens.server.api.metrics.MethodMetricsFactory;
 import org.apache.lens.server.api.metrics.MetricsService;
 import org.apache.lens.server.api.query.QueryExecutionService;
 import org.apache.lens.server.api.query.StatusChange;
@@ -192,7 +194,7 @@ public class MetricsServiceImpl extends AbstractService implements MetricsServic
     queryStatusListener = new AsyncQueryStatusListener();
     LensEventService eventService = (LensEventService) LensServices.get().getService(LensEventService.NAME);
     eventService.addListenerForType(queryStatusListener, StatusChange.class);
-    metricRegistry = new MetricRegistry();
+    metricRegistry = LensMetricsRegistry.getStaticRegistry();
     methodMetricsFactory = new MethodMetricsFactory(metricRegistry);
     setEnableResourceMethodMetering(hiveConf.getBoolean(LensConfConstants.ENABLE_RESOURCE_METHOD_METERING, false));
     healthCheck = new HealthCheckRegistry();

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server/src/main/java/org/apache/lens/server/model/LensContainerRequest.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/model/LensContainerRequest.java b/lens-server/src/main/java/org/apache/lens/server/model/LensContainerRequest.java
deleted file mode 100644
index 307ae10..0000000
--- a/lens-server/src/main/java/org/apache/lens/server/model/LensContainerRequest.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.lens.server.model;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.glassfish.jersey.media.multipart.FormDataBodyPart;
-import org.glassfish.jersey.media.multipart.FormDataMultiPart;
-import org.glassfish.jersey.message.internal.ReaderWriter;
-import org.glassfish.jersey.server.ContainerRequest;
-
-import com.google.common.base.Optional;
-import lombok.AllArgsConstructor;
-import lombok.NonNull;
-
-/**
- * model object to wrap ContainerRequest. provides functions useful in server code. making a wrapper ensures separation
- * of concerns.
- */
-@AllArgsConstructor
-public class LensContainerRequest {
-  @NonNull
-  private ContainerRequest containerRequest;
-
-  /**
-   * parses form data as multipart form data and extracts the value of given fieldName from it.
-   *
-   * @param fieldName
-   * @return Optional value of field passed as multipart post data
-   */
-  public Optional<String> getFormDataFieldValue(final String fieldName) {
-    FormDataBodyPart field = getFormData(FormDataMultiPart.class).getField(fieldName);
-    return field == null ? Optional.<String>absent() : Optional.of(field.getValue());
-  }
-
-  /**
-   * Utility method for reading form/multipart-form data from container request.
-   *
-   * @param clz Either Form.class or FormDataMultiPart.class
-   * @param <T> clz type
-   * @return an instance of T
-   */
-  public <T> T getFormData(Class<T> clz) {
-
-    InputStream in = containerRequest.getEntityStream();
-    if (in.getClass() != ByteArrayInputStream.class) {
-      // Buffer input
-      ByteArrayOutputStream baos = new ByteArrayOutputStream();
-      try {
-        ReaderWriter.writeTo(in, baos);
-      } catch (IOException e) {
-        throw new IllegalArgumentException(e);
-      }
-      in = new ByteArrayInputStream(baos.toByteArray());
-      containerRequest.setEntityStream(in);
-    }
-    ByteArrayInputStream bais = (ByteArrayInputStream) in;
-    T f = containerRequest.readEntity(clz);
-    bais.reset();
-    return f;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server/src/main/java/org/apache/lens/server/model/LensResourceMethod.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/model/LensResourceMethod.java b/lens-server/src/main/java/org/apache/lens/server/model/LensResourceMethod.java
deleted file mode 100644
index a3dca46..0000000
--- a/lens-server/src/main/java/org/apache/lens/server/model/LensResourceMethod.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.lens.server.model;
-
-import java.lang.annotation.Annotation;
-
-import org.apache.lens.server.api.annotations.MultiPurposeResource;
-
-import org.glassfish.jersey.server.model.Parameter;
-import org.glassfish.jersey.server.model.ResourceMethod;
-
-import com.google.common.base.Optional;
-import lombok.AllArgsConstructor;
-import lombok.NonNull;
-
-/**
- * Model object wrapping ResourceMethod. Adds utility methods.
- */
-@AllArgsConstructor
-public class LensResourceMethod {
-  @NonNull
-  private ResourceMethod resourceMethod;
-
-  /**
-   * if handler method is annotated with {@link org.apache.lens.server.api.annotations.MultiPurposeResource}, extract
-   * that and return {@link org.apache.lens.server.api.annotations.MultiPurposeResource#formParamName()}
-   *
-   * @return
-   */
-  public Optional<String> getMultiPurposeFormParam() {
-    return getHandlerAnnotation(MultiPurposeResource.class).isPresent()
-      ? Optional.of(getHandlerAnnotation(MultiPurposeResource.class).get().formParamName()) : Optional.<String>absent();
-  }
-
-  /**
-   * className.methodname.httpmethod
-   *
-   * @return qualified name of the handler method
-   */
-  public String name() {
-    return new StringBuilder()
-      .append(resourceMethod.getInvocable().getHandlingMethod().getDeclaringClass().getCanonicalName())
-      .append(".")
-      .append(resourceMethod.getInvocable().getHandlingMethod().getName())
-      .append(".")
-      .append(resourceMethod.getHttpMethod())
-      .toString();
-  }
-
-  /**
-   * Get annotation of class clz on the handler method
-   *
-   * @param <T>
-   * @param clz annotation class
-   * @return
-   */
-  private <T extends Annotation> Optional<T> getHandlerAnnotation(Class<T> clz) {
-    return Optional.fromNullable(resourceMethod.getInvocable().getHandlingMethod().getAnnotation(clz));
-  }
-
-  /**
-   * Extract default value of a parameter from the parameter annotations of the handler method
-   *
-   * @param paramName value of {@link org.glassfish.jersey.media.multipart.FormDataParam} annotation on the handler
-   *                  method. Which will be the name of the argument in the request parameters
-   * @return value of {@link javax.ws.rs.DefaultValue} annotation on the handler method
-   */
-  public String getDefaultValueForParam(final String paramName) {
-    for (Parameter param : resourceMethod.getInvocable().getParameters()) {
-      if (param.getSourceName().equals(paramName)) {
-        return param.getDefaultValue();
-      }
-    }
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/683516c9/lens-server/src/main/java/org/apache/lens/server/query/QueryExecutionServiceImpl.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/query/QueryExecutionServiceImpl.java b/lens-server/src/main/java/org/apache/lens/server/query/QueryExecutionServiceImpl.java
index 2f44938..f9f6645 100644
--- a/lens-server/src/main/java/org/apache/lens/server/query/QueryExecutionServiceImpl.java
+++ b/lens-server/src/main/java/org/apache/lens/server/query/QueryExecutionServiceImpl.java
@@ -43,6 +43,8 @@ import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.driver.*;
 import org.apache.lens.server.api.events.LensEventListener;
 import org.apache.lens.server.api.events.LensEventService;
+import org.apache.lens.server.api.metrics.MethodMetricsContext;
+import org.apache.lens.server.api.metrics.MethodMetricsFactory;
 import org.apache.lens.server.api.metrics.MetricsService;
 import org.apache.lens.server.api.query.*;
 import org.apache.lens.server.session.LensSessionImpl;
@@ -944,6 +946,9 @@ public class QueryExecutionServiceImpl extends LensService implements QueryExecu
     prepareQueryPurger.start();
   }
 
+  private static final String ALL_REWRITES_GAUGE = "ALL_CUBE_REWRITES";
+  private static final String ALL_DRIVERS_ESTIMATE_GAUGE = "ALL_DRIVER_ESTIMATES";
+  private static final String DRIVER_SELECTOR_GAUGE = "DRIVER_SELECTION";
   /**
    * Rewrite and select.
    *
@@ -951,11 +956,20 @@ public class QueryExecutionServiceImpl extends LensService implements QueryExecu
    * @throws LensException the lens exception
    */
   private void rewriteAndSelect(AbstractQueryContext ctx) throws LensException {
+    MethodMetricsContext rewriteGauge = MethodMetricsFactory.createMethodGauge(ctx.getConf(), false,
+      ALL_REWRITES_GAUGE);
     ctx.setDriverQueries(RewriteUtil.rewriteQuery(ctx));
+    rewriteGauge.markSuccess();
+    MethodMetricsContext estimateGauge = MethodMetricsFactory.createMethodGauge(ctx.getConf(), false,
+      ALL_DRIVERS_ESTIMATE_GAUGE);
     ctx.estimateCostForDrivers();
+    estimateGauge.markSuccess();
 
+    MethodMetricsContext selectGauge = MethodMetricsFactory.createMethodGauge(ctx.getConf(), false,
+      DRIVER_SELECTOR_GAUGE);
     // 2. select driver to run the query
     LensDriver driver = driverSelector.select(ctx, conf);
+    selectGauge.markSuccess();
 
     ctx.setSelectedDriver(driver);
   }
@@ -1325,6 +1339,7 @@ public class QueryExecutionServiceImpl extends LensService implements QueryExecu
         if (query == null) {
           throw new NotFoundException("Query not found " + queryHandle);
         }
+        // pass the query conf instead of service conf
         return query.toQueryContext(conf, drivers.values());
       }
       updateStatus(queryHandle);