You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by sh...@apache.org on 2015/12/30 08:10:46 UTC

[27/50] [abbrv] lens git commit: LENS-123 : Adds ability to load different instances of same driver class

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/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 7201e0d..ffd2d42 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
@@ -47,6 +47,7 @@ import org.apache.lens.driver.hive.HiveDriver;
 import org.apache.lens.server.BaseLensService;
 import org.apache.lens.server.LensServerConf;
 import org.apache.lens.server.LensServices;
+import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.driver.*;
 import org.apache.lens.server.api.error.LensException;
 import org.apache.lens.server.api.error.LensMultiCauseException;
@@ -72,7 +73,8 @@ import org.apache.lens.server.util.FairPriorityBlockingQueue;
 import org.apache.lens.server.util.UtilityMethods;
 
 import org.apache.commons.collections.CollectionUtils;
-import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.StringUtils;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -86,6 +88,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
+
 import lombok.*;
 import lombok.extern.slf4j.Slf4j;
 
@@ -337,25 +340,9 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
    * @throws LensException the lens exception
    */
   private void loadDriversAndSelector() throws LensException {
-    Class<?>[] driverClasses = conf.getClasses(DRIVER_CLASSES);
-    if (driverClasses != null) {
-      for (Class<?> driverClass : driverClasses) {
-        try {
-          LensDriver driver = (LensDriver) driverClass.newInstance();
-          driver.configure(LensServerConf.getConfForDrivers());
-          if (driver instanceof HiveDriver) {
-            driver.registerDriverEventListener(driverEventListener);
-          }
-          drivers.put(driverClass.getName(), driver);
-          log.info("Driver for {} is loaded", driverClass);
-        } catch (Exception e) {
-          log.warn("Could not load the driver:{}", driverClass, e);
-          throw new LensException("Could not load driver " + driverClass, e);
-        }
-      }
-    } else {
-      throw new LensException("No drivers specified");
-    }
+    //Load all configured Drivers
+    loadDrivers();
+    //Load configured Driver Selector
     try {
       Class<? extends DriverSelector> driverSelectorClass = conf.getClass(DRIVER_SELECTOR_CLASS,
         MinQueryCostSelector.class,
@@ -369,6 +356,87 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
     }
   }
 
+  /**
+   * Loads drivers for the configured Driver types in lens-site.xml
+   *
+   * The driver's resources (<driver_type>-site.xml and other files) should be present
+   * under directory conf/drivers/<driver-type>/<driver-name>
+   * Example :conf/drivers/hive/h1, conf/drivers/hive/h2, conf/drivers/jdbc/mysql1, conf/drivers/jdbc/vertica1
+   *
+   * @throws LensException
+   */
+  private void loadDrivers() throws LensException {
+    Collection<String> driverTypes = conf.getStringCollection(DRIVER_TYPES_AND_CLASSES);
+    if (driverTypes.isEmpty()) {
+      throw new LensException("No drivers configured");
+    }
+    File driversBaseDir = new File(System.getProperty(LensConfConstants.CONFIG_LOCATION,
+        LensConfConstants.DEFAULT_CONFIG_LOCATION), LensConfConstants.DRIVERS_BASE_DIR);
+    if (!driversBaseDir.isDirectory()) {
+      throw new LensException("No drivers found at location " + driversBaseDir.getAbsolutePath());
+    }
+    for (String driverType : driverTypes) {
+      if (StringUtils.isBlank(driverType)) {
+        throw new LensException("Driver type Configuration not specified correctly. Encountered blank driver type");
+      }
+      String[] driverTypeAndClass = StringUtils.split(driverType.trim(), ':');
+      if (driverTypeAndClass.length != 2) {
+        throw new LensException("Driver type Configuration not specified correctly : " + driverType);
+      }
+      loadDriversForType(driverTypeAndClass[0], driverTypeAndClass[1], driversBaseDir);
+    }
+    if (drivers.isEmpty()){
+      throw new LensException("No drivers loaded. Please check the drivers in :"+driversBaseDir);
+    }
+  }
+  /**
+   * Loads drivers of a particular type
+   *
+   * @param driverType : type of driver (hive, jdbc, el, etc)
+   * @param driverTypeClassName :driver class name
+   * @param driversBaseDir :path for drivers directory where all driver relates resources are avilable
+   * @throws LensException
+   */
+  private void loadDriversForType(String driverType, String driverTypeClassName, File driversBaseDir)
+    throws LensException {
+    File driverTypeBaseDir = new File(driversBaseDir, driverType);
+    File[] driverPaths = driverTypeBaseDir.listFiles();
+    if (!driverTypeBaseDir.isDirectory() || driverPaths == null || driverPaths.length == 0) {
+      // May be the deployment does not have drivers of this type. We can log and ignore.
+      log.warn("No drivers of type {} found in {}.", driverType, driverTypeBaseDir.getAbsolutePath());
+      return;
+    }
+    Class driverTypeClass = null;
+    try {
+      driverTypeClass = conf.getClassByName(driverTypeClassName);
+    } catch (Exception e) {
+      log.error("Could not load the driver type class {}", driverTypeClassName, e);
+      throw new LensException("Could not load Driver type class " + driverTypeClassName);
+    }
+    LensDriver driver = null;
+    String driverName = null;
+    for (File driverPath : driverPaths) {
+      try {
+        if (!driverPath.isDirectory()){
+          log.warn("Ignoring resource {} while loading drivers. A driver directory was expected instead",
+              driverPath.getAbsolutePath());
+          continue;
+        }
+        driverName = driverPath.getName();
+        driver = (LensDriver) driverTypeClass.newInstance();
+        driver.configure(LensServerConf.getConfForDrivers(), driverType, driverName);
+        // Register listener for all drivers. Drivers can choose to ignore this registration. As of now only Hive
+        // Driver supports driver event listeners.
+        driver.registerDriverEventListener(driverEventListener);
+        drivers.put(driver.getFullyQualifiedName(), driver);
+        log.info("Driver {} for type {} is loaded", driverPath.getName(), driverType);
+      } catch (Exception e) {
+        log.error("Could not load driver {} of type {}", driverPath.getName(), driverType, e);
+        throw new LensException("Could not load driver "+driverPath.getName()+ " of type "+ driverType);
+      }
+    }
+  }
+
   private MetricsService getMetrics() {
     if (metricsService == null) {
       metricsService = LensServices.get().getService(MetricsService.NAME);
@@ -2088,7 +2156,7 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
         long querySubmitTime = context.getSubmissionTime();
         if ((filterByStatus && status != context.getStatus().getStatus())
           || (filterByQueryName && !context.getQueryName().toLowerCase().contains(queryName))
-          || (filterByDriver && !context.getSelectedDriver().getClass().getName().equalsIgnoreCase(driver))
+          || (filterByDriver && !context.getSelectedDriver().getFullyQualifiedName().equalsIgnoreCase(driver))
           || (!"all".equalsIgnoreCase(userName) && !userName.equalsIgnoreCase(context.getSubmittedUser()))
           || (!(fromDate <= querySubmitTime && querySubmitTime <= toDate))) {
           itr.remove();
@@ -2305,21 +2373,25 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
     // Restore drivers
     synchronized (drivers) {
       int numDrivers = in.readInt();
+      String driverQualifiedName;
+      String driverClsName;
       for (int i = 0; i < numDrivers; i++) {
-        String driverClsName = in.readUTF();
-        LensDriver driver = drivers.get(driverClsName);
+        driverQualifiedName = in.readUTF();
+        driverClsName = in.readUTF();
+        LensDriver driver = drivers.get(driverQualifiedName);
         if (driver == null) {
           // this driver is removed in the current server restart
           // we will create an instance and read its state still.
           try {
             Class<? extends LensDriver> driverCls = (Class<? extends LensDriver>) Class.forName(driverClsName);
             driver = (LensDriver) driverCls.newInstance();
-            driver.configure(conf);
+            String[] driverTypeAndName = StringUtils.split(driverQualifiedName, '/');
+            driver.configure(conf, driverTypeAndName[0], driverTypeAndName[1]);
           } catch (Exception e) {
-            log.error("Could not instantiate driver:{}", driverClsName, e);
+            log.error("Could not instantiate driver:{} represented by class {}", driverQualifiedName, driverClsName, e);
             throw new IOException(e);
           }
-          log.info("Driver state for {} will be ignored", driverClsName);
+          log.info("Driver state for {} will be ignored", driverQualifiedName);
         }
         driver.readExternal(in);
       }
@@ -2342,8 +2414,8 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
         // set the selected driver if available, if not available for the cases of queued queries,
         // query service will do the selection from existing drivers and update
         if (driverAvailable) {
-          String clsName = in.readUTF();
-          ctx.getDriverContext().setSelectedDriver(drivers.get(clsName));
+          String selectedDriverQualifiedName = in.readUTF();
+          ctx.getDriverContext().setSelectedDriver(drivers.get(selectedDriverQualifiedName));
           ctx.setDriverQuery(ctx.getSelectedDriver(), ctx.getSelectedDriverQuery());
         }
         allQueries.put(ctx.getQueryHandle(), ctx);
@@ -2373,6 +2445,7 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
           break;
         case CLOSED:
           allQueries.remove(ctx.getQueryHandle());
+          log.info("Removed closed query from all Queries:"+ctx.getQueryHandle());
         }
       }
       queuedQueries.addAll(allRestoredQueuedQueries);
@@ -2391,8 +2464,11 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
     // persist all drivers
     synchronized (drivers) {
       out.writeInt(drivers.size());
-      for (LensDriver driver : drivers.values()) {
+      LensDriver driver = null;
+      for (Map.Entry<String, LensDriver> driverEntry : drivers.entrySet()) {
+        driver = driverEntry.getValue();
         synchronized (driver) {
+          out.writeUTF(driverEntry.getKey());
           out.writeUTF(driver.getClass().getName());
           driver.writeExternal(out);
         }
@@ -2407,7 +2483,7 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
           boolean isDriverAvailable = (ctx.getSelectedDriver() != null);
           out.writeBoolean(isDriverAvailable);
           if (isDriverAvailable) {
-            out.writeUTF(ctx.getSelectedDriver().getClass().getName());
+            out.writeUTF(ctx.getSelectedDriver().getFullyQualifiedName());
           }
         }
       }
@@ -2616,19 +2692,19 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
       List<ResourceEntry> resources = session.getLensSessionPersistInfo().getResources();
       if (resources != null && !resources.isEmpty()) {
         for (ResourceEntry resource : resources) {
-          log.info("Restoring resource {} for session {}", resource, lensSession);
+          log.info("{} Restoring resource {} for session {}", hiveDriver, resource, lensSession);
           String command = "add " + resource.getType().toLowerCase() + " " + resource.getLocation();
           try {
             // Execute add resource query in blocking mode
             hiveDriver.execute(createResourceQuery(command, sessionHandle, hiveDriver));
             resource.restoredResource();
-            log.info("Restored resource {} for session {}", resource, lensSession);
+            log.info("{} Restored resource {} for session {}", hiveDriver, resource, lensSession);
           } catch (Exception exc) {
-            log.error("Unable to add resource {} for session {}", resource, lensSession, exc);
+            log.error("{} Unable to add resource {} for session {}", hiveDriver, resource, lensSession, exc);
           }
         }
       } else {
-        log.info("No resources to restore for session {}", lensSession);
+        log.info("{} No resources to restore for session {}", hiveDriver, lensSession);
       }
     } catch (Exception e) {
       log.warn(
@@ -2730,7 +2806,7 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
 
     String command = "add " + res.getType().toLowerCase() + " " + uri;
     driver.execute(createResourceQuery(command, sessionHandle, driver));
-    log.info("Added resource to hive driver for session {} cmd: {}", sessionIdentifier, command);
+    log.info("Added resource to hive driver {} for session {} cmd: {}", driver, sessionIdentifier, command);
   }
 
   private boolean removeFromLaunchedQueries(final QueryContext finishedQuery) {

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/main/java/org/apache/lens/server/rewrite/RewriteUtil.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/rewrite/RewriteUtil.java b/lens-server/src/main/java/org/apache/lens/server/rewrite/RewriteUtil.java
index 6c464fb..abec2b3 100644
--- a/lens-server/src/main/java/org/apache/lens/server/rewrite/RewriteUtil.java
+++ b/lens-server/src/main/java/org/apache/lens/server/rewrite/RewriteUtil.java
@@ -368,7 +368,7 @@ public final class RewriteUtil {
       log.warn("Driver : {}  Skipped for the query rewriting due to ", driver, e);
       ctx.setDriverRewriteError(driver, e);
       failureCause = new StringBuilder(" Driver :")
-          .append(driver.getClass().getName())
+          .append(driver.getFullyQualifiedName())
           .append(" Cause :" + e.getLocalizedMessage())
           .toString();
     }

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/main/java/org/apache/lens/server/session/LensSessionImpl.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/session/LensSessionImpl.java b/lens-server/src/main/java/org/apache/lens/server/session/LensSessionImpl.java
index 9d8f198..cc62d92 100644
--- a/lens-server/src/main/java/org/apache/lens/server/session/LensSessionImpl.java
+++ b/lens-server/src/main/java/org/apache/lens/server/session/LensSessionImpl.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.util.*;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.ws.rs.NotFoundException;
 
@@ -435,13 +436,11 @@ public class LensSessionImpl extends HiveSessionImpl {
     final String location;
     // For tests
     /** The restore count. */
-    @Getter
-    transient int restoreCount;
+    transient AtomicInteger restoreCount = new AtomicInteger();
 
     /** Set of databases for which this resource has been added */
     final transient Set<String> databases = new HashSet<String>();
 
-
     /**
      * Instantiates a new resource entry.
      *
@@ -468,7 +467,15 @@ public class LensSessionImpl extends HiveSessionImpl {
      * Restored resource.
      */
     public void restoredResource() {
-      restoreCount++;
+      restoreCount.incrementAndGet();
+    }
+
+    /**
+     * Returns the value of restoreCount for the resource
+     * @return
+     */
+    public int getRestoreCount(){
+      return restoreCount.get();
     }
 
     /*

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/main/resources/lensserver-default.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/main/resources/lensserver-default.xml b/lens-server/src/main/resources/lensserver-default.xml
index 5f268cb..39b72dc 100644
--- a/lens-server/src/main/resources/lensserver-default.xml
+++ b/lens-server/src/main/resources/lensserver-default.xml
@@ -24,7 +24,7 @@
 <configuration>
   <property>
     <name>lens.server.drivers</name>
-    <value>org.apache.lens.driver.hive.HiveDriver</value>
+    <value>hive:org.apache.lens.driver.hive.HiveDriver</value>
     <description>Drivers enabled for this lens server instance</description>
   </property>
 

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/java/org/apache/lens/server/LensJerseyTest.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/LensJerseyTest.java b/lens-server/src/test/java/org/apache/lens/server/LensJerseyTest.java
index 8ba9353..8f52ddd 100644
--- a/lens-server/src/test/java/org/apache/lens/server/LensJerseyTest.java
+++ b/lens-server/src/test/java/org/apache/lens/server/LensJerseyTest.java
@@ -33,6 +33,7 @@ import java.util.concurrent.ConcurrentLinkedQueue;
 import javax.ws.rs.core.UriBuilder;
 
 import org.apache.lens.driver.hive.TestRemoteHiveDriver;
+import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.metrics.LensMetricsUtil;
 import org.apache.lens.server.api.metrics.MetricsService;
 import org.apache.lens.server.model.LogSegregationContext;
@@ -48,6 +49,7 @@ import org.testng.annotations.AfterSuite;
 import org.testng.annotations.BeforeSuite;
 
 import com.google.common.collect.Lists;
+
 import lombok.extern.slf4j.Slf4j;
 
 /**
@@ -120,6 +122,7 @@ public abstract class LensJerseyTest extends JerseyTest {
   public void startAll() throws Exception {
     log.info("Before suite");
     System.setProperty("lens.log.dir", "target/");
+    System.setProperty(LensConfConstants.CONFIG_LOCATION, "target/test-classes/");
     TestRemoteHiveDriver.createHS2Service();
     System.out.println("Remote hive server started!");
     HiveConf hiveConf = new HiveConf();

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java b/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
index 7b6c560..877200f 100644
--- a/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
+++ b/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
@@ -70,6 +70,11 @@ public class TestServerRestart extends LensAllApplicationJerseyTest {
   /** The data file. */
   private File dataFile;
 
+  /**
+   * No of valid hive drivers that can execute queries in this test class
+   */
+  private static final int NO_OF_HIVE_DRIVERS = 2;
+
   /*
    * (non-Javadoc)
    *
@@ -347,7 +352,10 @@ public class TestServerRestart extends LensAllApplicationJerseyTest {
     // Now we can expect that session resources have been added back exactly once
     for (int i = 0; i < sessionResources.size(); i++) {
       LensSessionImpl.ResourceEntry resourceEntry = sessionResources.get(i);
-      assertEquals(resourceEntry.getRestoreCount(), 1 + restoreCounts[i],
+      //The restore count can vary based on How many Hive Drivers were able to execute the estimate on the query
+      //successfully after Hive Server Restart.
+      Assert.assertTrue((resourceEntry.getRestoreCount() > restoreCounts[i]
+          && resourceEntry.getRestoreCount() <=  restoreCounts[i] + NO_OF_HIVE_DRIVERS),
           "Restore test failed for " + resourceEntry + " pre count=" + restoreCounts[i] + " post count=" + resourceEntry
               .getRestoreCount());
       log.info("@@ Latest count {}={}", resourceEntry, resourceEntry.getRestoreCount());

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/java/org/apache/lens/server/query/TestEventService.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/query/TestEventService.java b/lens-server/src/test/java/org/apache/lens/server/query/TestEventService.java
index 1dab35e..702a529 100644
--- a/lens-server/src/test/java/org/apache/lens/server/query/TestEventService.java
+++ b/lens-server/src/test/java/org/apache/lens/server/query/TestEventService.java
@@ -30,6 +30,7 @@ import org.apache.lens.api.query.QueryStatus;
 import org.apache.lens.server.EventServiceImpl;
 import org.apache.lens.server.LensServerConf;
 import org.apache.lens.server.LensServices;
+import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.error.LensException;
 import org.apache.lens.server.api.events.AsyncEventListener;
 import org.apache.lens.server.api.events.LensEvent;
@@ -275,6 +276,7 @@ public class TestEventService {
    */
   @BeforeTest
   public void setup() throws Exception {
+    System.setProperty(LensConfConstants.CONFIG_LOCATION, "target/test-classes/");
     LensServices.get().init(LensServerConf.getHiveConf());
     LensServices.get().start();
     service = LensServices.get().getService(LensEventService.NAME);

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/java/org/apache/lens/server/query/TestLensDAO.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/query/TestLensDAO.java b/lens-server/src/test/java/org/apache/lens/server/query/TestLensDAO.java
index bc1463f..01e846a 100644
--- a/lens-server/src/test/java/org/apache/lens/server/query/TestLensDAO.java
+++ b/lens-server/src/test/java/org/apache/lens/server/query/TestLensDAO.java
@@ -141,7 +141,7 @@ public class TestLensDAO extends LensJerseyTest {
 
     System.out.println("@@ State = " + queryContext.getStatus().getStatus().name());
     List<QueryHandle> daoTestQueryHandles = service.lensServerDao.findFinishedQueries(finishedLensQuery.getStatus(),
-      queryContext.getSubmittedUser(), queryContext.getSelectedDriver().getClass().getName(), "daotestquery1", -1L,
+        queryContext.getSubmittedUser(), queryContext.getSelectedDriver().getFullyQualifiedName(), "daotestquery1", -1L,
       Long.MAX_VALUE);
     Assert.assertEquals(daoTestQueryHandles.size(), 1);
     Assert.assertEquals(daoTestQueryHandles.get(0).getHandleId().toString(), finishedHandle);

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/java/org/apache/lens/server/query/TestQueryConstraints.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/query/TestQueryConstraints.java b/lens-server/src/test/java/org/apache/lens/server/query/TestQueryConstraints.java
index eb94c89..ab42a3d 100644
--- a/lens-server/src/test/java/org/apache/lens/server/query/TestQueryConstraints.java
+++ b/lens-server/src/test/java/org/apache/lens/server/query/TestQueryConstraints.java
@@ -19,7 +19,6 @@
 package org.apache.lens.server.query;
 
 import static org.apache.lens.server.api.LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY;
-import static org.apache.lens.server.api.util.LensUtil.getImplementations;
 
 import static org.testng.Assert.*;
 
@@ -39,7 +38,6 @@ import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.LensServerAPITestUtil;
 import org.apache.lens.server.api.driver.DriverSelector;
 import org.apache.lens.server.api.driver.LensDriver;
-import org.apache.lens.server.api.error.LensException;
 import org.apache.lens.server.api.metrics.MetricsService;
 import org.apache.lens.server.api.query.AbstractQueryContext;
 import org.apache.lens.server.api.query.QueryExecutionService;
@@ -52,13 +50,16 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.glassfish.jersey.client.ClientConfig;
 import org.glassfish.jersey.media.multipart.MultiPartFeature;
 import org.glassfish.jersey.test.TestProperties;
+
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.AfterTest;
 import org.testng.annotations.BeforeTest;
 import org.testng.annotations.Test;
 
 import com.beust.jcommander.internal.Lists;
+
 import com.google.common.base.Optional;
+
 import lombok.extern.slf4j.Slf4j;
 
 /**
@@ -69,52 +70,6 @@ import lombok.extern.slf4j.Slf4j;
 public class TestQueryConstraints extends LensJerseyTest {
   private HiveConf serverConf;
 
-  public static class MockHiveDriverBase extends HiveDriver {
-
-    private final Configuration customConf;
-
-    /**
-     * Instantiates a new hive driver.
-     *
-     * @throws LensException the lens exception
-     */
-    public MockHiveDriverBase() throws LensException {
-      customConf = new Configuration();
-      customConf.setInt("driver.max.concurrent.launched.queries", 2);
-      customConf.set(HiveDriver.QUERY_LAUNCHING_CONSTRAINT_FACTORIES_KEY,
-        "org.apache.lens.server.api.query.constraint.MaxConcurrentDriverQueriesConstraintFactory");
-    }
-
-    @Override
-    public void configure(Configuration conf) throws LensException {
-      super.configure(conf);
-      queryConstraints = getImplementations(HiveDriver.QUERY_LAUNCHING_CONSTRAINT_FACTORIES_KEY, customConf);
-    }
-  }
-
-  public static class HiveDriver1 extends MockHiveDriverBase {
-
-    /**
-     * Instantiates a new hive driver.
-     *
-     * @throws LensException the lens exception
-     */
-    public HiveDriver1() throws LensException {
-
-    }
-  }
-
-  public static class HiveDriver2 extends MockHiveDriverBase {
-
-    /**
-     * Instantiates a new hive driver.
-     *
-     * @throws LensException the lens exception
-     */
-    public HiveDriver2() throws LensException {
-    }
-  }
-
   public static class RoundRobinSelector implements DriverSelector {
     int counter = 0;
 
@@ -158,8 +113,8 @@ public class TestQueryConstraints extends LensJerseyTest {
   public HiveConf getServerConf() {
     if (serverConf == null) {
       serverConf = new HiveConf(super.getServerConf());
-      serverConf.set(LensConfConstants.DRIVER_CLASSES,
-        HiveDriver1.class.getName() + "," + HiveDriver2.class.getName());
+      // Lets test only mockHive. updating lens server conf for same
+      serverConf.set(LensConfConstants.DRIVER_TYPES_AND_CLASSES, "mockHive:" + HiveDriver.class.getName());
       serverConf.set("lens.server.driver.selector.class", RoundRobinSelector.class.getName());
       LensServerConf.getConfForDrivers().addResource(serverConf);
     }
@@ -265,11 +220,6 @@ public class TestQueryConstraints extends LensJerseyTest {
     QueryExecutionServiceImpl.QueryCount count = queryService.getQueryCountSnapshot();
     assertTrue(count.running <= 4, System.currentTimeMillis() + " " + count.running + " running queries: "
       + queryService.getLaunchedQueries());
-    if (count.running == 4) {
-      assertEquals(count.queued, 0);
-    } else {
-      assertEquals(count.waiting, 0);
-    }
   }
 
   private QueryHandle launchQuery() {

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java b/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
index c8a1cc6..f6693aa 100644
--- a/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
+++ b/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
@@ -222,6 +222,18 @@ public class TestQueryService extends LensJerseyTest {
     assertEquals(rs.getStatus(), 400);
   }
 
+  @Test
+  public void testLoadingMultipleDrivers() {
+    Collection<LensDriver> drivers = queryService.getDrivers();
+    assertEquals(drivers.size(), 4);
+    Set<String> driverNames = new HashSet<String>(drivers.size());
+    for(LensDriver driver : drivers){
+      assertEquals(driver.getConf().get("lens.driver.test.drivername"), driver.getFullyQualifiedName());
+      driverNames.add(driver.getFullyQualifiedName());
+    }
+    assertTrue(driverNames.containsAll(Arrays.asList("hive/hive1", "hive/hive2", "jdbc/jdbc1", "mock/fail1")));
+  }
+
   /**
    * Test rewrite failure in execute operation.
    *
@@ -464,8 +476,8 @@ public class TestQueryService extends LensJerseyTest {
       .get(LensPreparedQuery.class);
     assertTrue(ctx.getUserQuery().equalsIgnoreCase("select ID from " + TEST_TABLE));
     assertTrue(ctx.getDriverQuery().equalsIgnoreCase("select ID from " + TEST_TABLE));
-    assertEquals(ctx.getSelectedDriverClassName(),
-      org.apache.lens.driver.hive.HiveDriver.class.getCanonicalName());
+    //both drivers hive/hive1 and hive/hive2 are capable of handling the query as they point to the same hive server
+    assertTrue(ctx.getSelectedDriverName().equals("hive/hive1") || ctx.getSelectedDriverName().equals("hive/hive2"));
     assertNull(ctx.getConf().getProperties().get("my.property"));
 
     // Update conf for prepared query
@@ -539,8 +551,8 @@ public class TestQueryService extends LensJerseyTest {
       .request().get(LensPreparedQuery.class);
     assertTrue(ctx.getUserQuery().equalsIgnoreCase("select ID from " + TEST_TABLE));
     assertTrue(ctx.getDriverQuery().equalsIgnoreCase("select ID from " + TEST_TABLE));
-    assertEquals(ctx.getSelectedDriverClassName(),
-      org.apache.lens.driver.hive.HiveDriver.class.getCanonicalName());
+    //both drivers hive/hive1 and hive/hive2 are capable of handling the query as they point to the same hive server
+    assertTrue(ctx.getSelectedDriverName().equals("hive/hive1") || ctx.getSelectedDriverName().equals("hive/hive2"));
     assertNull(ctx.getConf().getProperties().get("my.property"));
 
     // Update conf for prepared query
@@ -1275,6 +1287,9 @@ public class TestQueryService extends LensJerseyTest {
         if (driver instanceof HiveDriver) {
           addedToHiveDriver =
             ((HiveDriver) driver).areDBResourcesAddedForSession(sessionHandle.getPublicId().toString(), DB_WITH_JARS);
+          if (addedToHiveDriver){
+            break; //There are two Hive drivers now both pointing to same hive server. So break after first success
+          }
         }
       }
       assertTrue(addedToHiveDriver);
@@ -1396,12 +1411,15 @@ public class TestQueryService extends LensJerseyTest {
 
     assertTrue(reg.getGauges().keySet().containsAll(Arrays.asList(
         "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-DRIVER_SELECTION",
-        "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-HiveDriver-CUBE_REWRITE",
-        "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-HiveDriver-DRIVER_ESTIMATE",
-        "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-HiveDriver-RewriteUtil-rewriteQuery",
-        "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-JDBCDriver-CUBE_REWRITE",
-        "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-JDBCDriver-DRIVER_ESTIMATE",
-        "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-JDBCDriver-RewriteUtil-rewriteQuery",
+        "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-hive/hive1-CUBE_REWRITE",
+        "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-hive/hive1-DRIVER_ESTIMATE",
+        "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-hive/hive1-RewriteUtil-rewriteQuery",
+        "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-hive/hive2-CUBE_REWRITE",
+        "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-hive/hive2-DRIVER_ESTIMATE",
+        "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-hive/hive2-RewriteUtil-rewriteQuery",
+        "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-jdbc/jdbc1-CUBE_REWRITE",
+        "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-jdbc/jdbc1-DRIVER_ESTIMATE",
+        "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-jdbc/jdbc1-RewriteUtil-rewriteQuery",
         "lens.MethodMetricGauge.TestQueryService-testEstimateGauges-PARALLEL_ESTIMATE")),
       reg.getGauges().keySet().toString());
   }

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/java/org/apache/lens/server/rewrite/TestRewriting.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/rewrite/TestRewriting.java b/lens-server/src/test/java/org/apache/lens/server/rewrite/TestRewriting.java
index 2827b96..202db82 100644
--- a/lens-server/src/test/java/org/apache/lens/server/rewrite/TestRewriting.java
+++ b/lens-server/src/test/java/org/apache/lens/server/rewrite/TestRewriting.java
@@ -177,7 +177,7 @@ public class TestRewriting {
     MockDriver driver = new MockDriver();
     LensConf lensConf = new LensConf();
     Configuration conf = new Configuration();
-    driver.configure(conf);
+    driver.configure(conf, null, null);
     drivers.add(driver);
 
     CubeQueryRewriter mockWriter = getMockedRewriter();
@@ -190,7 +190,7 @@ public class TestRewriting {
     runRewrites(RewriteUtil.rewriteQuery(ctx));
 
     conf.set(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY, TestRewriting.class.getSimpleName());
-    driver.configure(conf);
+    driver.configure(conf, null, null);
     String q2 = "cube select name from table";
     Assert.assertTrue(RewriteUtil.isCubeQuery(q2));
     cubeQueries = RewriteUtil.findCubePositions(q2, hconf);
@@ -201,8 +201,8 @@ public class TestRewriting {
     MetricRegistry reg = LensMetricsRegistry.getStaticRegistry();
 
     Assert.assertTrue(reg.getGauges().keySet().containsAll(Arrays.asList(
-      "lens.MethodMetricGauge.TestRewriting-MockDriver-RewriteUtil-rewriteQuery",
-      "lens.MethodMetricGauge.TestRewriting-MockDriver-1-RewriteUtil-rewriteQuery-toHQL")));
+      "lens.MethodMetricGauge.TestRewriting-"+driver.getFullyQualifiedName()+"-RewriteUtil-rewriteQuery",
+      "lens.MethodMetricGauge.TestRewriting-"+driver.getFullyQualifiedName()+"-1-RewriteUtil-rewriteQuery-toHQL")));
     conf.unset(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY);
 
     q2 = "insert overwrite directory 'target/rewrite' cube select name from table";
@@ -290,9 +290,10 @@ public class TestRewriting {
     runRewrites(RewriteUtil.rewriteQuery(ctx));
     reg = LensMetricsRegistry.getStaticRegistry();
     Assert.assertTrue(reg.getGauges().keySet().containsAll(Arrays.asList(
-      "lens.MethodMetricGauge.TestRewriting-multiple-MockDriver-1-RewriteUtil-rewriteQuery-toHQL",
-      "lens.MethodMetricGauge.TestRewriting-multiple-MockDriver-2-RewriteUtil-rewriteQuery-toHQL",
-      "lens.MethodMetricGauge.TestRewriting-multiple-MockDriver-RewriteUtil-rewriteQuery")));
+      "lens.MethodMetricGauge.TestRewriting-"+driver.getFullyQualifiedName()+"-1-RewriteUtil-rewriteQuery-toHQL",
+      "lens.MethodMetricGauge.TestRewriting-multiple-"+driver.getFullyQualifiedName()
+        +"-2-RewriteUtil-rewriteQuery-toHQL",
+      "lens.MethodMetricGauge.TestRewriting-multiple-"+driver.getFullyQualifiedName()+"-RewriteUtil-rewriteQuery")));
     conf.unset(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY);
 
     q2 = "select * from (cube select name from table) a full outer join"
@@ -418,7 +419,7 @@ public class TestRewriting {
 
     // failing query for second driver
     MockDriver driver2 = new MockDriver();
-    driver2.configure(conf);
+    driver2.configure(conf, null, null);
     drivers.add(driver2);
 
     Assert.assertEquals(drivers.size(), 2);

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/resources/drivers/hive/hive1/hivedriver-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/drivers/hive/hive1/hivedriver-site.xml b/lens-server/src/test/resources/drivers/hive/hive1/hivedriver-site.xml
new file mode 100644
index 0000000..6362473
--- /dev/null
+++ b/lens-server/src/test/resources/drivers/hive/hive1/hivedriver-site.xml
@@ -0,0 +1,85 @@
+<?xml version="1.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.
+
+-->
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<configuration>
+
+  <property>
+    <name>lens.driver.hive.connection.class</name>
+    <value>org.apache.lens.driver.hive.RemoteThriftConnection</value>
+    <description>The connection class from HiveDriver to HiveServer.</description>
+  </property>
+
+  <property>
+     
+    <name>hive.metastore.local</name>
+     
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>hive.metastore.warehouse.dir</name>
+    <value>${project.build.directory}/hive/warehouse</value>
+  </property>
+
+  <property>
+    <name>javax.jdo.option.ConnectionURL</name>
+    <value>jdbc:derby:;databaseName=target/metastore_db;create=true</value>
+    <description>JDBC connect string for a JDBC metastore</description>
+  </property>
+
+  <property>
+    <name>hive.lock.manager</name>
+    <value>org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.bind.host</name>
+    <value>localhost</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.port</name>
+    <value>12345</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.client.retry.limit</name>
+    <value>3</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.client.connect.retry.limit</name>
+    <value>3</value>
+  </property>
+
+  <property>
+    <name>lens.driver.test.key</name>
+    <value>set</value>
+  </property>
+
+  <property>
+    <name>lens.driver.test.drivername</name>
+    <value>hive/hive1</value>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/resources/drivers/hive/hive2/hivedriver-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/drivers/hive/hive2/hivedriver-site.xml b/lens-server/src/test/resources/drivers/hive/hive2/hivedriver-site.xml
new file mode 100644
index 0000000..d5be96e
--- /dev/null
+++ b/lens-server/src/test/resources/drivers/hive/hive2/hivedriver-site.xml
@@ -0,0 +1,85 @@
+<?xml version="1.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.
+
+-->
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<configuration>
+
+  <property>
+    <name>lens.driver.hive.connection.class</name>
+    <value>org.apache.lens.driver.hive.RemoteThriftConnection</value>
+    <description>The connection class from HiveDriver to HiveServer.</description>
+  </property>
+
+  <property>
+     
+    <name>hive.metastore.local</name>
+     
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>hive.metastore.warehouse.dir</name>
+    <value>${project.build.directory}/hive/warehouse</value>
+  </property>
+
+  <property>
+    <name>javax.jdo.option.ConnectionURL</name>
+    <value>jdbc:derby:;databaseName=target/metastore_db;create=true</value>
+    <description>JDBC connect string for a JDBC metastore</description>
+  </property>
+
+  <property>
+    <name>hive.lock.manager</name>
+    <value>org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.bind.host</name>
+    <value>localhost</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.port</name>
+    <value>12345</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.client.retry.limit</name>
+    <value>3</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.client.connect.retry.limit</name>
+    <value>3</value>
+  </property>
+
+  <property>
+    <name>lens.driver.test.key</name>
+    <value>set</value>
+  </property>
+
+  <property>
+    <name>lens.driver.test.drivername</name>
+    <value>hive/hive2</value>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/resources/drivers/jdbc/jdbc1/jdbcdriver-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/drivers/jdbc/jdbc1/jdbcdriver-site.xml b/lens-server/src/test/resources/drivers/jdbc/jdbc1/jdbcdriver-site.xml
new file mode 100644
index 0000000..9ed0c87
--- /dev/null
+++ b/lens-server/src/test/resources/drivers/jdbc/jdbc1/jdbcdriver-site.xml
@@ -0,0 +1,58 @@
+<?xml version="1.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.
+
+-->
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<configuration>
+  <property>
+    <name>lens.driver.jdbc.driver.class</name>
+    <value>org.hsqldb.jdbcDriver</value>
+  </property>
+  <property>
+    <name>lens.driver.jdbc.db.uri</name>
+    <value>jdbc:hsqldb:./target/db-storage.db;MODE=MYSQL</value>
+  </property>
+  <property>
+    <name>lens.driver.jdbc.db.user</name>
+    <value>SA</value>
+  </property>
+  <property>
+    <name>lens.cube.query.driver.supported.storages</name>
+    <value>mydb</value>
+    <final>true</final>
+  </property>
+  <property>
+    <name>lens.driver.jdbc.query.rewriter</name>
+    <value>org.apache.lens.driver.jdbc.ColumnarSQLRewriter</value>
+  </property>
+  <property>
+    <name>lens.driver.jdbc.explain.keyword</name>
+    <value>explain plan for</value>
+  </property>
+  <property>
+    <name>lens.driver.test.key</name>
+    <value>set</value>
+  </property>
+  <property>
+    <name>lens.driver.test.drivername</name>
+    <value>jdbc/jdbc1</value>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/resources/drivers/mock/fail1/failing-query-driver-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/drivers/mock/fail1/failing-query-driver-site.xml b/lens-server/src/test/resources/drivers/mock/fail1/failing-query-driver-site.xml
new file mode 100644
index 0000000..b836282
--- /dev/null
+++ b/lens-server/src/test/resources/drivers/mock/fail1/failing-query-driver-site.xml
@@ -0,0 +1,32 @@
+<?xml version="1.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.
+  -->
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<configuration>
+  <property>
+    <name>lens.driver.test.key</name>
+    <value>set</value>
+  </property>
+
+  <property>
+    <name>lens.driver.test.drivername</name>
+    <value>mock/fail1</value>
+  </property>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/resources/drivers/mockHive/mockHive1/hivedriver-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/drivers/mockHive/mockHive1/hivedriver-site.xml b/lens-server/src/test/resources/drivers/mockHive/mockHive1/hivedriver-site.xml
new file mode 100644
index 0000000..723e9a1
--- /dev/null
+++ b/lens-server/src/test/resources/drivers/mockHive/mockHive1/hivedriver-site.xml
@@ -0,0 +1,95 @@
+<?xml version="1.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.
+
+-->
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<configuration>
+
+  <property>
+    <name>lens.driver.hive.connection.class</name>
+    <value>org.apache.lens.driver.hive.RemoteThriftConnection</value>
+    <description>The connection class from HiveDriver to HiveServer.</description>
+  </property>
+
+  <property>
+     
+    <name>hive.metastore.local</name>
+     
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>hive.metastore.warehouse.dir</name>
+    <value>${project.build.directory}/hive/warehouse</value>
+  </property>
+
+  <property>
+    <name>javax.jdo.option.ConnectionURL</name>
+    <value>jdbc:derby:;databaseName=target/metastore_db;create=true</value>
+    <description>JDBC connect string for a JDBC metastore</description>
+  </property>
+
+  <property>
+    <name>hive.lock.manager</name>
+    <value>org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.bind.host</name>
+    <value>localhost</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.port</name>
+    <value>12345</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.client.retry.limit</name>
+    <value>3</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.client.connect.retry.limit</name>
+    <value>3</value>
+  </property>
+
+  <property>
+    <name>lens.driver.test.key</name>
+    <value>set</value>
+  </property>
+
+  <property>
+    <name>lens.driver.test.drivername</name>
+    <value>mockHive/mockHive1</value>
+  </property>
+
+  <property>
+    <name>driver.max.concurrent.launched.queries</name>
+    <value>2</value>
+  </property>
+
+  <property>
+    <name>lens.driver.hive.query.launching.constraint.factories</name>
+    <value>org.apache.lens.server.api.query.constraint.MaxConcurrentDriverQueriesConstraintFactory</value>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/resources/drivers/mockHive/mockHive2/hivedriver-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/drivers/mockHive/mockHive2/hivedriver-site.xml b/lens-server/src/test/resources/drivers/mockHive/mockHive2/hivedriver-site.xml
new file mode 100644
index 0000000..50f82da
--- /dev/null
+++ b/lens-server/src/test/resources/drivers/mockHive/mockHive2/hivedriver-site.xml
@@ -0,0 +1,95 @@
+<?xml version="1.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.
+
+-->
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<configuration>
+
+  <property>
+    <name>lens.driver.hive.connection.class</name>
+    <value>org.apache.lens.driver.hive.RemoteThriftConnection</value>
+    <description>The connection class from HiveDriver to HiveServer.</description>
+  </property>
+
+  <property>
+     
+    <name>hive.metastore.local</name>
+     
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>hive.metastore.warehouse.dir</name>
+    <value>${project.build.directory}/hive/warehouse</value>
+  </property>
+
+  <property>
+    <name>javax.jdo.option.ConnectionURL</name>
+    <value>jdbc:derby:;databaseName=target/metastore_db;create=true</value>
+    <description>JDBC connect string for a JDBC metastore</description>
+  </property>
+
+  <property>
+    <name>hive.lock.manager</name>
+    <value>org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.bind.host</name>
+    <value>localhost</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.port</name>
+    <value>12345</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.client.retry.limit</name>
+    <value>3</value>
+  </property>
+
+  <property>
+    <name>hive.server2.thrift.client.connect.retry.limit</name>
+    <value>3</value>
+  </property>
+
+  <property>
+    <name>lens.driver.test.key</name>
+    <value>set</value>
+  </property>
+
+  <property>
+    <name>lens.driver.test.drivername</name>
+    <value>mockHive/mockHive2</value>
+  </property>
+
+  <property>
+    <name>driver.max.concurrent.launched.queries</name>
+    <value>2</value>
+  </property>
+
+  <property>
+    <name>lens.driver.hive.query.launching.constraint.factories</name>
+    <value>org.apache.lens.server.api.query.constraint.MaxConcurrentDriverQueriesConstraintFactory</value>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/resources/failing-query-driver-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/failing-query-driver-site.xml b/lens-server/src/test/resources/failing-query-driver-site.xml
deleted file mode 100644
index fee022d..0000000
--- a/lens-server/src/test/resources/failing-query-driver-site.xml
+++ /dev/null
@@ -1,27 +0,0 @@
-<?xml version="1.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.
-  -->
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
-<configuration>
-  <property>
-    <name>lens.driver.test.key</name>
-    <value>set</value>
-  </property>
-</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/resources/hivedriver-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/hivedriver-site.xml b/lens-server/src/test/resources/hivedriver-site.xml
deleted file mode 100644
index f2aed88..0000000
--- a/lens-server/src/test/resources/hivedriver-site.xml
+++ /dev/null
@@ -1,80 +0,0 @@
-<?xml version="1.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.
-
--->
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
-<configuration>
-
-  <property>
-    <name>lens.driver.hive.connection.class</name>
-    <value>org.apache.lens.driver.hive.RemoteThriftConnection</value>
-    <description>The connection class from HiveDriver to HiveServer.</description>
-  </property>
-
-  <property>
-     
-    <name>hive.metastore.local</name>
-     
-    <value>true</value>
-  </property>
-
-  <property>
-    <name>hive.metastore.warehouse.dir</name>
-    <value>${project.build.directory}/hive/warehouse</value>
-  </property>
-
-  <property>
-    <name>javax.jdo.option.ConnectionURL</name>
-    <value>jdbc:derby:;databaseName=target/metastore_db;create=true</value>
-    <description>JDBC connect string for a JDBC metastore</description>
-  </property>
-
-  <property>
-    <name>hive.lock.manager</name>
-    <value>org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager</value>
-  </property>
-
-  <property>
-    <name>hive.server2.thrift.bind.host</name>
-    <value>localhost</value>
-  </property>
-
-  <property>
-    <name>hive.server2.thrift.port</name>
-    <value>12345</value>
-  </property>
-
-  <property>
-    <name>hive.server2.thrift.client.retry.limit</name>
-    <value>3</value>
-  </property>
-
-  <property>
-    <name>hive.server2.thrift.client.connect.retry.limit</name>
-    <value>3</value>
-  </property>
-
-  <property>
-    <name>lens.driver.test.key</name>
-    <value>set</value>
-  </property>
-
-</configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/resources/jdbcdriver-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/jdbcdriver-site.xml b/lens-server/src/test/resources/jdbcdriver-site.xml
deleted file mode 100644
index 1b14f54..0000000
--- a/lens-server/src/test/resources/jdbcdriver-site.xml
+++ /dev/null
@@ -1,55 +0,0 @@
-<?xml version="1.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.
-
--->
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
-<configuration>
-  <property>
-    <name>lens.driver.jdbc.driver.class</name>
-    <value>org.hsqldb.jdbcDriver</value>
-  </property>
-  <property>
-    <name>lens.driver.jdbc.db.uri</name>
-    <value>jdbc:hsqldb:./target/db-storage.db;MODE=MYSQL</value>
-  </property>
-  <property>
-    <name>lens.driver.jdbc.db.user</name>
-    <value>SA</value>
-  </property>
-  <property>
-    <name>lens.cube.query.driver.supported.storages</name>
-    <value>mydb</value>
-    <final>true</final>
-  </property>
-  <property>
-    <name>lens.driver.jdbc.query.rewriter</name>
-    <value>org.apache.lens.driver.jdbc.ColumnarSQLRewriter</value>
-  </property>
-  <property>
-    <name>lens.driver.jdbc.explain.keyword</name>
-    <value>explain plan for</value>
-  </property>
-  <property>
-    <name>lens.driver.test.key</name>
-    <value>set</value>
-  </property>
-
-</configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/lens-server/src/test/resources/lens-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/lens-site.xml b/lens-server/src/test/resources/lens-site.xml
index cc887ef..9cb4a6f 100644
--- a/lens-server/src/test/resources/lens-site.xml
+++ b/lens-server/src/test/resources/lens-site.xml
@@ -24,8 +24,8 @@
 <configuration>
   <property>
     <name>lens.server.drivers</name>
-    <value>org.apache.lens.driver.hive.HiveDriver,org.apache.lens.driver.jdbc.JDBCDriver,
-      org.apache.lens.server.common.FailingQueryDriver</value>
+    <value>hive:org.apache.lens.driver.hive.HiveDriver,jdbc:org.apache.lens.driver.jdbc.JDBCDriver,
+      mock:org.apache.lens.server.common.FailingQueryDriver</value> <!--$LENS_CONF/drivers-->
   </property>
 
   <property>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/src/site/apt/admin/config-server.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/admin/config-server.apt b/src/site/apt/admin/config-server.apt
index 141f2b3..fea2336 100644
--- a/src/site/apt/admin/config-server.apt
+++ b/src/site/apt/admin/config-server.apt
@@ -31,10 +31,39 @@ Configuring lens server
 
 * Driver configuration
 
-  The supported drivers are configured through classnames in this release. The configuration 
-  for each driver can be specified in their site.xml file. For example, HiveDriver's
-  configuration should be specified in hivedriver-site.xml. Supported drivers in this version
-  are org.apache.lens.driver.hive.HiveDriver, org.apache.lens.driver.jdbc.JDBCDriver
+  The supported drivers are configured through type and classname in this release. The property 
+  lens.server.drivers in server configuration specifies the driver types supported by the system and 
+  their corresponding driver implementation.
+
+  System can have multiple drivers of same type, for example MySQL and Postgres can be two drivers of
+  type JDBC. The configuration for each driver is specified in its site.xml file which should be 
+  present under the folder <Server_Conf_location>/drivers/<driver-type>/<driver-name>. 
+  For example, Hive driver's configuration should be specified in 
+  <LensServer_Install_Location>/conf/drivers/hive/hive1/hivedriver-site.xml where hive1 is the driver name.
+  A driver is identified by its fully qualified name which includes the driver type and driver name. For the
+  example discussed above the fully qualified name would be "hive/hive1"
+  
+  The organization of drivers in the system is illustrated below 
+
++---+
+
+  ├── conf
+      └── drivers
+          ├── hive
+          │   ├── hive1
+          │   │   └── hivedriver-site.xml
+          │   └── hive2
+          │       └── hivedriver-site.xml
+          └── jdbc
+              ├── jdbc1
+              │   └── jdbcdriver-site.xml
+              └── jdbc2
+                  └── jdbcdriver-site.xml
+
++---+
+
+  Supported drivers in this version are 
+  org.apache.lens.driver.hive.HiveDriver, org.apache.lens.driver.jdbc.JDBCDriver
   and org.apache.lens.driver.es.ESDriver.
   The configuration parameters and their description are explained in their respective docs listed below
   HiveDriver - {{{./hivedriver-config.html} here}}

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/src/site/apt/admin/config.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/admin/config.apt b/src/site/apt/admin/config.apt
index 88c1489..bcf4b7d 100644
--- a/src/site/apt/admin/config.apt
+++ b/src/site/apt/admin/config.apt
@@ -65,7 +65,7 @@ Lens server configuration
 *--+--+---+--+
 |19|lens.server.driver.selector.class|org.apache.lens.server.api.driver.MinQueryCostSelector|Class for selecting best driver given the query context|
 *--+--+---+--+
-|20|lens.server.drivers|org.apache.lens.driver.hive.HiveDriver|Drivers enabled for this lens server instance|
+|20|lens.server.drivers|hive:org.apache.lens.driver.hive.HiveDriver|Drivers enabled for this lens server instance|
 *--+--+---+--+
 |21|lens.server.enable.console.metrics|false|Enable metrics to be reported on console|
 *--+--+---+--+

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/src/site/apt/lenshome/install-and-run.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/lenshome/install-and-run.apt b/src/site/apt/lenshome/install-and-run.apt
index 9eadc5c..961d926 100644
--- a/src/site/apt/lenshome/install-and-run.apt
+++ b/src/site/apt/lenshome/install-and-run.apt
@@ -98,14 +98,24 @@ Installing and Running Lens
   │   ├── lens-config.sh
   │   └── lens-ctl
   ├── conf
-  │   ├── hivedriver-site.xml
-  │   ├── jdbcdriver-site.xml
+  │   ├── drivers
+  │   │   ├── hive
+  │   │	  │   └── hive1
+  │   │	  │       └── hivedriver-site.xml
+  │   │	  └── jdbc
+  │   │	      └── jdbc1
+  │   │	          └── jdbcdriver-site.xml
   │   ├── lens-env.sh
   │   ├── lens-site.xml
   │   └── logback.xml
   ├── conf-pseudo-distr
-  │   ├── hivedriver-site.xml
-  │   ├── jdbcdriver-site.xml
+  │   ├── drivers
+  │   │   ├── hive
+  │   │	  │   └── hive1
+  │   │	  │       └── hivedriver-site.xml
+  │   │	  └── jdbc
+  │   │	      └── jdbc1
+  │   │	          └── jdbcdriver-site.xml
   │   ├── lens-env.sh
   │   ├── lens-site.xml
   │   └── logback.xml

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/tools/conf-pseudo-distr/server/drivers/hive/hive1/hivedriver-site.xml
----------------------------------------------------------------------
diff --git a/tools/conf-pseudo-distr/server/drivers/hive/hive1/hivedriver-site.xml b/tools/conf-pseudo-distr/server/drivers/hive/hive1/hivedriver-site.xml
new file mode 100644
index 0000000..4804356
--- /dev/null
+++ b/tools/conf-pseudo-distr/server/drivers/hive/hive1/hivedriver-site.xml
@@ -0,0 +1,57 @@
+<?xml version="1.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.
+
+-->
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<configuration>
+<property>
+  <name>lens.driver.hive.connection.class</name>
+  <value>org.apache.lens.driver.hive.RemoteThriftConnection</value>
+  <description>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. </description>
+</property>
+
+<property>
+  <name>hive.server2.thrift.bind.host</name>
+  <value>localhost</value>
+  <description>The host on which hive server is running</description>
+</property>
+
+<property>
+  <name>hive.server2.thrift.port</name>
+  <value>10000</value>
+  <description>The port on which hive server is running</description>
+</property>
+
+<!-- Adding in supported storages by hive driver -->
+<property>
+  <name>lens.cube.query.driver.supported.storages</name>
+  <value>local,cluster</value>
+  <final>true</final>
+</property>
+<property>
+  <name>hive.aux.jars.path</name>
+  <value>file:///opt/lens/lens-ml-dist/target/apache-lens-2.5.0-beta-SNAPSHOT-ml/lib/lens-ml-lib-2.5.0-beta-SNAPSHOT.jar,file:///usr/local/spark-1.3.0-bin-hadoop2.4/lib/spark-assembly-1.3.0-hadoop2.4.0.jar</value>
+</property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/tools/conf-pseudo-distr/server/drivers/jdbc/jdbc1/jdbcdriver-site.xml
----------------------------------------------------------------------
diff --git a/tools/conf-pseudo-distr/server/drivers/jdbc/jdbc1/jdbcdriver-site.xml b/tools/conf-pseudo-distr/server/drivers/jdbc/jdbc1/jdbcdriver-site.xml
new file mode 100644
index 0000000..7fad125
--- /dev/null
+++ b/tools/conf-pseudo-distr/server/drivers/jdbc/jdbc1/jdbcdriver-site.xml
@@ -0,0 +1,50 @@
+<?xml version="1.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.
+
+-->
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<configuration>
+  <property>
+    <name>lens.driver.jdbc.driver.class</name>
+    <value>jdbc:org.hsqldb.jdbcDriver</value>
+  </property>
+  <property>
+    <name>lens.driver.jdbc.db.uri</name>
+    <value>jdbc:hsqldb:/tmp/db-storage.db;MODE=MYSQL;readonly=true</value>
+  </property>
+  <property>
+    <name>lens.driver.jdbc.db.user</name>
+    <value>SA</value>
+  </property>
+  <property>
+    <name>lens.cube.query.driver.supported.storages</name>
+    <value>mydb</value>
+    <final>true</final>
+  </property>
+  <property>
+    <name>lens.driver.jdbc.query.rewriter</name>
+    <value>org.apache.lens.driver.jdbc.ColumnarSQLRewriter</value>
+  </property>
+  <property>
+    <name>lens.driver.jdbc.explain.keyword</name>
+    <value>explain plan for </value>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/tools/conf-pseudo-distr/server/hivedriver-site.xml
----------------------------------------------------------------------
diff --git a/tools/conf-pseudo-distr/server/hivedriver-site.xml b/tools/conf-pseudo-distr/server/hivedriver-site.xml
deleted file mode 100644
index 4804356..0000000
--- a/tools/conf-pseudo-distr/server/hivedriver-site.xml
+++ /dev/null
@@ -1,57 +0,0 @@
-<?xml version="1.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.
-
--->
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
-<configuration>
-<property>
-  <name>lens.driver.hive.connection.class</name>
-  <value>org.apache.lens.driver.hive.RemoteThriftConnection</value>
-  <description>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. </description>
-</property>
-
-<property>
-  <name>hive.server2.thrift.bind.host</name>
-  <value>localhost</value>
-  <description>The host on which hive server is running</description>
-</property>
-
-<property>
-  <name>hive.server2.thrift.port</name>
-  <value>10000</value>
-  <description>The port on which hive server is running</description>
-</property>
-
-<!-- Adding in supported storages by hive driver -->
-<property>
-  <name>lens.cube.query.driver.supported.storages</name>
-  <value>local,cluster</value>
-  <final>true</final>
-</property>
-<property>
-  <name>hive.aux.jars.path</name>
-  <value>file:///opt/lens/lens-ml-dist/target/apache-lens-2.5.0-beta-SNAPSHOT-ml/lib/lens-ml-lib-2.5.0-beta-SNAPSHOT.jar,file:///usr/local/spark-1.3.0-bin-hadoop2.4/lib/spark-assembly-1.3.0-hadoop2.4.0.jar</value>
-</property>
-</configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/tools/conf-pseudo-distr/server/jdbcdriver-site.xml
----------------------------------------------------------------------
diff --git a/tools/conf-pseudo-distr/server/jdbcdriver-site.xml b/tools/conf-pseudo-distr/server/jdbcdriver-site.xml
deleted file mode 100644
index 37540dd..0000000
--- a/tools/conf-pseudo-distr/server/jdbcdriver-site.xml
+++ /dev/null
@@ -1,50 +0,0 @@
-<?xml version="1.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.
-
--->
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
-<configuration>
-  <property>
-    <name>lens.driver.jdbc.driver.class</name>
-    <value>org.hsqldb.jdbcDriver</value>
-  </property>
-  <property>
-    <name>lens.driver.jdbc.db.uri</name>
-    <value>jdbc:hsqldb:/tmp/db-storage.db;MODE=MYSQL;readonly=true</value>
-  </property>
-  <property>
-    <name>lens.driver.jdbc.db.user</name>
-    <value>SA</value>
-  </property>
-  <property>
-    <name>lens.cube.query.driver.supported.storages</name>
-    <value>mydb</value>
-    <final>true</final>
-  </property>
-  <property>
-    <name>lens.driver.jdbc.query.rewriter</name>
-    <value>org.apache.lens.driver.jdbc.ColumnarSQLRewriter</value>
-  </property>
-  <property>
-    <name>lens.driver.jdbc.explain.keyword</name>
-    <value>explain plan for </value>
-  </property>
-</configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/tools/conf-pseudo-distr/server/lens-site.xml
----------------------------------------------------------------------
diff --git a/tools/conf-pseudo-distr/server/lens-site.xml b/tools/conf-pseudo-distr/server/lens-site.xml
index f43d07e..dfb9d21 100644
--- a/tools/conf-pseudo-distr/server/lens-site.xml
+++ b/tools/conf-pseudo-distr/server/lens-site.xml
@@ -31,7 +31,7 @@
 
 <property>
   <name>lens.server.drivers</name>
-  <value>org.apache.lens.driver.hive.HiveDriver,org.apache.lens.driver.jdbc.JDBCDriver</value>
+  <value>hive:org.apache.lens.driver.hive.HiveDriver,jdbc:org.apache.lens.driver.jdbc.JDBCDriver</value>
 </property>
 
 <property>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/tools/conf/server/drivers/hive/hive1/hivedriver-site.xml
----------------------------------------------------------------------
diff --git a/tools/conf/server/drivers/hive/hive1/hivedriver-site.xml b/tools/conf/server/drivers/hive/hive1/hivedriver-site.xml
new file mode 100644
index 0000000..2e8e7fa
--- /dev/null
+++ b/tools/conf/server/drivers/hive/hive1/hivedriver-site.xml
@@ -0,0 +1,41 @@
+<?xml version="1.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.
+
+-->
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<configuration>
+<property>
+  <name>hive.metastore.warehouse.dir</name>
+  <value>/tmp/hive/warehouse</value>
+</property>
+
+<property>
+  <name>hive.lock.manager</name>
+  <value>org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager</value>
+</property>
+
+<!-- Adding in supported storages by hive driver -->
+<property>
+  <name>lens.cube.query.driver.supported.storages</name>
+  <value>local,cluster</value>
+  <final>true</final>
+</property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/tools/conf/server/drivers/jdbc/jdbc1/jdbcdriver-site.xml
----------------------------------------------------------------------
diff --git a/tools/conf/server/drivers/jdbc/jdbc1/jdbcdriver-site.xml b/tools/conf/server/drivers/jdbc/jdbc1/jdbcdriver-site.xml
new file mode 100644
index 0000000..37540dd
--- /dev/null
+++ b/tools/conf/server/drivers/jdbc/jdbc1/jdbcdriver-site.xml
@@ -0,0 +1,50 @@
+<?xml version="1.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.
+
+-->
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<configuration>
+  <property>
+    <name>lens.driver.jdbc.driver.class</name>
+    <value>org.hsqldb.jdbcDriver</value>
+  </property>
+  <property>
+    <name>lens.driver.jdbc.db.uri</name>
+    <value>jdbc:hsqldb:/tmp/db-storage.db;MODE=MYSQL;readonly=true</value>
+  </property>
+  <property>
+    <name>lens.driver.jdbc.db.user</name>
+    <value>SA</value>
+  </property>
+  <property>
+    <name>lens.cube.query.driver.supported.storages</name>
+    <value>mydb</value>
+    <final>true</final>
+  </property>
+  <property>
+    <name>lens.driver.jdbc.query.rewriter</name>
+    <value>org.apache.lens.driver.jdbc.ColumnarSQLRewriter</value>
+  </property>
+  <property>
+    <name>lens.driver.jdbc.explain.keyword</name>
+    <value>explain plan for </value>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/tools/conf/server/hivedriver-site.xml
----------------------------------------------------------------------
diff --git a/tools/conf/server/hivedriver-site.xml b/tools/conf/server/hivedriver-site.xml
deleted file mode 100644
index 2e8e7fa..0000000
--- a/tools/conf/server/hivedriver-site.xml
+++ /dev/null
@@ -1,41 +0,0 @@
-<?xml version="1.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.
-
--->
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
-<configuration>
-<property>
-  <name>hive.metastore.warehouse.dir</name>
-  <value>/tmp/hive/warehouse</value>
-</property>
-
-<property>
-  <name>hive.lock.manager</name>
-  <value>org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager</value>
-</property>
-
-<!-- Adding in supported storages by hive driver -->
-<property>
-  <name>lens.cube.query.driver.supported.storages</name>
-  <value>local,cluster</value>
-  <final>true</final>
-</property>
-</configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/tools/conf/server/jdbcdriver-site.xml
----------------------------------------------------------------------
diff --git a/tools/conf/server/jdbcdriver-site.xml b/tools/conf/server/jdbcdriver-site.xml
deleted file mode 100644
index 37540dd..0000000
--- a/tools/conf/server/jdbcdriver-site.xml
+++ /dev/null
@@ -1,50 +0,0 @@
-<?xml version="1.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.
-
--->
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
-<configuration>
-  <property>
-    <name>lens.driver.jdbc.driver.class</name>
-    <value>org.hsqldb.jdbcDriver</value>
-  </property>
-  <property>
-    <name>lens.driver.jdbc.db.uri</name>
-    <value>jdbc:hsqldb:/tmp/db-storage.db;MODE=MYSQL;readonly=true</value>
-  </property>
-  <property>
-    <name>lens.driver.jdbc.db.user</name>
-    <value>SA</value>
-  </property>
-  <property>
-    <name>lens.cube.query.driver.supported.storages</name>
-    <value>mydb</value>
-    <final>true</final>
-  </property>
-  <property>
-    <name>lens.driver.jdbc.query.rewriter</name>
-    <value>org.apache.lens.driver.jdbc.ColumnarSQLRewriter</value>
-  </property>
-  <property>
-    <name>lens.driver.jdbc.explain.keyword</name>
-    <value>explain plan for </value>
-  </property>
-</configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/114dab34/tools/conf/server/lens-site.xml
----------------------------------------------------------------------
diff --git a/tools/conf/server/lens-site.xml b/tools/conf/server/lens-site.xml
index 2b12b83..0803da1 100644
--- a/tools/conf/server/lens-site.xml
+++ b/tools/conf/server/lens-site.xml
@@ -24,7 +24,7 @@
 <configuration>
 <property>
   <name>lens.server.drivers</name>
-  <value>org.apache.lens.driver.hive.HiveDriver,org.apache.lens.driver.jdbc.JDBCDriver</value>
+  <value>hive:org.apache.lens.driver.hive.HiveDriver,jdbc:org.apache.lens.driver.jdbc.JDBCDriver</value>
 </property>
 
 <property>