You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by su...@apache.org on 2017/11/17 08:26:02 UTC

lens git commit: LENS-1483 : Implementation for Jdbc driver cost

Repository: lens
Updated Branches:
  refs/heads/master 3b657dee4 -> 231121425


LENS-1483 : Implementation for Jdbc driver cost


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

Branch: refs/heads/master
Commit: 231121425f156bd572fb847fe87847ee093bbb9a
Parents: 3b657de
Author: Rajitha R <ra...@gmail.com>
Authored: Fri Nov 17 13:55:44 2017 +0530
Committer: sushilmohanty <su...@apache.org>
Committed: Fri Nov 17 13:55:44 2017 +0530

----------------------------------------------------------------------
 .../apache/lens/api/query/QueryCostType.java    |  2 +-
 .../FactPartitionBasedQueryCostCalculator.java  | 21 ++++--
 .../cube/query/cost/StaticCostCalculator.java   | 48 +++++++++++++
 ...stFactPartitionBasedQueryCostCalculator.java |  4 +-
 .../org/apache/lens/driver/hive/HiveDriver.java |  9 +--
 .../src/main/resources/hivedriver-default.xml   | 12 ++++
 .../apache/lens/driver/hive/TestHiveDriver.java |  1 -
 .../org/apache/lens/driver/jdbc/JDBCDriver.java | 41 +++++++----
 .../driver/jdbc/JDBCDriverConfConstants.java    |  4 ++
 .../src/main/resources/jdbcdriver-default.xml   | 25 +++++++
 .../apache/lens/driver/jdbc/TestJdbcDriver.java | 15 ++--
 .../lens/server/api/LensConfConstants.java      | 22 ++++++
 .../query/cost/FactPartitionBasedQueryCost.java | 12 ++--
 .../lens/server/api/query/cost/QueryCost.java   |  9 ++-
 .../api/query/cost/QueryCostCalculator.java     |  2 +
 .../api/query/cost/QueryCostTypeDecider.java    | 33 +++++++++
 .../api/query/cost/QueryCostTypeRangeConf.java  | 71 ++++++++++++++++++
 .../cost/RangeBasedQueryCostTypeDecider.java    | 45 ++++++++++++
 .../server/api/query/cost/StaticQueryCost.java  | 76 ++++++++++++++++++++
 .../server/api/query/priority/RangeConf.java    |  2 +-
 .../api/query/cost/MockQueryCostCalculator.java |  5 ++
 .../cost/TestFactPartitionBasedQueryCost.java   | 27 +++++++
 .../query/QueryContextComparatorTest.java       | 11 +--
 src/site/apt/admin/hivedriver-config.apt        | 20 +++---
 src/site/apt/admin/jdbcdriver-config.apt        | 66 +++++++++--------
 25 files changed, 507 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-api/src/main/java/org/apache/lens/api/query/QueryCostType.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/query/QueryCostType.java b/lens-api/src/main/java/org/apache/lens/api/query/QueryCostType.java
index 37eda42..f65578e 100644
--- a/lens-api/src/main/java/org/apache/lens/api/query/QueryCostType.java
+++ b/lens-api/src/main/java/org/apache/lens/api/query/QueryCostType.java
@@ -27,5 +27,5 @@ import javax.xml.bind.annotation.XmlType;
 @XmlType
 @XmlEnum
 public enum QueryCostType {
-  LOW, MEDIUM, HIGH
+  VERY_LOW, LOW, MEDIUM, HIGH
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-cube/src/main/java/org/apache/lens/cube/query/cost/FactPartitionBasedQueryCostCalculator.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/query/cost/FactPartitionBasedQueryCostCalculator.java b/lens-cube/src/main/java/org/apache/lens/cube/query/cost/FactPartitionBasedQueryCostCalculator.java
index 3c157ee..6d845f8 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/query/cost/FactPartitionBasedQueryCostCalculator.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/query/cost/FactPartitionBasedQueryCostCalculator.java
@@ -18,6 +18,9 @@
  */
 package org.apache.lens.cube.query.cost;
 
+import static org.apache.lens.server.api.LensConfConstants.DRIVER_COST_TYPE_RANGES;
+import static org.apache.lens.server.api.LensConfConstants.DRIVER_QUERY_COST_TYPE_DEFAULT_RANGES;
+
 import java.util.Map;
 import java.util.Set;
 
@@ -26,9 +29,7 @@ import org.apache.lens.cube.metadata.UpdatePeriod;
 import org.apache.lens.server.api.driver.LensDriver;
 import org.apache.lens.server.api.error.LensException;
 import org.apache.lens.server.api.query.AbstractQueryContext;
-import org.apache.lens.server.api.query.cost.FactPartitionBasedQueryCost;
-import org.apache.lens.server.api.query.cost.QueryCost;
-import org.apache.lens.server.api.query.cost.QueryCostCalculator;
+import org.apache.lens.server.api.query.cost.*;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
@@ -37,6 +38,8 @@ public class FactPartitionBasedQueryCostCalculator implements QueryCostCalculato
 
   public static final String UPDATE_PERIOD_WEIGHT_PREFIX = "update.period.weight.";
 
+  protected QueryCostTypeDecider queryCostTypeDecider;
+
   /**
    * Calculates total cost based on weights of selected tables and their selected partitions
    *
@@ -91,9 +94,19 @@ public class FactPartitionBasedQueryCostCalculator implements QueryCostCalculato
   }
 
   @Override
+  public void init(LensDriver lensDriver) {
+    queryCostTypeDecider = new RangeBasedQueryCostTypeDecider(
+      lensDriver.getConf().get(DRIVER_COST_TYPE_RANGES, DRIVER_QUERY_COST_TYPE_DEFAULT_RANGES));
+  }
+
+  @Override
   public QueryCost calculateCost(final AbstractQueryContext queryContext, LensDriver driver) throws LensException {
     Double cost = getTotalPartitionCost(queryContext, driver);
-    return cost == null ? null : new FactPartitionBasedQueryCost(cost);
+    QueryCost queryCost =  cost == null ? null : new FactPartitionBasedQueryCost(cost);
+    if (queryCost != null) {
+      queryCost.setQueryCostType(queryCostTypeDecider.decideCostType(queryCost));
+    }
+    return queryCost;
   }
 
   public Map<String, Set<?>> getAllPartitions(AbstractQueryContext queryContext, LensDriver driver) {

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-cube/src/main/java/org/apache/lens/cube/query/cost/StaticCostCalculator.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/query/cost/StaticCostCalculator.java b/lens-cube/src/main/java/org/apache/lens/cube/query/cost/StaticCostCalculator.java
new file mode 100644
index 0000000..2fc069a
--- /dev/null
+++ b/lens-cube/src/main/java/org/apache/lens/cube/query/cost/StaticCostCalculator.java
@@ -0,0 +1,48 @@
+/**
+ * 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.query.cost;
+
+import static org.apache.lens.server.api.LensConfConstants.*;
+
+import org.apache.lens.server.api.driver.LensDriver;
+import org.apache.lens.server.api.error.LensException;
+import org.apache.lens.server.api.query.AbstractQueryContext;
+import org.apache.lens.server.api.query.cost.*;
+
+public class StaticCostCalculator implements QueryCostCalculator {
+
+  private QueryCost queryCost;
+
+  @Override
+  public void init(LensDriver lensDriver) throws LensException {
+    QueryCostTypeDecider queryCostTypeDecider = new RangeBasedQueryCostTypeDecider(
+      lensDriver.getConf().get(DRIVER_COST_TYPE_RANGES, DRIVER_QUERY_COST_TYPE_DEFAULT_RANGES));
+    this.queryCost = new StaticQueryCost(lensDriver.getConf().getDouble(DRIVER_QUERY_COST, DEFAULT_DRIVER_QUERY_COST));
+    this.queryCost.setQueryCostType(queryCostTypeDecider.decideCostType(this.queryCost));
+  }
+
+  @Override
+  public QueryCost calculateCost(AbstractQueryContext queryContext, LensDriver driver) throws LensException {
+    return this.queryCost;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-cube/src/test/java/org/apache/lens/cube/query/cost/TestFactPartitionBasedQueryCostCalculator.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/query/cost/TestFactPartitionBasedQueryCostCalculator.java b/lens-cube/src/test/java/org/apache/lens/cube/query/cost/TestFactPartitionBasedQueryCostCalculator.java
index b5e73fd..8402253 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/query/cost/TestFactPartitionBasedQueryCostCalculator.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/query/cost/TestFactPartitionBasedQueryCostCalculator.java
@@ -47,7 +47,8 @@ import com.google.common.collect.Sets;
 
 public class TestFactPartitionBasedQueryCostCalculator {
   AbstractQueryContext queryContext;
-  FactPartitionBasedQueryCostCalculator calculator = new FactPartitionBasedQueryCostCalculator();
+  FactPartitionBasedQueryCostCalculator calculator =
+    new FactPartitionBasedQueryCostCalculator();
   LensDriver driver;
   private static String latest = "latest";
 
@@ -56,6 +57,7 @@ public class TestFactPartitionBasedQueryCostCalculator {
     driver = mock(LensDriver.class);
     when(driver.getConf()).thenReturn(new Configuration());
     queryContext = mock(AbstractQueryContext.class);
+    calculator.init(driver);
 
     ImmutableMap<String, Double> tableWeights = new ImmutableMap.Builder<String, Double>().build();
 

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-driver-hive/src/main/java/org/apache/lens/driver/hive/HiveDriver.java
----------------------------------------------------------------------
diff --git a/lens-driver-hive/src/main/java/org/apache/lens/driver/hive/HiveDriver.java b/lens-driver-hive/src/main/java/org/apache/lens/driver/hive/HiveDriver.java
index 0b4db9a..2eb94aa 100644
--- a/lens-driver-hive/src/main/java/org/apache/lens/driver/hive/HiveDriver.java
+++ b/lens-driver-hive/src/main/java/org/apache/lens/driver/hive/HiveDriver.java
@@ -1,4 +1,4 @@
-/**
+  /**
  * 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
@@ -44,9 +44,7 @@ import org.apache.lens.server.api.events.LensEventListener;
 import org.apache.lens.server.api.query.AbstractQueryContext;
 import org.apache.lens.server.api.query.PreparedQueryContext;
 import org.apache.lens.server.api.query.QueryContext;
-import org.apache.lens.server.api.query.cost.FactPartitionBasedQueryCost;
-import org.apache.lens.server.api.query.cost.QueryCost;
-import org.apache.lens.server.api.query.cost.QueryCostCalculator;
+import org.apache.lens.server.api.query.cost.*;
 import org.apache.lens.server.api.query.priority.CostRangePriorityDecider;
 import org.apache.lens.server.api.query.priority.CostToPriorityRangeConf;
 import org.apache.lens.server.api.query.priority.QueryPriorityDecider;
@@ -337,6 +335,7 @@ public class HiveDriver extends AbstractLensDriver {
     isEmbedded = (connectionClass.getName().equals(EmbeddedThriftConnection.class.getName()));
     connectionExpiryTimeout = getConf().getLong(HS2_CONNECTION_EXPIRY_DELAY, DEFAULT_EXPIRY_DELAY);
     whetherCalculatePriority = getConf().getBoolean(HS2_CALCULATE_PRIORITY, true);
+
     Class<? extends QueryCostCalculator> queryCostCalculatorClass = getConf().getClass(HS2_COST_CALCULATOR,
       FactPartitionBasedQueryCostCalculator.class, QueryCostCalculator.class);
     try {
@@ -344,6 +343,8 @@ public class HiveDriver extends AbstractLensDriver {
     } catch (InstantiationException | IllegalAccessException e) {
       throw new LensException("Can't instantiate query cost calculator of class: " + queryCostCalculatorClass, e);
     }
+    //For initializing the decider class instance
+    queryCostCalculator.init(this);
     queryPriorityDecider = new CostRangePriorityDecider(
       new CostToPriorityRangeConf(getConf().get(HS2_PRIORITY_RANGES, HS2_PRIORITY_DEFAULT_RANGES))
     );

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-driver-hive/src/main/resources/hivedriver-default.xml
----------------------------------------------------------------------
diff --git a/lens-driver-hive/src/main/resources/hivedriver-default.xml b/lens-driver-hive/src/main/resources/hivedriver-default.xml
index a13d3b0..2b2dc0b 100644
--- a/lens-driver-hive/src/main/resources/hivedriver-default.xml
+++ b/lens-driver-hive/src/main/resources/hivedriver-default.xml
@@ -165,4 +165,16 @@
     implementation of org.apache.lens.server.api.query.collect.WaitingQueriesSelectionPolicy.</description>
   </property>
 
+  <property>
+    <name>lens.driver.cost.query.decider.class</name>
+    <value>org.apache.lens.server.api.query.cost.RangeBasedQueryCostTypeDecider</value>
+    <description>Decider class which looks at ranges passed in config and decides the querycosttype  </description>
+  </property>
+
+  <property>
+    <name>lens.driver.cost.type.ranges</name>
+    <value>LOW,0.0,HIGH</value>
+    <description>Cost based Query type mapping</description>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/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 1261409..49bfb68 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
@@ -89,7 +89,6 @@ public class TestHiveDriver {
   protected SessionState ss;
   private CostRangePriorityDecider alwaysNormalPriorityDecider
     = new CostRangePriorityDecider(new CostToPriorityRangeConf(""));
-
   /**
    * Before test.
    *

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/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 528f857..e810e7c 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
@@ -40,6 +40,7 @@ import org.apache.lens.api.LensConf;
 import org.apache.lens.api.query.QueryHandle;
 import org.apache.lens.api.query.QueryPrepareHandle;
 import org.apache.lens.cube.parse.HQLParser;
+import org.apache.lens.cube.query.cost.StaticCostCalculator;
 import org.apache.lens.server.api.driver.*;
 import org.apache.lens.server.api.driver.DriverQueryStatus.DriverQueryState;
 import org.apache.lens.server.api.error.LensDriverErrorCode;
@@ -51,8 +52,7 @@ import org.apache.lens.server.api.query.AbstractQueryContext;
 import org.apache.lens.server.api.query.PreparedQueryContext;
 import org.apache.lens.server.api.query.QueryContext;
 import org.apache.lens.server.api.query.constraint.MaxConcurrentDriverQueriesConstraintFactory;
-import org.apache.lens.server.api.query.cost.FactPartitionBasedQueryCost;
-import org.apache.lens.server.api.query.cost.QueryCost;
+import org.apache.lens.server.api.query.cost.*;
 import org.apache.lens.server.api.query.rewrite.QueryRewriter;
 import org.apache.lens.server.api.util.LensUtil;
 import org.apache.lens.server.model.LogSegregationContext;
@@ -99,6 +99,8 @@ public class JDBCDriver extends AbstractLensDriver {
 
   private boolean isStatementCancelSupported;
 
+  QueryCostCalculator queryCostCalculator;
+
   /**
    * Data related to a query submitted to JDBCDriver.
    */
@@ -382,6 +384,17 @@ public class JDBCDriver extends AbstractLensDriver {
     super.configure(conf, driverType, driverName);
     init();
     configured = true;
+    Class<? extends QueryCostCalculator> queryCostCalculatorClass = getConf().getClass(JDBC_COST_CALCULATOR,
+      StaticCostCalculator.class, QueryCostCalculator.class);
+    try {
+      queryCostCalculator = queryCostCalculatorClass.newInstance();
+    } catch (InstantiationException | IllegalAccessException e) {
+      throw new LensException("Can't instantiate query cost calculator of class: " + queryCostCalculatorClass, e);
+    }
+
+    //For initializing the decider class instance
+    queryCostCalculator.init(this);
+
     log.info("JDBC Driver {} configured", getFullyQualifiedName());
   }
 
@@ -421,6 +434,9 @@ public class JDBCDriver extends AbstractLensDriver {
       DEFAULT_STATEMENT_CANCEL_SUPPORTED);
   }
 
+  public QueryCost calculateQueryCost(AbstractQueryContext qctx) throws LensException {
+    return queryCostCalculator.calculateCost(qctx, this);
+  }
   /**
    * Check configured.
    *
@@ -513,22 +529,21 @@ public class JDBCDriver extends AbstractLensDriver {
     return rewrittenQuery;
   }
 
-  static final QueryCost JDBC_DRIVER_COST = new FactPartitionBasedQueryCost(0);
-
   /**
    * Dummy JDBC query Plan class to get min cost selector working.
    */
   private static class JDBCQueryPlan extends DriverQueryPlan {
+
+    @Getter
+    private final QueryCost cost;
+
+    JDBCQueryPlan(QueryCost cost){
+      this.cost = cost;
+    }
     @Override
     public String getPlan() {
       return "";
     }
-
-    @Override
-    public QueryCost getCost() {
-      // this means that JDBC driver is only selected for tables with just DB storage.
-      return JDBC_DRIVER_COST;
-    }
   }
 
   private static final String VALIDATE_GAUGE = "validate-thru-prepare";
@@ -542,7 +557,7 @@ public class JDBCDriver extends AbstractLensDriver {
       VALIDATE_GAUGE);
     validate(qctx);
     validateGauge.markSuccess();
-    return JDBC_DRIVER_COST;
+    return calculateQueryCost(qctx);
   }
 
   /**
@@ -590,7 +605,7 @@ public class JDBCDriver extends AbstractLensDriver {
         result.close();
       }
     }
-    JDBCQueryPlan jqp = new JDBCQueryPlan();
+    JDBCQueryPlan jqp = new JDBCQueryPlan(calculateQueryCost(explainCtx));
     explainCtx.getDriverContext().setDriverQueryPlan(this, jqp);
     return jqp;
   }
@@ -802,7 +817,7 @@ public class JDBCDriver extends AbstractLensDriver {
   public DriverQueryPlan explainAndPrepare(PreparedQueryContext pContext) throws LensException {
     checkConfigured();
     prepare(pContext);
-    return new JDBCQueryPlan();
+    return new JDBCQueryPlan(calculateQueryCost(pContext));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-driver-jdbc/src/main/java/org/apache/lens/driver/jdbc/JDBCDriverConfConstants.java
----------------------------------------------------------------------
diff --git a/lens-driver-jdbc/src/main/java/org/apache/lens/driver/jdbc/JDBCDriverConfConstants.java b/lens-driver-jdbc/src/main/java/org/apache/lens/driver/jdbc/JDBCDriverConfConstants.java
index 51abc96..f2bfb69 100644
--- a/lens-driver-jdbc/src/main/java/org/apache/lens/driver/jdbc/JDBCDriverConfConstants.java
+++ b/lens-driver-jdbc/src/main/java/org/apache/lens/driver/jdbc/JDBCDriverConfConstants.java
@@ -51,6 +51,10 @@ public final class JDBCDriverConfConstants {
   /** The Constant JDBC_PASSWORD. */
   public static final String JDBC_PASSWORD = JDBC_DRIVER_PFX + "db.password";
 
+  public static final String JDBC_COST_CALCULATOR = JDBC_DRIVER_PFX + "cost.calculator.class";
+
+  public static final String JDBC_COST_QUERY_DECIDER = JDBC_DRIVER_PFX + "cost.query.decider.class";
+
   public enum ConnectionPoolProperties {
     /** The Constant JDBC_POOL_MAX_SIZE_DEFAULT. */
     JDBC_POOL_MAX_SIZE("maxPoolSize", JDBC_DRIVER_PFX + "pool.max.size", 15),

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-driver-jdbc/src/main/resources/jdbcdriver-default.xml
----------------------------------------------------------------------
diff --git a/lens-driver-jdbc/src/main/resources/jdbcdriver-default.xml b/lens-driver-jdbc/src/main/resources/jdbcdriver-default.xml
index 5545fac..511fe71 100644
--- a/lens-driver-jdbc/src/main/resources/jdbcdriver-default.xml
+++ b/lens-driver-jdbc/src/main/resources/jdbcdriver-default.xml
@@ -269,4 +269,29 @@
     <description>Flag to indicate Whether cancel on JDBC statement is supported. If not supported,
       framework wont call cancel on JDBC statement.</description>
   </property>
+
+  <property>
+    <name>lens.driver.jdbc.cost.calculator.class</name>
+    <value>org.apache.lens.cube.query.cost.StaticCostCalculator</value>
+    <description>Cost calculator class. By default calculating cost through static values </description>
+  </property>
+
+  <property>
+    <name>lens.driver.query.cost</name>
+    <value>0.0</value>
+    <description>Jdbc driver static cost value</description>
+  </property>
+
+  <property>
+    <name>lens.driver.cost.query.decider.class</name>
+    <value>org.apache.lens.server.api.query.cost.RangeBasedQueryCostTypeDecider</value>
+    <description>Decider class which looks at ranges passed in config and decides the querycosttype </description>
+  </property>
+
+  <property>
+    <name>lens.driver.cost.type.ranges</name>
+    <value>LOW,0.0,HIGH</value>
+    <description>Cost based Query type mapping</description>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/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 446e759..506935b 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
@@ -21,6 +21,8 @@ package org.apache.lens.driver.jdbc;
 import static org.apache.lens.driver.jdbc.JDBCDriverConfConstants.*;
 import static org.apache.lens.driver.jdbc.JDBCDriverConfConstants.ConnectionPoolProperties.*;
 
+import static org.apache.lens.server.api.LensConfConstants.DRIVER_COST_TYPE_RANGES;
+
 import static org.testng.Assert.*;
 
 import java.sql.*;
@@ -29,6 +31,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.lens.api.LensConf;
+import org.apache.lens.api.query.QueryCostType;
 import org.apache.lens.api.query.QueryHandle;
 import org.apache.lens.api.query.ResultRow;
 import org.apache.lens.server.api.LensConfConstants;
@@ -40,6 +43,7 @@ import org.apache.lens.server.api.query.ExplainQueryContext;
 import org.apache.lens.server.api.query.PreparedQueryContext;
 import org.apache.lens.server.api.query.QueryContext;
 import org.apache.lens.server.api.query.cost.QueryCost;
+import org.apache.lens.server.api.query.cost.StaticQueryCost;
 import org.apache.lens.server.api.util.LensUtil;
 
 import org.apache.hadoop.conf.Configuration;
@@ -72,6 +76,8 @@ public class TestJdbcDriver {
 
   Collection<LensDriver> drivers;
 
+  static final StaticQueryCost JDBC_COST = new StaticQueryCost(0.0, QueryCostType.LOW);
+
   /**
    * Test create jdbc driver.
    *
@@ -85,6 +91,7 @@ public class TestJdbcDriver {
     baseConf.set(JDBC_USER, "SA");
     baseConf.set(JDBC_PASSWORD, "");
     baseConf.set(JDBC_EXPLAIN_KEYWORD_PARAM, "explain plan for ");
+    baseConf.set(DRIVER_COST_TYPE_RANGES, "VERY_LOW,0.0,LOW,0.001,HIGH");
     hConf = new HiveConf(baseConf, this.getClass());
 
     driver = new JDBCDriver();
@@ -263,7 +270,7 @@ public class TestJdbcDriver {
     ExplainQueryContext ctx = createExplainContext(query1, baseConf);
     Assert.assertNull(ctx.getFinalDriverQuery(driver));
     QueryCost cost = driver.estimate(ctx);
-    Assert.assertEquals(cost, JDBCDriver.JDBC_DRIVER_COST);
+    Assert.assertEquals(cost, JDBC_COST);
     Assert.assertNotNull(ctx.getFinalDriverQuery(driver));
 
     // Test connection leak for estimate
@@ -336,7 +343,7 @@ public class TestJdbcDriver {
     // run estimate and execute - because server would first run estimate and then execute with same context
     QueryContext ctx = createQueryContext(query1, metricConf);
     QueryCost cost = driver.estimate(ctx);
-    Assert.assertEquals(cost, JDBCDriver.JDBC_DRIVER_COST);
+    Assert.assertEquals(cost, JDBC_COST);
     LensResultSet result = driver.execute(ctx);
     Assert.assertNotNull(result);
 
@@ -344,13 +351,13 @@ public class TestJdbcDriver {
     // run estimate and prepare - because server would first run estimate and then prepare with same context
     PreparedQueryContext pContext = new PreparedQueryContext(query1, "SA", metricConf, drivers);
     cost = driver.estimate(pContext);
-    Assert.assertEquals(cost, JDBCDriver.JDBC_DRIVER_COST);
+    Assert.assertEquals(cost, JDBC_COST);
     driver.prepare(pContext);
 
     // test explain and prepare
     PreparedQueryContext pContext2 = new PreparedQueryContext(query1, "SA", metricConf, drivers);
     cost = driver.estimate(pContext2);
-    Assert.assertEquals(cost, JDBCDriver.JDBC_DRIVER_COST);
+    Assert.assertEquals(cost, JDBC_COST);
     driver.prepare(pContext2);
     driver.explainAndPrepare(pContext2);
   }

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/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 7fd487c..d5273be 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
@@ -1253,4 +1253,26 @@ public final class LensConfConstants {
    * Default Value of the config "lens.cube.metastore.enable.datacompleteness.check"
    */
   public static final boolean DEFAULT_ENABLE_DATACOMPLETENESS_CHECK = false;
+
+  /**
+   * This property is for setting static cost to driver
+   */
+  public static final String DRIVER_QUERY_COST = DRIVER_PFX + "query.cost";
+
+  /**
+   * Config param for defining query type ranges.
+   */
+  public static final String DRIVER_COST_TYPE_RANGES = "lens.driver.cost.type.ranges";
+
+  /*
+  * Default query cost range
+  * */
+  public static final String DRIVER_QUERY_COST_TYPE_DEFAULT_RANGES = "LOW,0.0,HIGH";
+
+  /**
+   * Default Value of the config "lens.driver.query.cost"
+   */
+  public static final double DEFAULT_DRIVER_QUERY_COST = 0.0;
+
+  public static final String DRIVER_COST_QUERY_DECIDER = DRIVER_PFX + "cost.query.decider.class";
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/FactPartitionBasedQueryCost.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/FactPartitionBasedQueryCost.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/FactPartitionBasedQueryCost.java
index eba8f0d..2e23dd4 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/FactPartitionBasedQueryCost.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/FactPartitionBasedQueryCost.java
@@ -27,17 +27,24 @@ import org.apache.lens.api.query.QueryCostType;
 
 import com.google.common.base.Preconditions;
 import lombok.EqualsAndHashCode;
+import lombok.Getter;
+import lombok.Setter;
 import lombok.ToString;
 
 @ToString
 @EqualsAndHashCode
 public class FactPartitionBasedQueryCost implements QueryCost<FactPartitionBasedQueryCost>, Serializable {
 
+  @Getter
   private final double partitionCost;
+  @Getter
+  @Setter
+  private QueryCostType queryCostType;
 
   public FactPartitionBasedQueryCost(final double partitionCost) {
     Preconditions.checkArgument(partitionCost >= 0, "Cost can't be negative");
     this.partitionCost = partitionCost;
+    this.queryCostType = QueryCostType.HIGH;
   }
 
   @Override
@@ -46,11 +53,6 @@ public class FactPartitionBasedQueryCost implements QueryCost<FactPartitionBased
   }
 
   @Override
-  public QueryCostType getQueryCostType() {
-    return partitionCost == 0 ? QueryCostType.LOW : QueryCostType.HIGH;
-  }
-
-  @Override
   public long getEstimatedExecTimeMillis() throws UnsupportedOperationException {
     throw new UnsupportedOperationException("Estimated time is not implemented");
   }

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCost.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCost.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCost.java
index 9d7320a..fcbbeb6 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCost.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCost.java
@@ -21,7 +21,6 @@
  */
 package org.apache.lens.server.api.query.cost;
 
-
 import org.apache.lens.api.query.QueryCostType;
 
 /**
@@ -34,8 +33,16 @@ public interface QueryCost<T extends QueryCost> extends Comparable<T> {
 
   QueryCostType getQueryCostType();
 
+  void setQueryCostType(QueryCostType queryCostType);
+
   long getEstimatedExecTimeMillis() throws UnsupportedOperationException;
 
   double getEstimatedResourceUsage() throws UnsupportedOperationException;
 
+  @Override
+  default int compareTo(QueryCost queryCost) {
+    return (new Double(this.getEstimatedResourceUsage()))
+      .compareTo(queryCost.getEstimatedResourceUsage());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCostCalculator.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCostCalculator.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCostCalculator.java
index 8c71c00..bf8e936 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCostCalculator.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCostCalculator.java
@@ -23,6 +23,8 @@ import org.apache.lens.server.api.error.LensException;
 import org.apache.lens.server.api.query.AbstractQueryContext;
 
 public interface QueryCostCalculator {
+
+  void init(LensDriver lensDriver) throws LensException;
   /**
    * @param queryContext
    * @return calculated cost based on anything available in abstract query context

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCostTypeDecider.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCostTypeDecider.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCostTypeDecider.java
new file mode 100644
index 0000000..8b2fdbe
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCostTypeDecider.java
@@ -0,0 +1,33 @@
+/**
+ * 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.cost;
+
+
+import org.apache.lens.api.query.QueryCostType;
+import org.apache.lens.server.api.error.LensException;
+
+public interface QueryCostTypeDecider {
+
+  /**
+   * @param cost
+   * @return calculate queryCostType based on the pre calculated query cost
+   * @throws LensException when can't decide queryCostType.
+   */
+  QueryCostType decideCostType(QueryCost cost) throws LensException;
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCostTypeRangeConf.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCostTypeRangeConf.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCostTypeRangeConf.java
new file mode 100644
index 0000000..4a9daae
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/QueryCostTypeRangeConf.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.query.cost;
+
+
+import org.apache.lens.api.query.QueryCostType;
+import org.apache.lens.server.api.query.priority.RangeConf;
+
+public class QueryCostTypeRangeConf extends RangeConf<Double, QueryCostType> {
+  /**
+   * Super constructor
+   *
+   * @param confValue
+   * @see RangeConf#RangeConf(String)
+   */
+  public QueryCostTypeRangeConf(String confValue) {
+    super(confValue);
+  }
+
+  /**
+   * Parse key method
+   *
+   * @param s
+   * @return parsed float from string s
+   * @see RangeConf#parseKey(String)
+   */
+  @Override
+  protected Double parseKey(String s) {
+    return Double.parseDouble(s);
+  }
+
+  /**
+   * Parse value method
+   *
+   * @param s
+   * @return parsed QueryCostType from String s
+   * @see RangeConf#parseValue(String)
+   */
+  @Override
+  protected QueryCostType parseValue(String s) {
+    return QueryCostType.valueOf(s);
+  }
+
+  /**
+   * Default value is "HIGH".
+   * @return "HIGH"
+   */
+  @Override
+  protected String getDefaultConf() {
+    return QueryCostType.HIGH.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/RangeBasedQueryCostTypeDecider.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/RangeBasedQueryCostTypeDecider.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/RangeBasedQueryCostTypeDecider.java
new file mode 100644
index 0000000..bc97594
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/RangeBasedQueryCostTypeDecider.java
@@ -0,0 +1,45 @@
+/**
+ * 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.cost;
+
+import org.apache.lens.api.query.QueryCostType;
+import org.apache.lens.server.api.error.LensException;
+
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+@RequiredArgsConstructor
+public class RangeBasedQueryCostTypeDecider implements QueryCostTypeDecider {
+
+  @NonNull
+  private final QueryCostTypeRangeConf queryCostTypeRangeMap;
+
+  public RangeBasedQueryCostTypeDecider(String queryCostTypeRange) {
+    this.queryCostTypeRangeMap = new QueryCostTypeRangeConf(queryCostTypeRange);
+  }
+
+  @Override
+  public QueryCostType decideCostType(@NonNull final QueryCost cost) throws LensException {
+    QueryCostType q = queryCostTypeRangeMap.get(cost.getEstimatedResourceUsage());
+    log.info("cost was: {}, decided querytype: {}", cost, q);
+    return q;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/StaticQueryCost.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/StaticQueryCost.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/StaticQueryCost.java
new file mode 100644
index 0000000..8cdc254
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/cost/StaticQueryCost.java
@@ -0,0 +1,76 @@
+/**
+ * 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.cost;
+
+import java.io.Serializable;
+
+import org.apache.lens.api.query.QueryCostType;
+
+import lombok.EqualsAndHashCode;
+import lombok.Getter;
+import lombok.Setter;
+import lombok.ToString;
+
+@ToString
+@EqualsAndHashCode
+public class StaticQueryCost implements QueryCost<StaticQueryCost>, Serializable {
+
+  private final double staticCost;
+  @Getter
+  @Setter
+  private QueryCostType queryCostType;
+
+  public StaticQueryCost(final double cost) {
+    this.staticCost = cost;
+  }
+
+  //Added for testcase
+  public StaticQueryCost(final double cost, final QueryCostType queryCostType) {
+    this.staticCost = cost;
+    this.queryCostType = queryCostType;
+  }
+
+  @Override
+  public StaticQueryCost add(final StaticQueryCost other) {
+    return new StaticQueryCost(staticCost + other.staticCost);
+  }
+
+  @Override
+  public long getEstimatedExecTimeMillis() throws UnsupportedOperationException {
+    throw new UnsupportedOperationException("Estimated time is not implemented");
+  }
+
+  @Override
+  public double getEstimatedResourceUsage() throws UnsupportedOperationException {
+    return staticCost;
+  }
+
+  @Override
+  public int compareTo(final StaticQueryCost staticQueryCost) {
+    return new Double(staticCost).compareTo(staticQueryCost.staticCost);
+  }
+
+  @Override
+  public String toString() {
+    return getQueryCostType() + "(" + getEstimatedResourceUsage() + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-server-api/src/main/java/org/apache/lens/server/api/query/priority/RangeConf.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/priority/RangeConf.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/priority/RangeConf.java
index 43cc53c..fd44be6 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/priority/RangeConf.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/priority/RangeConf.java
@@ -62,7 +62,7 @@ public abstract class RangeConf<K extends Comparable<K>, V> {
    *
    * @param confValue
    */
-  RangeConf(String confValue) {
+  public RangeConf(String confValue) {
     if (confValue == null || confValue.isEmpty()) {
       confValue = getDefaultConf();
     }

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-server-api/src/test/java/org/apache/lens/server/api/query/cost/MockQueryCostCalculator.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/test/java/org/apache/lens/server/api/query/cost/MockQueryCostCalculator.java b/lens-server-api/src/test/java/org/apache/lens/server/api/query/cost/MockQueryCostCalculator.java
index fb4085e..72223f7 100644
--- a/lens-server-api/src/test/java/org/apache/lens/server/api/query/cost/MockQueryCostCalculator.java
+++ b/lens-server-api/src/test/java/org/apache/lens/server/api/query/cost/MockQueryCostCalculator.java
@@ -25,6 +25,11 @@ import org.apache.lens.server.api.query.AbstractQueryContext;
 
 public class MockQueryCostCalculator implements QueryCostCalculator {
   @Override
+  public void init(LensDriver lensDriver) throws LensException {
+
+  }
+
+  @Override
   public QueryCost calculateCost(AbstractQueryContext queryContext, LensDriver driver) throws LensException {
     return new FactPartitionBasedQueryCost(10.0);
   }

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-server-api/src/test/java/org/apache/lens/server/api/query/cost/TestFactPartitionBasedQueryCost.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/test/java/org/apache/lens/server/api/query/cost/TestFactPartitionBasedQueryCost.java b/lens-server-api/src/test/java/org/apache/lens/server/api/query/cost/TestFactPartitionBasedQueryCost.java
index e047c91..bd93d02 100644
--- a/lens-server-api/src/test/java/org/apache/lens/server/api/query/cost/TestFactPartitionBasedQueryCost.java
+++ b/lens-server-api/src/test/java/org/apache/lens/server/api/query/cost/TestFactPartitionBasedQueryCost.java
@@ -25,16 +25,34 @@ import static org.testng.Assert.*;
 
 import org.apache.lens.api.query.QueryCostType;
 import org.apache.lens.api.serialize.SerializationTest;
+import org.apache.lens.server.api.error.LensException;
 
+import org.testng.annotations.BeforeTest;
 import org.testng.annotations.Test;
 
 
 public class TestFactPartitionBasedQueryCost {
+
+  RangeBasedQueryCostTypeDecider costRangeQueryTypeDecider =
+    new RangeBasedQueryCostTypeDecider(new QueryCostTypeRangeConf("VERY_LOW,0.0,LOW,0.1,HIGH"));
   QueryCost cost0 = new FactPartitionBasedQueryCost(0.0);
   QueryCost cost1 = new FactPartitionBasedQueryCost(0.2);
   QueryCost cost11 = new FactPartitionBasedQueryCost(0.2);
   QueryCost cost2 = new FactPartitionBasedQueryCost(0.3);
+  QueryCost scost0 = new StaticQueryCost(-1.0);
+  QueryCost scost1 = new StaticQueryCost(0.0);
+  QueryCost scost2 = new StaticQueryCost(1.0);
 
+  @BeforeTest
+  public void beforeTest() throws LensException {
+    cost0.setQueryCostType(costRangeQueryTypeDecider.decideCostType(cost0));
+    cost1.setQueryCostType(costRangeQueryTypeDecider.decideCostType(cost1));
+    cost11.setQueryCostType(costRangeQueryTypeDecider.decideCostType(cost11));
+    cost2.setQueryCostType(costRangeQueryTypeDecider.decideCostType(cost2));
+    scost0.setQueryCostType(costRangeQueryTypeDecider.decideCostType(scost0));
+    scost1.setQueryCostType(costRangeQueryTypeDecider.decideCostType(scost1));
+    scost2.setQueryCostType(costRangeQueryTypeDecider.decideCostType(scost2));
+  }
   @Test(expectedExceptions = {IllegalArgumentException.class})
   public void testInvalid() {
     new FactPartitionBasedQueryCost(-0.5);
@@ -50,6 +68,7 @@ public class TestFactPartitionBasedQueryCost {
     assertEquals(cost1.getQueryCostType(), QueryCostType.HIGH);
     assertEquals(cost2.getQueryCostType(), QueryCostType.HIGH);
     assertEquals(cost0.getQueryCostType(), QueryCostType.LOW);
+    assertEquals(scost0.getQueryCostType(), QueryCostType.VERY_LOW);
   }
 
   @Test(expectedExceptions = {UnsupportedOperationException.class})
@@ -67,6 +86,9 @@ public class TestFactPartitionBasedQueryCost {
     assertEquals(cost1.compareTo(cost2), -1);
     assertEquals(cost2.compareTo(cost1), 1);
     assertEquals(cost1.compareTo(cost11), 0);
+    assertEquals(scost1.compareTo(scost0), 1);
+    assertEquals(scost0.compareTo(scost1), -1);
+    assertEquals(scost2.compareTo(scost0), 1);
   }
 
   @Test
@@ -80,4 +102,9 @@ public class TestFactPartitionBasedQueryCost {
   public void testFactPartitionBasedQueryCostIsSerializable() {
     new SerializationTest().verifySerializationAndDeserialization(new FactPartitionBasedQueryCost(Double.MAX_VALUE));
   }
+
+  @Test
+  public void testStaticQueryCostIsSerializable() {
+    new SerializationTest().verifySerializationAndDeserialization(new StaticQueryCost(Double.MAX_VALUE));
+  }
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/lens-server/src/test/java/org/apache/lens/server/query/QueryContextComparatorTest.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/query/QueryContextComparatorTest.java b/lens-server/src/test/java/org/apache/lens/server/query/QueryContextComparatorTest.java
index 20243f4..a800e2d 100644
--- a/lens-server/src/test/java/org/apache/lens/server/query/QueryContextComparatorTest.java
+++ b/lens-server/src/test/java/org/apache/lens/server/query/QueryContextComparatorTest.java
@@ -32,6 +32,7 @@ import org.apache.lens.server.api.query.comparators.FIFOQueryComparator;
 import org.apache.lens.server.api.query.comparators.QueryCostComparator;
 import org.apache.lens.server.api.query.comparators.QueryPriorityComparator;
 import org.apache.lens.server.api.query.cost.QueryCost;
+import org.apache.lens.server.api.query.cost.StaticQueryCost;
 
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
@@ -111,13 +112,15 @@ public class QueryContextComparatorTest {
 
     QueryContext query1 = mock(QueryContext.class);
     when(query1.getPriority()).thenReturn(Priority.HIGH);
-    QueryCost qcO1 = mock(QueryCost.class);
-    when(query1.getSelectedDriverQueryCost()).thenReturn(qcO1);
+
+    QueryCost s1 = new StaticQueryCost(0.0);
+    QueryCost s2 = new StaticQueryCost(0.0);
+
+    when(query1.getSelectedDriverQueryCost()).thenReturn(s1);
 
     QueryContext query2 = mock(QueryContext.class);
     when(query2.getPriority()).thenReturn(Priority.HIGH);
-    QueryCost qcO2 = mock(QueryCost.class);
-    when(query2.getSelectedDriverQueryCost()).thenReturn(qcO2);
+    when(query2.getSelectedDriverQueryCost()).thenReturn(s2);
 
     when(query1.getSubmissionTime()).thenReturn(submitTimeQuery1);
     when(query2.getSubmissionTime()).thenReturn(submitTimeQuery2);

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/src/site/apt/admin/hivedriver-config.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/admin/hivedriver-config.apt b/src/site/apt/admin/hivedriver-config.apt
index a9f020d..b1a8431 100644
--- a/src/site/apt/admin/hivedriver-config.apt
+++ b/src/site/apt/admin/hivedriver-config.apt
@@ -42,15 +42,19 @@ Hive driver configuration
 *--+--+---+--+
 |9|lens.cube.query.replace.timedim|true|Tells whether timedim attribute queried in the time range should be replaced with its corresponding partition column name.|
 *--+--+---+--+
-|10|lens.driver.hive.calculate.priority|true|Whether priority should be calculated for hive mr jobs or not|
+|10|lens.driver.cost.query.decider.class|org.apache.lens.server.api.query.cost.RangeBasedQueryCostTypeDecider|Decider class which looks at ranges passed in config and decides the querycosttype|
 *--+--+---+--+
-|11|lens.driver.hive.connection.class|org.apache.lens.driver.hive.EmbeddedThriftConnection|The connection class from HiveDriver to HiveServer. The default is an embedded connection which does not require a remote hive server. For connecting to a hiveserver end point, remote connection should be used. The possible values are org.apache.lens.driver.hive.EmbeddedThriftConnection and org.apache.lens.driver.hive.RemoteThriftConnection.|
+|11|lens.driver.cost.type.ranges|LOW,0.0,HIGH|Cost based Query type mapping|
 *--+--+---+--+
-|12|lens.driver.hive.cost.calculator.class|org.apache.lens.cube.query.cost.FactPartitionBasedQueryCostCalculator|Cost calculator class. By default calculating cost through fact partitions.|
+|12|lens.driver.hive.calculate.priority|true|Whether priority should be calculated for hive mr jobs or not|
 *--+--+---+--+
-|13|lens.driver.hive.hs2.connection.expiry.delay|600000|The idle time (in milliseconds) for expiring connection from hivedriver to HiveServer2|
+|13|lens.driver.hive.connection.class|org.apache.lens.driver.hive.EmbeddedThriftConnection|The connection class from HiveDriver to HiveServer. The default is an embedded connection which does not require a remote hive server. For connecting to a hiveserver end point, remote connection should be used. The possible values are org.apache.lens.driver.hive.EmbeddedThriftConnection and org.apache.lens.driver.hive.RemoteThriftConnection.|
 *--+--+---+--+
-|14|lens.driver.hive.priority.ranges|VERY_HIGH,7.0,HIGH,30.0,NORMAL,90,LOW|Priority Ranges. The numbers are the costs of the query.                                                                                                                                                    \ |
+|14|lens.driver.hive.cost.calculator.class|org.apache.lens.cube.query.cost.FactPartitionBasedQueryCostCalculator|Cost calculator class. By default calculating cost through fact partitions.|
+*--+--+---+--+
+|15|lens.driver.hive.hs2.connection.expiry.delay|600000|The idle time (in milliseconds) for expiring connection from hivedriver to HiveServer2|
+*--+--+---+--+
+|16|lens.driver.hive.priority.ranges|VERY_HIGH,7.0,HIGH,30.0,NORMAL,90,LOW|Priority Ranges. The numbers are the costs of the query.                                                                                                                                                    \ |
 |  |                                |                                     |The cost is calculated based on partition weights and fact weights. The interpretation of the default config is:                                                                                            \ |
 |  |                                |                                     |                                                                                                                                                                                                            \ |
 |  |                                |                                     |cost \<= 7\ \ \ \ \ \ \ \ \ \ \ :\ \ \ \ \ Priority = VERY_HIGH                                                                                                                                             \ |
@@ -66,10 +70,10 @@ Hive driver configuration
 |  |                                |                                     |One use case in range tuning can be that you never want queries to run with VERY_HIGH, assuming no other changes, you'll modify the value of this param in hivedriver-site.xml to be HIGH,30.0,NORMAL,90,LOW\ |
 |  |                                |                                     |via the configs, you can tune both the ranges and partition weights. this would give the end user more control.                                                                                               |
 *--+--+---+--+
-|15|lens.driver.hive.query.hook.classes| |The query hook classes for hive driver. By default there are no hooks. To add a hook, you should look at the default implementation and from there it'll be easy to derive what value can be added through a new hook. Multiple hooks can be provided by providing comma seperated name of classes.|
+|17|lens.driver.hive.query.hook.classes| |The query hook classes for hive driver. By default there are no hooks. To add a hook, you should look at the default implementation and from there it'll be easy to derive what value can be added through a new hook. Multiple hooks can be provided by providing comma seperated name of classes.|
 *--+--+---+--+
-|16|lens.driver.hive.query.launching.constraint.factories| |Factories used to instantiate constraints enforced on queries by driver. A query will be launched only if all constraints pass. Every Factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.constraint.QueryLaunchingConstraint.|
+|18|lens.driver.hive.query.launching.constraint.factories| |Factories used to instantiate constraints enforced on queries by driver. A query will be launched only if all constraints pass. Every Factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.constraint.QueryLaunchingConstraint.|
 *--+--+---+--+
-|17|lens.driver.hive.waiting.queries.selection.policy.factories| |Factories used to instantiate driver specific waiting queries selection policies. Every factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.collect.WaitingQueriesSelectionPolicy.|
+|19|lens.driver.hive.waiting.queries.selection.policy.factories| |Factories used to instantiate driver specific waiting queries selection policies. Every factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.collect.WaitingQueriesSelectionPolicy.|
 *--+--+---+--+
 The configuration parameters and their default values

http://git-wip-us.apache.org/repos/asf/lens/blob/23112142/src/site/apt/admin/jdbcdriver-config.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/admin/jdbcdriver-config.apt b/src/site/apt/admin/jdbcdriver-config.apt
index eb56927..c50872b 100644
--- a/src/site/apt/admin/jdbcdriver-config.apt
+++ b/src/site/apt/admin/jdbcdriver-config.apt
@@ -34,63 +34,71 @@ Jdbc driver configuration
 *--+--+---+--+
 |5|lens.cube.query.time.range.writer.class|org.apache.lens.cube.parse.BetweenTimeRangeWriter|The timerange writer class which specifies how the resolved partitions in timeranges should be written in final query. Available writers are org.apache.lens.cube.parse.ORTimeRangeWriter and org.apache.lens.cube.parse.BetweenTimeRangeWriter|
 *--+--+---+--+
-|6|lens.driver.jdbc.connection.properties| |Connection properties for jdbc connection.|
+|6|lens.driver.cost.query.decider.class|org.apache.lens.server.api.query.cost.RangeBasedQueryCostTypeDecider|Decider class which looks at ranges passed in config and decides the querycosttype|
 *--+--+---+--+
-|7|lens.driver.jdbc.connection.provider| |A contract for obtaining JDBC connections|
+|7|lens.driver.cost.type.ranges|LOW,0.0,HIGH|Cost based Query type mapping|
 *--+--+---+--+
-|8|lens.driver.jdbc.db.password| |The database user's password|
+|8|lens.driver.jdbc.connection.properties| |Connection properties for jdbc connection.|
 *--+--+---+--+
-|9|lens.driver.jdbc.db.uri| |JDBC connection URL in the format jdbc:dbms://host:port/dbname|
+|9|lens.driver.jdbc.connection.provider| |A contract for obtaining JDBC connections|
 *--+--+---+--+
-|10|lens.driver.jdbc.db.user| |The database user on whose behalf the connection is being made|
+|10|lens.driver.jdbc.cost.calculator.class|org.apache.lens.cube.query.cost.StaticCostCalculator|Cost calculator class. By default calculating cost through static values|
 *--+--+---+--+
-|11|lens.driver.jdbc.driver.class|com.mysql.jdbc.Driver|Type of JDBC driver used to connect backend database|
+|11|lens.driver.jdbc.db.password| |The database user's password|
 *--+--+---+--+
-|12|lens.driver.jdbc.enable.resultset.streaming.retrieval|false|Flag to enable row by row retrieval of result set from the database server. This is used to enable streaming result sets for MySQL. This is set to false by default.|
+|12|lens.driver.jdbc.db.uri| |JDBC connection URL in the format jdbc:dbms://host:port/dbname|
 *--+--+---+--+
-|13|lens.driver.jdbc.estimate.connection.properties| |Connection properties for jdbc estimate connection.|
+|13|lens.driver.jdbc.db.user| |The database user on whose behalf the connection is being made|
 *--+--+---+--+
-|14|lens.driver.jdbc.estimate.db.password| |The database user's password, for estimate queries. If this property is unspecified, value for lens.driver.jdbc.db.password would be used. Override this property to tune estimate connection pool|
+|14|lens.driver.jdbc.driver.class|com.mysql.jdbc.Driver|Type of JDBC driver used to connect backend database|
 *--+--+---+--+
-|15|lens.driver.jdbc.estimate.db.uri| |JDBC connection URL in the format jdbc:dbms://host:port/dbname for estimate queries. If this property is unspecified, value for lens.driver.jdbc.db.uri will be used.|
+|15|lens.driver.jdbc.enable.resultset.streaming.retrieval|false|Flag to enable row by row retrieval of result set from the database server. This is used to enable streaming result sets for MySQL. This is set to false by default.|
 *--+--+---+--+
-|16|lens.driver.jdbc.estimate.db.user| |The database user on whose behalf the connection is being made, for estimate queries. If this property is unspecified, value for lens.driver.jdbc.db.user would be used. Override this property to tune estimate connection pool|
+|16|lens.driver.jdbc.estimate.connection.properties| |Connection properties for jdbc estimate connection.|
 *--+--+---+--+
-|17|lens.driver.jdbc.estimate.driver.class| |Type of JDBC driver used to connect backend database for estimate queries. If This property is not specified, value for lens.driver.jdbc.driver.class will be used. Override this property to tune estimate connection pool|
+|17|lens.driver.jdbc.estimate.db.password| |The database user's password, for estimate queries. If this property is unspecified, value for lens.driver.jdbc.db.password would be used. Override this property to tune estimate connection pool|
 *--+--+---+--+
-|18|lens.driver.jdbc.estimate.get.connection.timeout| |Response timeout in milliseconds of any JDBC call invoking data transmission over a connection socket , for estimate queries. If this property is not specified, value for lens.driver.jdbc.get.connection.timeout would be used. Override this property to tune estimate connection pool.|
+|18|lens.driver.jdbc.estimate.db.uri| |JDBC connection URL in the format jdbc:dbms://host:port/dbname for estimate queries. If this property is unspecified, value for lens.driver.jdbc.db.uri will be used.|
 *--+--+---+--+
-|19|lens.driver.jdbc.estimate.pool.idle.time| |Maximum idle time in sec before a connection is closed, for estimate queries. If this property is not specified, value for lens.driver.jdbc.pool.idle.time would be used. Override this property to tune estimate connection pool.|
+|19|lens.driver.jdbc.estimate.db.user| |The database user on whose behalf the connection is being made, for estimate queries. If this property is unspecified, value for lens.driver.jdbc.db.user would be used. Override this property to tune estimate connection pool|
 *--+--+---+--+
-|20|lens.driver.jdbc.estimate.pool.max.size| |Maximum number of concurrent connections allowed in pool, for estimate queries. If this property is unspecified, value for lens.driver.jdbc.pool.max.size would be used. Override this property to tune estimate connection pool|
+|20|lens.driver.jdbc.estimate.driver.class| |Type of JDBC driver used to connect backend database for estimate queries. If This property is not specified, value for lens.driver.jdbc.driver.class will be used. Override this property to tune estimate connection pool|
 *--+--+---+--+
-|21|lens.driver.jdbc.estimate.pool.max.statements| |Maximum number of prepared statements to cache per connection, for estimate queries. If this property is not specified, value for lens.driver.jdbc.pool.max.statements would be used.|
+|21|lens.driver.jdbc.estimate.get.connection.timeout| |Response timeout in milliseconds of any JDBC call invoking data transmission over a connection socket , for estimate queries. If this property is not specified, value for lens.driver.jdbc.get.connection.timeout would be used. Override this property to tune estimate connection pool.|
 *--+--+---+--+
-|22|lens.driver.jdbc.explain.keyword|Explain|Explain keyword used to get the query plan of underlying database|
+|22|lens.driver.jdbc.estimate.pool.idle.time| |Maximum idle time in sec before a connection is closed, for estimate queries. If this property is not specified, value for lens.driver.jdbc.pool.idle.time would be used. Override this property to tune estimate connection pool.|
 *--+--+---+--+
-|23|lens.driver.jdbc.fetch.size|1000|Fetch size for JDBC result set|
+|23|lens.driver.jdbc.estimate.pool.max.size| |Maximum number of concurrent connections allowed in pool, for estimate queries. If this property is unspecified, value for lens.driver.jdbc.pool.max.size would be used. Override this property to tune estimate connection pool|
 *--+--+---+--+
-|24|lens.driver.jdbc.get.connection.timeout|10000|The number of milliseconds a client calling getConnection() will wait for a Connection to be checked-in or acquired when the pool is exhausted. Zero means wait indefinitely. Setting any positive value will cause the getConnection () call to time-out and break with an SQLException after the specified number of milliseconds. The default value of this property is 10 secs.|
+|24|lens.driver.jdbc.estimate.pool.max.statements| |Maximum number of prepared statements to cache per connection, for estimate queries. If this property is not specified, value for lens.driver.jdbc.pool.max.statements would be used.|
 *--+--+---+--+
-|25|lens.driver.jdbc.pool.idle.time|600|Maximum idle time in sec before a connection is closed|
+|25|lens.driver.jdbc.explain.keyword|Explain|Explain keyword used to get the query plan of underlying database|
 *--+--+---+--+
-|26|lens.driver.jdbc.pool.max.size|15|Maximum number of concurrent connections allowed in pool|
+|26|lens.driver.jdbc.fetch.size|1000|Fetch size for JDBC result set|
 *--+--+---+--+
-|27|lens.driver.jdbc.pool.max.statements|20|Maximum number of prepared statements to cache per connection|
+|27|lens.driver.jdbc.get.connection.timeout|10000|The number of milliseconds a client calling getConnection() will wait for a Connection to be checked-in or acquired when the pool is exhausted. Zero means wait indefinitely. Setting any positive value will cause the getConnection () call to time-out and break with an SQLException after the specified number of milliseconds. The default value of this property is 10 secs.|
 *--+--+---+--+
-|28|lens.driver.jdbc.query.launching.constraint.factories|org.apache.lens.server.api.query.constraint.MaxConcurrentDriverQueriesConstraintFactory,
+|28|lens.driver.jdbc.pool.idle.time|600|Maximum idle time in sec before a connection is closed|
+*--+--+---+--+
+|29|lens.driver.jdbc.pool.max.size|15|Maximum number of concurrent connections allowed in pool|
+*--+--+---+--+
+|30|lens.driver.jdbc.pool.max.statements|20|Maximum number of prepared statements to cache per connection|
+*--+--+---+--+
+|31|lens.driver.jdbc.query.launching.constraint.factories|org.apache.lens.server.api.query.constraint.MaxConcurrentDriverQueriesConstraintFactory,
       org.apache.lens.driver.jdbc.MaxJDBCConnectionCheckConstraintFactory|Factories used to instantiate constraints enforced on queries by driver. A query will be launched only if all constraints pass. Every Factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.constraint.QueryLaunchingConstraint.|
 *--+--+---+--+
-|29|lens.driver.jdbc.query.rewriter|org.apache.lens.driver.jdbc.ColumnarSQLRewriter|Rewriting the HQL to optimized sql queries|
+|32|lens.driver.jdbc.query.rewriter|org.apache.lens.driver.jdbc.ColumnarSQLRewriter|Rewriting the HQL to optimized sql queries|
+*--+--+---+--+
+|33|lens.driver.jdbc.regex.replacement.values|to_date=date, format_number=format, date_sub\((.*?)\,\s*([0-9]+\s*)\)=date_sub($1\, interval $2 day), date_add\((.*?)\,\s*([0-9]+\s*)\)=date_add($1\, interval $2 day)|Rewriting the HQL to optimized sql queries|
 *--+--+---+--+
-|30|lens.driver.jdbc.regex.replacement.values|to_date=date, format_number=format, date_sub\((.*?)\,\s*([0-9]+\s*)\)=date_sub($1\, interval $2 day), date_add\((.*?)\,\s*([0-9]+\s*)\)=date_add($1\, interval $2 day)|Rewriting the HQL to optimized sql queries|
+|34|lens.driver.jdbc.statement.cancel.supported|true|Flag to indicate Whether cancel on JDBC statement is supported. If not supported, framework wont call cancel on JDBC statement.|
 *--+--+---+--+
-|31|lens.driver.jdbc.statement.cancel.supported|true|Flag to indicate Whether cancel on JDBC statement is supported. If not supported, framework wont call cancel on JDBC statement.|
+|35|lens.driver.jdbc.validate.through.prepare|true|Flag to enable query syntactic and semantic validation using prepared statement.|
 *--+--+---+--+
-|32|lens.driver.jdbc.validate.through.prepare|true|Flag to enable query syntactic and semantic validation using prepared statement.|
+|36|lens.driver.jdbc.waiting.queries.selection.policy.factories|org.apache.lens.server.api.query.collect.DriverSpecificWaitingQueriesSelectionPolicyFactory|Factories used to instantiate driver specific waiting queries selection policies. Every factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.collect.WaitingQueriesSelectionPolicy.|
 *--+--+---+--+
-|33|lens.driver.jdbc.waiting.queries.selection.policy.factories|org.apache.lens.server.api.query.collect.DriverSpecificWaitingQueriesSelectionPolicyFactory|Factories used to instantiate driver specific waiting queries selection policies. Every factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.collect.WaitingQueriesSelectionPolicy.|
+|37|lens.driver.query.cost|0.0|Jdbc driver static cost value|
 *--+--+---+--+
-|34|lens.query.timeout.millis|3600000|The runtime(millis) of the query after which query will be timedout and cancelled. Default is 1 hour for jdbc queries.|
+|38|lens.query.timeout.millis|3600000|The runtime(millis) of the query after which query will be timedout and cancelled. Default is 1 hour for jdbc queries.|
 *--+--+---+--+
 The configuration parameters and their default values