You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2017/12/22 08:51:37 UTC

[1/2] hive git commit: HIVE-18224: Introduce interface above driver (Zoltan Haindrich, reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master 3e930bfa3 -> 0e2dab913


http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
index cde7a3e..061817f 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
@@ -31,7 +31,8 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.exec.repl.ReplDumpWork;
 import org.apache.hadoop.hive.ql.parse.repl.PathBuilder;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
@@ -58,7 +59,7 @@ import static org.junit.Assert.fail;
 class WarehouseInstance implements Closeable {
   final String functionsRoot;
   private Logger logger;
-  private Driver driver;
+  private IDriver driver;
   HiveConf hiveConf;
   MiniDFSCluster miniDFSCluster;
   private HiveMetaStoreClient client;
@@ -132,7 +133,7 @@ class WarehouseInstance implements Closeable {
     FileSystem testPathFileSystem = FileSystem.get(testPath.toUri(), hiveConf);
     testPathFileSystem.mkdirs(testPath);
 
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
     SessionState.start(new CliSessionState(hiveConf));
     client = new HiveMetaStoreClient(hiveConf);
     // change the value for the next instance.

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/StorageBasedMetastoreTestBase.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/StorageBasedMetastoreTestBase.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/StorageBasedMetastoreTestBase.java
index dc3af3c..512a68e 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/StorageBasedMetastoreTestBase.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/StorageBasedMetastoreTestBase.java
@@ -31,7 +31,8 @@ import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.security.authorization.AuthorizationPreEventListener;
 import org.apache.hadoop.hive.ql.security.authorization.StorageBasedAuthorizationProvider;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -47,7 +48,7 @@ import org.junit.Before;
 public class StorageBasedMetastoreTestBase {
   protected HiveConf clientHiveConf;
   protected HiveMetaStoreClient msc;
-  protected Driver driver;
+  protected IDriver driver;
   protected UserGroupInformation ugi;
   private static int objNum = 0;
 
@@ -87,7 +88,7 @@ public class StorageBasedMetastoreTestBase {
 
     SessionState.start(new CliSessionState(clientHiveConf));
     msc = new HiveMetaStoreClient(clientHiveConf);
-    driver = new Driver(clientHiveConf);
+    driver = DriverFactory.newDriver(clientHiveConf);
 
     setupFakeUser();
     InjectableDummyAuthenticator.injectMode(false);

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java
index 6a668aa..b91fd29 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java
@@ -33,7 +33,8 @@ import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.security.DummyHiveMetastoreAuthorizationProvider.AuthCallContext;
 import org.apache.hadoop.hive.ql.security.authorization.AuthorizationPreEventListener;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -46,7 +47,7 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 public class TestAuthorizationPreEventListener extends TestCase {
   private HiveConf clientHiveConf;
   private HiveMetaStoreClient msc;
-  private Driver driver;
+  private IDriver driver;
 
   @Override
   protected void setUp() throws Exception {
@@ -74,7 +75,7 @@ public class TestAuthorizationPreEventListener extends TestCase {
 
     SessionState.start(new CliSessionState(clientHiveConf));
     msc = new HiveMetaStoreClient(clientHiveConf);
-    driver = new Driver(clientHiveConf);
+    driver = DriverFactory.newDriver(clientHiveConf);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java
index 57ff8c9..5dbbeb0 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java
@@ -30,7 +30,8 @@ import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -44,7 +45,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 public class TestClientSideAuthorizationProvider extends TestCase {
   protected HiveConf clientHiveConf;
   protected HiveMetaStoreClient msc;
-  protected Driver driver;
+  protected IDriver driver;
   protected UserGroupInformation ugi;
 
 
@@ -85,7 +86,7 @@ public class TestClientSideAuthorizationProvider extends TestCase {
 
     SessionState.start(new CliSessionState(clientHiveConf));
     msc = new HiveMetaStoreClient(clientHiveConf);
-    driver = new Driver(clientHiveConf);
+    driver = DriverFactory.newDriver(clientHiveConf);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java
index edb46fd..6ed2009 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java
@@ -38,7 +38,8 @@ import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.io.HiveInputFormat;
 import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
@@ -71,7 +72,7 @@ public class TestMetastoreAuthorizationProvider extends TestCase {
 
   protected HiveConf clientHiveConf;
   protected HiveMetaStoreClient msc;
-  protected Driver driver;
+  protected IDriver driver;
   protected UserGroupInformation ugi;
 
 
@@ -118,7 +119,7 @@ public class TestMetastoreAuthorizationProvider extends TestCase {
 
     SessionState.start(new CliSessionState(clientHiveConf));
     msc = new HiveMetaStoreClient(clientHiveConf);
-    driver = new Driver(clientHiveConf);
+    driver = DriverFactory.newDriver(clientHiveConf);
   }
 
   protected void setupMetaStoreReadAuthorization() {

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMultiAuthorizationPreEventListener.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMultiAuthorizationPreEventListener.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMultiAuthorizationPreEventListener.java
index 2059370..9e52548 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMultiAuthorizationPreEventListener.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMultiAuthorizationPreEventListener.java
@@ -28,7 +28,8 @@ import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.security.DummyHiveMetastoreAuthorizationProvider.AuthCallContext;
 import org.apache.hadoop.hive.ql.security.authorization.AuthorizationPreEventListener;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -43,7 +44,7 @@ import org.junit.Test;
 public class TestMultiAuthorizationPreEventListener {
   private static HiveConf clientHiveConf;
   private static HiveMetaStoreClient msc;
-  private static Driver driver;
+  private static IDriver driver;
 
   @BeforeClass
   public static void setUp() throws Exception {
@@ -67,7 +68,7 @@ public class TestMultiAuthorizationPreEventListener {
 
     SessionState.start(new CliSessionState(clientHiveConf));
     msc = new HiveMetaStoreClient(clientHiveConf);
-    driver = new Driver(clientHiveConf);
+    driver = DriverFactory.newDriver(clientHiveConf);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
index 19694b0..9b5f7c1 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
@@ -38,7 +38,8 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
@@ -62,7 +63,7 @@ import static org.mockito.Mockito.when;
 public class TestHiveAuthorizerCheckInvocation {
   private final Logger LOG = LoggerFactory.getLogger(this.getClass().getName());;
   protected static HiveConf conf;
-  protected static Driver driver;
+  protected static IDriver driver;
   private static final String tableName = TestHiveAuthorizerCheckInvocation.class.getSimpleName()
       + "Table";
   private static final String viewName = TestHiveAuthorizerCheckInvocation.class.getSimpleName()
@@ -102,7 +103,7 @@ public class TestHiveAuthorizerCheckInvocation {
     conf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict");
 
     SessionState.start(conf);
-    driver = new Driver(conf);
+    driver = DriverFactory.newDriver(conf);
     runCmd("create table " + tableName
         + " (i int, j int, k string) partitioned by (city string, `date` string) ");
     runCmd("create view " + viewName + " as select * from " + tableName);

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerShowFilters.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerShowFilters.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerShowFilters.java
index 5922a8c..cd44795 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerShowFilters.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerShowFilters.java
@@ -31,7 +31,8 @@ import org.apache.hadoop.hive.UtilsForTest;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
 import org.apache.hadoop.hive.ql.security.SessionStateUserAuthenticator;
@@ -48,7 +49,7 @@ import org.mockito.Mockito;
  */
 public class TestHiveAuthorizerShowFilters {
   protected static HiveConf conf;
-  protected static Driver driver;
+  protected static IDriver driver;
   private static final String tableName1 = (TestHiveAuthorizerShowFilters.class.getSimpleName() + "table1")
       .toLowerCase();
   private static final String tableName2 = (TestHiveAuthorizerShowFilters.class.getSimpleName() + "table2")
@@ -123,7 +124,7 @@ public class TestHiveAuthorizerShowFilters {
     UtilsForTest.setNewDerbyDbLocation(conf, TestHiveAuthorizerShowFilters.class.getSimpleName());
 
     SessionState.start(conf);
-    driver = new Driver(conf);
+    driver = DriverFactory.newDriver(conf);
     runCmd("create table " + tableName1
         + " (i int, j int, k string) partitioned by (city string, `date` string) ");
     runCmd("create table " + tableName2 + "(i int)");

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
index 75eeaf6..eae0a33 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
@@ -56,7 +56,8 @@ import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.io.AcidInputFormat;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.HiveInputFormat;
@@ -96,7 +97,7 @@ public class TestCompactor {
   public TemporaryFolder stagingFolder = new TemporaryFolder();
   private HiveConf conf;
   IMetaStoreClient msClient;
-  private Driver driver;
+  private IDriver driver;
 
   @Before
   public void setup() throws Exception {
@@ -122,7 +123,7 @@ public class TestCompactor {
 
     conf = hiveConf;
     msClient = new HiveMetaStoreClient(conf);
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
     SessionState.start(new CliSessionState(hiveConf));
 
 
@@ -1375,7 +1376,7 @@ public class TestCompactor {
   /**
    * convenience method to execute a select stmt and dump results to log file
    */
-  private static List<String> execSelectAndDumpData(String selectStmt, Driver driver, String msg)
+  private static List<String> execSelectAndDumpData(String selectStmt, IDriver driver, String msg)
     throws  Exception {
     executeStatementOnDriver(selectStmt, driver);
     ArrayList<String> valuesReadFromHiveDriver = new ArrayList<String>();
@@ -1391,7 +1392,7 @@ public class TestCompactor {
    * Execute Hive CLI statement
    * @param cmd arbitrary statement to execute
    */
-  static void executeStatementOnDriver(String cmd, Driver driver) throws IOException, CommandNeedRetryException {
+  static void executeStatementOnDriver(String cmd, IDriver driver) throws IOException, CommandNeedRetryException {
     LOG.debug("Executing: " + cmd);
     CommandProcessorResponse cpr = driver.run(cmd);
     if(cpr.getResponseCode() != 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 88034d7..f79dbac 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -182,7 +182,7 @@ public class QTestUtil {
   protected Hive db;
   protected QueryState queryState;
   protected HiveConf conf;
-  private Driver drv;
+  private IDriver drv;
   private BaseSemanticAnalyzer sem;
   protected final boolean overWrite;
   private CliDriver cliDriver;
@@ -558,7 +558,7 @@ public class QTestUtil {
       System.out.println("Setting hive-site: "+HiveConf.getHiveSiteLocation());
     }
 
-    queryState = new QueryState.Builder().withHiveConf(new HiveConf(Driver.class)).build();
+    queryState = new QueryState.Builder().withHiveConf(new HiveConf(IDriver.class)).build();
     conf = queryState.getConf();
     this.hadoopVer = getHadoopMainVersion(hadoopVer);
     qMap = new TreeMap<String, String>();
@@ -989,7 +989,7 @@ public class QTestUtil {
 
     // allocate and initialize a new conf since a test can
     // modify conf by using 'set' commands
-    conf = new HiveConf(Driver.class);
+    conf = new HiveConf(IDriver.class);
     initConf();
     initConfFromSetup();
 
@@ -1127,7 +1127,7 @@ public class QTestUtil {
     SessionState.start(conf);
     conf.set("hive.execution.engine", execEngine);
     db = Hive.get(conf);
-    drv = new Driver(conf);
+    drv = DriverFactory.newDriver(conf);
     pd = new ParseDriver();
     sem = new SemanticAnalyzer(queryState);
   }
@@ -1602,7 +1602,6 @@ public class QTestUtil {
           if (matcher.find()) {
             line = line.replaceAll(prp.pattern.pattern(), prp.replacement);
             partialMaskWasMatched = true;
-            break;
           }
         }
       }
@@ -1713,6 +1712,10 @@ public class QTestUtil {
     ArrayList<PatternReplacementPair> ppm = new ArrayList<>();
     ppm.add(new PatternReplacementPair(Pattern.compile("\\{\"transactionid\":[1-9][0-9]*,\"bucketid\":"),
       "{\"transactionid\":### Masked txnid ###,\"bucketid\":"));
+
+    ppm.add(new PatternReplacementPair(Pattern.compile("attempt_[0-9]+"), "attempt_#ID#"));
+    ppm.add(new PatternReplacementPair(Pattern.compile("vertex_[0-9_]+"), "vertex_#ID#"));
+    ppm.add(new PatternReplacementPair(Pattern.compile("task_[0-9_]+"), "task_#ID#"));
     partialPlanMask = ppm.toArray(new PatternReplacementPair[ppm.size()]);
   }
   /* This list may be modified by specific cli drivers to mask strings that change on every test */

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index b168906..d632ee8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hive.metastore.HiveMetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Schema;
 import org.apache.hadoop.hive.ql.exec.ConditionalTask;
+import org.apache.hadoop.hive.ql.exec.DagUtils;
 import org.apache.hadoop.hive.ql.exec.ExplainTask;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
@@ -103,7 +104,6 @@ import org.apache.hadoop.hive.ql.parse.SemanticAnalyzerFactory;
 import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
-import org.apache.hadoop.hive.ql.processors.CommandProcessor;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.security.authorization.AuthorizationUtils;
 import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
@@ -130,9 +130,7 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
 
 
-public class Driver implements CommandProcessor {
-
-  public static final String MAPREDUCE_WORKFLOW_NODE_NAME = "mapreduce.workflow.node.name";
+public class Driver implements IDriver {
 
   static final private String CLASS_NAME = Driver.class.getName();
   private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
@@ -273,6 +271,7 @@ public class Driver implements CommandProcessor {
   }
 
 
+  @Override
   public Schema getSchema() {
     return schema;
   }
@@ -362,6 +361,7 @@ public class Driver implements CommandProcessor {
   /**
    * Set the maximum number of rows returned by getResults
    */
+  @Override
   public void setMaxRows(int maxRows) {
     this.maxRows = maxRows;
   }
@@ -435,6 +435,8 @@ public class Driver implements CommandProcessor {
    * @param conf The HiveConf which should be used
    * @return The new QueryState object
    */
+  // move to driverFactory ; with those constructors...
+  @Deprecated
   private static QueryState getNewQueryState(HiveConf conf) {
     return new QueryState.Builder().withGenerateNewQueryId(true).withHiveConf(conf).build();
   }
@@ -460,6 +462,7 @@ public class Driver implements CommandProcessor {
    * @param command
    *          The SQL query to compile.
    */
+  @Override
   public int compile(String command) {
     return compile(command, true);
   }
@@ -599,7 +602,7 @@ public class Driver implements CommandProcessor {
       perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.ANALYZE);
       BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(queryState, tree);
       List<HiveSemanticAnalyzerHook> saHooks =
-          hooksLoader.getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, console);
+          hooksLoader.getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, console, HiveSemanticAnalyzerHook.class);
 
       // Flush the metastore cache.  This assures that we don't pick up objects from a previous
       // query running in this same thread.  This has to be done after we get our semantic
@@ -1194,6 +1197,7 @@ public class Driver implements CommandProcessor {
   /**
    * @return The current query plan associated with this Driver, if any.
    */
+  @Override
   public QueryPlan getPlan() {
     return plan;
   }
@@ -1201,6 +1205,7 @@ public class Driver implements CommandProcessor {
   /**
    * @return The current FetchTask associated with the Driver's plan, if any.
    */
+  @Override
   public FetchTask getFetchTask() {
     return fetchTask;
   }
@@ -1368,11 +1373,13 @@ public class Driver implements CommandProcessor {
   }
 
   @Override
+
   public CommandProcessorResponse run(String command)
       throws CommandNeedRetryException {
     return run(command, false);
   }
 
+  @Override
   public CommandProcessorResponse run()
       throws CommandNeedRetryException {
     return run(null, true);
@@ -1439,6 +1446,7 @@ public class Driver implements CommandProcessor {
     }
   }
 
+  @Override
   public CommandProcessorResponse compileAndRespond(String command) {
     try {
       compileInternal(command, false);
@@ -1601,7 +1609,7 @@ public class Driver implements CommandProcessor {
       // Get all the driver run hooks and pre-execute them.
       List<HiveDriverRunHook> driverRunHooks;
       try {
-        driverRunHooks = hooksLoader.getHooks(HiveConf.ConfVars.HIVE_DRIVER_RUN_HOOKS, console);
+        driverRunHooks = hooksLoader.getHooks(HiveConf.ConfVars.HIVE_DRIVER_RUN_HOOKS, console, HiveDriverRunHook.class);
         for (HiveDriverRunHook driverRunHook : driverRunHooks) {
             driverRunHook.preDriverRun(hookContext);
         }
@@ -1841,13 +1849,12 @@ public class Driver implements CommandProcessor {
           ss.getSessionId(), Thread.currentThread().getName(), ss.isHiveServerQuery(), perfLogger, queryInfo);
       hookContext.setHookType(HookContext.HookType.PRE_EXEC_HOOK);
 
-      for (Hook peh : hooksLoader.getHooks(HiveConf.ConfVars.PREEXECHOOKS, console)) {
+      for (Hook peh : hooksLoader.getHooks(HiveConf.ConfVars.PREEXECHOOKS, console, ExecuteWithHookContext.class)) {
         perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.PRE_HOOK + peh.getClass().getName());
 
         ((ExecuteWithHookContext) peh).run(hookContext);
 
         perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PRE_HOOK + peh.getClass().getName());
-
       }
 
       // Trigger query hooks before query execution.
@@ -2033,7 +2040,7 @@ public class Driver implements CommandProcessor {
 
       hookContext.setHookType(HookContext.HookType.POST_EXEC_HOOK);
       // Get all the post execution hooks and execute them.
-      for (Hook peh : hooksLoader.getHooks(HiveConf.ConfVars.POSTEXECHOOKS, console)) {
+      for (Hook peh : hooksLoader.getHooks(HiveConf.ConfVars.POSTEXECHOOKS, console, ExecuteWithHookContext.class)) {
         perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.POST_HOOK + peh.getClass().getName());
 
         ((ExecuteWithHookContext) peh).run(hookContext);
@@ -2200,7 +2207,7 @@ public class Driver implements CommandProcessor {
     hookContext.setErrorMessage(errorMessage);
     hookContext.setException(exception);
     // Get all the failure execution hooks and execute them.
-    for (Hook ofh : hooksLoader.getHooks(HiveConf.ConfVars.ONFAILUREHOOKS, console)) {
+    for (Hook ofh : hooksLoader.getHooks(HiveConf.ConfVars.ONFAILUREHOOKS, console, ExecuteWithHookContext.class)) {
       perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.FAILURE_HOOK + ofh.getClass().getName());
 
       ((ExecuteWithHookContext) ofh).run(hookContext);
@@ -2234,7 +2241,7 @@ public class Driver implements CommandProcessor {
       if (noName) {
         conf.set(MRJobConfig.JOB_NAME, jobname + " (" + tsk.getId() + ")");
       }
-      conf.set(MAPREDUCE_WORKFLOW_NODE_NAME, tsk.getId());
+      conf.set(DagUtils.MAPREDUCE_WORKFLOW_NODE_NAME, tsk.getId());
       Utilities.setWorkflowAdjacencies(conf, plan);
       cxt.incCurJobNo(1);
       console.printInfo("Launching Job " + cxt.getCurJobNo() + " out of " + jobs);
@@ -2259,11 +2266,13 @@ public class Driver implements CommandProcessor {
     return tskRun;
   }
 
+  @Override
   public boolean isFetchingTable() {
     return fetchTask != null;
   }
 
   @SuppressWarnings("unchecked")
+  @Override
   public boolean getResults(List res) throws IOException, CommandNeedRetryException {
     if (lDrvState.driverState == DriverState.DESTROYED || lDrvState.driverState == DriverState.CLOSED) {
       throw new IOException("FAILED: query has been cancelled, closed, or destroyed.");
@@ -2328,6 +2337,7 @@ public class Driver implements CommandProcessor {
     return true;
   }
 
+  @Override
   public void resetFetch() throws IOException {
     if (lDrvState.driverState == DriverState.DESTROYED || lDrvState.driverState == DriverState.CLOSED) {
       throw new IOException("FAILED: driver has been cancelled, closed or destroyed.");
@@ -2350,6 +2360,7 @@ public class Driver implements CommandProcessor {
     return tryCount;
   }
 
+  @Override
   public void setTryCount(int tryCount) {
     this.tryCount = tryCount;
   }
@@ -2445,6 +2456,7 @@ public class Driver implements CommandProcessor {
   }
 
   // is called to stop the query if it is running, clean query results, and release resources.
+  @Override
   public int close() {
     lDrvState.stateLock.lock();
     try {
@@ -2468,6 +2480,7 @@ public class Driver implements CommandProcessor {
 
   // is usually called after close() to commit or rollback a query and end the driver life cycle.
   // do not understand why it is needed and wonder if it could be combined with close.
+  @Override
   public void destroy() {
     lDrvState.stateLock.lock();
     try {
@@ -2502,6 +2515,7 @@ public class Driver implements CommandProcessor {
   }
 
 
+  @Override
   public QueryDisplay getQueryDisplay() {
     return queryDisplay;
   }
@@ -2510,6 +2524,7 @@ public class Driver implements CommandProcessor {
    * Set the HS2 operation handle's guid string
    * @param opId base64 encoded guid string
    */
+  @Override
   public void setOperationId(String opId) {
     this.operationId = opId;
   }
@@ -2517,6 +2532,8 @@ public class Driver implements CommandProcessor {
   /**
    * Resets QueryState to get new queryId on Driver reuse.
    */
+
+  @Override
   public void resetQueryState() {
     // Note: Driver cleanup for reuse at this point is not very clear. The assumption here is that
     // repeated compile/execute calls create new contexts, plan, etc., so we don't need to worry

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/ql/src/java/org/apache/hadoop/hive/ql/DriverFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/DriverFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/DriverFactory.java
new file mode 100644
index 0000000..e7432d7
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/DriverFactory.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+/**
+ * Constructs a driver for ql clients
+ */
+public class DriverFactory {
+
+  public static IDriver newDriver(HiveConf conf) {
+    return newDriver(getNewQueryState(conf), null, null);
+  }
+
+  enum ExecutionStrategy {
+    none {
+      @Override
+      IDriver build(QueryState queryState, String userName, QueryInfo queryInfo) {
+        return new Driver(queryState, userName, queryInfo);
+      }
+    };
+
+    abstract IDriver build(QueryState queryState, String userName, QueryInfo queryInfo);
+  }
+
+  public static IDriver newDriver(QueryState queryState, String userName, QueryInfo queryInfo) {
+    ExecutionStrategy strategy = ExecutionStrategy.none;
+    return strategy.build(queryState, userName, queryInfo);
+  }
+
+  private static QueryState getNewQueryState(HiveConf conf) {
+    // FIXME: isolate hiveConf used for a single query
+    return new QueryState.Builder().withGenerateNewQueryId(true).withHiveConf(conf).build();
+  }
+
+  // FIXME: remove this method ; and use the conf at the callsite...
+  @Deprecated
+  public static IDriver newDriver() {
+    // only CLIDriver enter at this point
+    HiveConf conf = (SessionState.get() != null) ? SessionState.get().getConf() : new HiveConf();
+    return newDriver(conf);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/ql/src/java/org/apache/hadoop/hive/ql/IDriver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/IDriver.java b/ql/src/java/org/apache/hadoop/hive/ql/IDriver.java
new file mode 100644
index 0000000..96f2c4d
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/IDriver.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.ql.exec.FetchTask;
+import org.apache.hadoop.hive.ql.processors.CommandProcessor;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+
+/**
+ * Hive query executer driver
+ */
+public interface IDriver extends CommandProcessor {
+
+  int compile(String string);
+
+  CommandProcessorResponse compileAndRespond(String statement);
+
+  QueryPlan getPlan();
+
+  QueryDisplay getQueryDisplay();
+
+  void setOperationId(String guid64);
+
+  void setTryCount(int maxValue);
+
+  CommandProcessorResponse run() throws CommandNeedRetryException;
+  @Override
+  CommandProcessorResponse run(String command) throws CommandNeedRetryException;
+
+
+  // create some "cover" to the result?
+  boolean getResults(List res) throws IOException, CommandNeedRetryException;
+
+  void setMaxRows(int maxRows);
+
+  FetchTask getFetchTask();
+
+  Schema getSchema();
+
+  boolean isFetchingTable();
+
+  void resetFetch() throws IOException;
+
+  // close&destroy is used in seq coupling most of the time - the difference is either not clear; or not relevant - remove?
+  int close();
+  void destroy();
+
+  void resetQueryState();
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/ql/src/java/org/apache/hadoop/hive/ql/QueryLifeTimeHookRunner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/QueryLifeTimeHookRunner.java b/ql/src/java/org/apache/hadoop/hive/ql/QueryLifeTimeHookRunner.java
index 6bdf7eb..61f9c6d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/QueryLifeTimeHookRunner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/QueryLifeTimeHookRunner.java
@@ -65,7 +65,7 @@ class QueryLifeTimeHookRunner {
     List<QueryLifeTimeHook> propertyDefinedHoooks;
     try {
       propertyDefinedHoooks = hooksLoader.getHooks(
-              HiveConf.ConfVars.HIVE_QUERY_LIFETIME_HOOKS, console);
+          HiveConf.ConfVars.HIVE_QUERY_LIFETIME_HOOKS, console, QueryLifeTimeHook.class);
     } catch (IllegalAccessException | InstantiationException | ClassNotFoundException e) {
       throw new IllegalArgumentException(e);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/ql/src/java/org/apache/hadoop/hive/ql/exec/DagUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DagUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DagUtils.java
index aed1b2c..eadb3be 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DagUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DagUtils.java
@@ -18,22 +18,23 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
-import com.google.common.base.Strings;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 
+import com.google.common.base.Strings;
 
 public class DagUtils {
 
+  public static final String MAPREDUCE_WORKFLOW_NODE_NAME = "mapreduce.workflow.node.name";
+
   public static String getQueryName(Configuration conf) {
     String name = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYNAME);
     if (Strings.isNullOrEmpty(name)) {
       return conf.get(MRJobConfig.JOB_NAME);
     } else {
-      return name + " (" + conf.get(Driver.MAPREDUCE_WORKFLOW_NODE_NAME) + ")";
+      return name + " (" + conf.get(DagUtils.MAPREDUCE_WORKFLOW_NODE_NAME) + ")";
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java
index 4380fe3..3facc24 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java
@@ -32,7 +32,7 @@ public class HookUtils {
     String redactedString = logString;
 
     if (conf != null && logString != null) {
-      List<Redactor> queryRedactors = new HooksLoader(conf).getHooks(ConfVars.QUERYREDACTORHOOKS);
+      List<Redactor> queryRedactors = new HooksLoader(conf).getHooks(ConfVars.QUERYREDACTORHOOKS, Redactor.class);
       for (Redactor redactor : queryRedactors) {
         redactor.setConf(conf);
         redactedString = redactor.redactQuery(redactedString);

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/ql/src/java/org/apache/hadoop/hive/ql/hooks/HooksLoader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HooksLoader.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HooksLoader.java
index 0008726..199004f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HooksLoader.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HooksLoader.java
@@ -58,10 +58,10 @@ public class HooksLoader {
    * @throws IllegalAccessException if the specified class names could not be accessed
    * @throws InstantiationException if the specified class names could not be instantiated
    */
-  public <T extends Hook> List<T> getHooks(HiveConf.ConfVars hookConfVar, SessionState.LogHelper console)
-          throws IllegalAccessException, InstantiationException, ClassNotFoundException {
+  public final <T extends Hook> List<T> getHooks(HiveConf.ConfVars hookConfVar, SessionState.LogHelper console, Class<?> clazz)
+      throws IllegalAccessException, InstantiationException, ClassNotFoundException {
     try {
-      return getHooks(hookConfVar);
+      return getHooks(hookConfVar, clazz);
     } catch (ClassNotFoundException e) {
       console.printError(hookConfVar.varname + " Class not found: " + e.getMessage());
       throw e;
@@ -75,6 +75,9 @@ public class HooksLoader {
    * of each class and then returns them in a {@link List}.
    *
    * @param hookConfVar The configuration variable specifying a comma separated list of the hook class names
+   * @param class2
+   * @param class1
+   * @param console
    *
    * @return a list of the hooks objects, in the order they are listed in the value of hookConfVar
    *
@@ -82,7 +85,7 @@ public class HooksLoader {
    * @throws IllegalAccessException if the specified class names could not be accessed
    * @throws InstantiationException if the specified class names could not be instantiated
    */
-  public <T extends Hook> List<T> getHooks(HiveConf.ConfVars hookConfVar)
+  public <T extends Hook> List<T> getHooks(HiveConf.ConfVars hookConfVar, Class<?> clazz)
           throws InstantiationException, IllegalAccessException, ClassNotFoundException {
     String csHooks = conf.getVar(hookConfVar);
     ImmutableList.Builder<T> hooks = ImmutableList.builder();

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java
index 8d0690d..094a1bd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java
@@ -31,6 +31,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.metadata.*;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
@@ -44,7 +46,7 @@ public final class CommandProcessorFactory {
     // prevent instantiation
   }
 
-  private static final Map<HiveConf, Driver> mapDrivers = Collections.synchronizedMap(new HashMap<HiveConf, Driver>());
+  private static final Map<HiveConf, IDriver> mapDrivers = Collections.synchronizedMap(new HashMap<HiveConf, IDriver>());
 
   public static CommandProcessor get(String cmd)
       throws SQLException {
@@ -121,9 +123,10 @@ public final class CommandProcessorFactory {
       if (conf == null) {
         return new Driver();
       }
-      Driver drv = mapDrivers.get(conf);
+      IDriver drv = mapDrivers.get(conf);
       if (drv == null) {
-        drv = new Driver();
+        // FIXME: why this method didn't use the conf constructor?
+        drv = DriverFactory.newDriver();
         mapDrivers.put(conf, drv);
       } else {
         drv.resetQueryState();
@@ -133,7 +136,7 @@ public final class CommandProcessorFactory {
   }
 
   public static void clean(HiveConf conf) {
-    Driver drv = mapDrivers.get(conf);
+    IDriver drv = mapDrivers.get(conf);
     if (drv != null) {
       drv.destroy();
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
index 773dd51..6d60a50 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -49,7 +49,8 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Schema;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.QueryDisplay;
 import org.apache.hadoop.hive.ql.QueryInfo;
 import org.apache.hadoop.hive.ql.QueryState;
@@ -89,7 +90,7 @@ import org.codehaus.jackson.map.ObjectMapper;
  */
 @SuppressWarnings("deprecation")
 public class SQLOperation extends ExecuteStatementOperation {
-  private Driver driver = null;
+  private IDriver driver = null;
   private CommandProcessorResponse response;
   private TableSchema resultSchema = null;
   private Schema mResultSchema = null;
@@ -159,7 +160,7 @@ public class SQLOperation extends ExecuteStatementOperation {
   public void prepare(QueryState queryState) throws HiveSQLException {
     setState(OperationState.RUNNING);
     try {
-      driver = new Driver(queryState, getParentSession().getUserName(), queryInfo);
+      driver = DriverFactory.newDriver(queryState, getParentSession().getUserName(), queryInfo);
 
       // Start the timer thread for cancelling the query when query timeout is reached
       // queryTimeout == 0 means no timeout

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
index cc4dd52..b903142 100644
--- a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
+++ b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
@@ -654,7 +654,8 @@ public class SessionManager extends CompositeService {
 
   // execute session hooks
   private void executeSessionHooks(HiveSession session) throws Exception {
-    List<HiveSessionHook> sessionHooks = new HooksLoader(hiveConf).getHooks(HiveConf.ConfVars.HIVE_SERVER2_SESSION_HOOK);
+    List<HiveSessionHook> sessionHooks =
+        new HooksLoader(hiveConf).getHooks(HiveConf.ConfVars.HIVE_SERVER2_SESSION_HOOK, HiveSessionHook.class);
     for (HiveSessionHook sessionHook : sessionHooks) {
       sessionHook.run(new HiveSessionHookContextImpl(session));
     }


[2/2] hive git commit: HIVE-18224: Introduce interface above driver (Zoltan Haindrich, reviewed by Ashutosh Chauhan)

Posted by kg...@apache.org.
HIVE-18224: Introduce interface above driver (Zoltan Haindrich, reviewed by Ashutosh Chauhan)

Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>


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

Branch: refs/heads/master
Commit: 0e2dab913989a2ccc4fde1f2ff7dfea097f4d493
Parents: 3e930bf
Author: Zoltan Haindrich <ki...@rxd.hu>
Authored: Fri Dec 22 09:37:52 2017 +0100
Committer: Zoltan Haindrich <ki...@rxd.hu>
Committed: Fri Dec 22 09:38:53 2017 +0100

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/cli/CliDriver.java   | 10 +--
 .../hadoop/hive/cli/TestCliDriverMethods.java   |  4 +-
 .../apache/hive/hcatalog/cli/HCatDriver.java    | 26 ++++++--
 .../hive/hcatalog/cli/TestSemanticAnalysis.java |  7 +-
 .../hive/hcatalog/cli/TestUseDatabase.java      |  7 +-
 .../hive/hcatalog/data/HCatDataCheckUtil.java   | 13 ++--
 .../hive/hcatalog/mapreduce/HCatBaseTest.java   |  7 +-
 .../hcatalog/mapreduce/TestPassProperties.java  |  7 +-
 .../hcatalog/pig/AbstractHCatLoaderTest.java    | 10 +--
 .../hive/hcatalog/pig/TestE2EScenarios.java     |  7 +-
 .../pig/TestHCatLoaderComplexSchema.java        |  7 +-
 .../hcatalog/pig/TestHCatLoaderEncryption.java  | 15 +++--
 .../hive/hcatalog/pig/TestHCatStorerMulti.java  |  7 +-
 .../hcatalog/listener/TestMsgBusConnection.java |  7 +-
 .../listener/TestNotificationListener.java      |  4 +-
 .../hive/hcatalog/streaming/HiveEndPoint.java   |  7 +-
 .../hive/hcatalog/streaming/TestStreaming.java  | 17 ++---
 .../api/repl/commands/TestCommands.java         |  7 +-
 .../hbase/TestPigHBaseStorageHandler.java       |  7 +-
 .../listener/TestDbNotificationListener.java    |  7 +-
 .../mapreduce/TestSequenceFileReadWrite.java    |  7 +-
 .../hadoop/hive/metastore/TestFilterHooks.java  | 13 ++--
 .../hive/metastore/TestMarkPartition.java       |  7 +-
 .../metastore/TestMetaStoreAuthorization.java   |  5 +-
 .../TestMetaStoreEndFunctionListener.java       | 13 ++--
 .../metastore/TestMetaStoreEventListener.java   | 16 ++---
 .../TestMetaStoreEventListenerOnlyOnCommit.java | 13 ++--
 .../metastore/TestMetaStoreInitListener.java    | 13 ++--
 .../metastore/TestMetaStoreListenersError.java  |  6 +-
 .../hive/metastore/TestMetaStoreMetrics.java    |  9 ++-
 .../hive/metastore/TestMetastoreVersion.java    | 30 ++++-----
 .../apache/hadoop/hive/ql/TestAcidOnTez.java    |  8 +--
 .../hadoop/hive/ql/TestCreateUdfEntities.java   |  6 +-
 ...estDDLWithRemoteMetastoreSecondNamenode.java |  4 +-
 .../hadoop/hive/ql/history/TestHiveHistory.java |  5 +-
 .../TestSemanticAnalyzerHookLoading.java        |  5 +-
 .../hive/ql/parse/TestReplicationScenarios.java | 41 ++++++------
 .../hadoop/hive/ql/parse/WarehouseInstance.java |  7 +-
 .../security/StorageBasedMetastoreTestBase.java |  7 +-
 .../TestAuthorizationPreEventListener.java      |  7 +-
 .../TestClientSideAuthorizationProvider.java    |  7 +-
 .../TestMetastoreAuthorizationProvider.java     |  7 +-
 .../TestMultiAuthorizationPreEventListener.java |  7 +-
 .../TestHiveAuthorizerCheckInvocation.java      |  7 +-
 .../plugin/TestHiveAuthorizerShowFilters.java   |  7 +-
 .../hive/ql/txn/compactor/TestCompactor.java    | 11 ++--
 .../org/apache/hadoop/hive/ql/QTestUtil.java    | 13 ++--
 .../java/org/apache/hadoop/hive/ql/Driver.java  | 39 +++++++----
 .../apache/hadoop/hive/ql/DriverFactory.java    | 62 ++++++++++++++++++
 .../java/org/apache/hadoop/hive/ql/IDriver.java | 69 ++++++++++++++++++++
 .../hadoop/hive/ql/QueryLifeTimeHookRunner.java |  2 +-
 .../apache/hadoop/hive/ql/exec/DagUtils.java    |  9 +--
 .../apache/hadoop/hive/ql/hooks/HookUtils.java  |  2 +-
 .../hadoop/hive/ql/hooks/HooksLoader.java       | 11 ++--
 .../ql/processors/CommandProcessorFactory.java  | 11 ++--
 .../service/cli/operation/SQLOperation.java     |  7 +-
 .../service/cli/session/SessionManager.java     |  3 +-
 57 files changed, 440 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
----------------------------------------------------------------------
diff --git a/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java b/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
index bd0b422..2cd3c95 100644
--- a/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
+++ b/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
@@ -67,7 +67,7 @@ import org.apache.hadoop.hive.conf.VariableSubstitution;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.mr.HadoopJobExecHelper;
@@ -182,7 +182,7 @@ public class CliDriver {
     }  else { // local mode
       try {
         CommandProcessor proc = CommandProcessorFactory.get(tokens, (HiveConf) conf);
-        if (proc instanceof Driver) {
+        if (proc instanceof IDriver) {
           // Let Driver strip comments using sql parser
           ret = processLocalCmd(cmd, proc, ss);
         } else {
@@ -227,8 +227,8 @@ public class CliDriver {
       try {
         needRetry = false;
         if (proc != null) {
-          if (proc instanceof Driver) {
-            Driver qp = (Driver) proc;
+          if (proc instanceof IDriver) {
+            IDriver qp = (IDriver) proc;
             PrintStream out = ss.out;
             long start = System.currentTimeMillis();
             if (ss.getIsVerbose()) {
@@ -321,7 +321,7 @@ public class CliDriver {
    * @param qp Driver that executed the command
    * @param out PrintStream which to send output to
    */
-  private void printHeader(Driver qp, PrintStream out) {
+  private void printHeader(IDriver qp, PrintStream out) {
     List<FieldSchema> fieldSchemas = qp.getSchema().getFieldSchemas();
     if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER)
           && fieldSchemas != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java
----------------------------------------------------------------------
diff --git a/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java b/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java
index 8f1c15e..0f7a1ee 100644
--- a/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java
+++ b/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java
@@ -54,7 +54,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Schema;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.util.Shell;
 
@@ -166,7 +166,7 @@ public class TestCliDriverMethods extends TestCase {
         .thenReturn(true);
     cliDriver.setConf(conf);
 
-    Driver proc = mock(Driver.class);
+    IDriver proc = mock(IDriver.class);
 
     CommandProcessorResponse cpr = mock(CommandProcessorResponse.class);
     when(cpr.getResponseCode()).thenReturn(0);

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/HCatDriver.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/HCatDriver.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/HCatDriver.java
index 4873595..e41db34 100644
--- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/HCatDriver.java
+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/HCatDriver.java
@@ -18,13 +18,17 @@
  */
 package org.apache.hive.hcatalog.cli;
 
+import java.io.IOException;
+import java.util.ArrayList;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -32,14 +36,19 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.hcatalog.common.HCatConstants;
 
-public class HCatDriver extends Driver {
+public class HCatDriver {
+
+  private IDriver driver;
+
+  public HCatDriver() {
+    driver = DriverFactory.newDriver();
+  }
 
-  @Override
   public CommandProcessorResponse run(String command) {
 
     CommandProcessorResponse cpr = null;
     try {
-      cpr = super.run(command);
+      cpr = driver.run(command);
     } catch (CommandNeedRetryException e) {
       return new CommandProcessorResponse(-1, e.toString(), "");
     }
@@ -139,4 +148,13 @@ public class HCatDriver extends Driver {
       }
     }
   }
+
+  public int close() {
+    return driver.close();
+  }
+
+  public boolean getResults(ArrayList<String> res) throws IOException, CommandNeedRetryException {
+    return driver.getResults(res);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java
index d249d43..110c57e 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java
@@ -27,7 +27,8 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
@@ -57,7 +58,7 @@ public class TestSemanticAnalysis extends HCatBaseTest {
   private static final Logger LOG = LoggerFactory.getLogger(TestSemanticAnalysis.class);
   private static final String TBL_NAME = "junit_sem_analysis";
 
-  private Driver hcatDriver = null;
+  private IDriver hcatDriver = null;
   private String query;
 
   @Before
@@ -72,7 +73,7 @@ public class TestSemanticAnalysis extends HCatBaseTest {
       hcatConf.set(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK.varname,
           HCatSemanticAnalyzer.class.getName());
       hcatConf.setBoolVar(HiveConf.ConfVars.METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES, false);
-      hcatDriver = new Driver(hcatConf);
+      hcatDriver = DriverFactory.newDriver(hcatConf);
       SessionState.start(new CliSessionState(hcatConf));
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestUseDatabase.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestUseDatabase.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestUseDatabase.java
index 1606982..8bdb210 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestUseDatabase.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestUseDatabase.java
@@ -27,7 +27,8 @@ import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer;
@@ -35,7 +36,7 @@ import org.apache.hive.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer;
 /* Unit test for GitHub Howl issue #3 */
 public class TestUseDatabase extends TestCase {
 
-  private Driver hcatDriver;
+  private IDriver hcatDriver;
 
   @Override
   protected void setUp() throws Exception {
@@ -46,7 +47,7 @@ public class TestUseDatabase extends TestCase {
     hcatConf.set(ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
 
     hcatConf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName());
-    hcatDriver = new Driver(hcatConf);
+    hcatDriver = DriverFactory.newDriver(hcatConf);
     SessionState.start(new CliSessionState(hcatConf));
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/HCatDataCheckUtil.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/HCatDataCheckUtil.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/HCatDataCheckUtil.java
index ff56234..9eb3b7a 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/HCatDataCheckUtil.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/HCatDataCheckUtil.java
@@ -27,7 +27,8 @@ import java.util.Map.Entry;
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.hcatalog.MiniCluster;
 import org.slf4j.Logger;
@@ -40,7 +41,7 @@ public class HCatDataCheckUtil {
 
   private static final Logger LOG = LoggerFactory.getLogger(HCatDataCheckUtil.class);
 
-  public static Driver instantiateDriver(MiniCluster cluster) {
+  public static IDriver instantiateDriver(MiniCluster cluster) {
     HiveConf hiveConf = new HiveConf(HCatDataCheckUtil.class);
     for (Entry e : cluster.getProperties().entrySet()) {
       hiveConf.set(e.getKey().toString(), e.getValue().toString());
@@ -50,7 +51,7 @@ public class HCatDataCheckUtil {
     hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
 
     LOG.debug("Hive conf : {}", hiveConf.getAllProperties());
-    Driver driver = new Driver(hiveConf);
+    IDriver driver = DriverFactory.newDriver(hiveConf);
     SessionState.start(new CliSessionState(hiveConf));
     return driver;
   }
@@ -64,7 +65,7 @@ public class HCatDataCheckUtil {
     MiniCluster.createInputFile(cluster, fileName, input);
   }
 
-  public static void createTable(Driver driver, String tableName, String createTableArgs)
+  public static void createTable(IDriver driver, String tableName, String createTableArgs)
     throws CommandNeedRetryException, IOException {
     String createTable = "create table " + tableName + createTableArgs;
     int retCode = driver.run(createTable).getResponseCode();
@@ -73,11 +74,11 @@ public class HCatDataCheckUtil {
     }
   }
 
-  public static void dropTable(Driver driver, String tablename) throws IOException, CommandNeedRetryException {
+  public static void dropTable(IDriver driver, String tablename) throws IOException, CommandNeedRetryException {
     driver.run("drop table if exists " + tablename);
   }
 
-  public static ArrayList<String> formattedRun(Driver driver, String name, String selectCmd)
+  public static ArrayList<String> formattedRun(IDriver driver, String name, String selectCmd)
     throws CommandNeedRetryException, IOException {
     driver.run(selectCmd);
     ArrayList<String> src_values = new ArrayList<String>();

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatBaseTest.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatBaseTest.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatBaseTest.java
index f78eb15..616391f 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatBaseTest.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatBaseTest.java
@@ -23,7 +23,8 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.hcatalog.common.HCatUtil;
 import org.apache.pig.ExecType;
@@ -49,7 +50,7 @@ public abstract class HCatBaseTest {
   protected static final String TEST_WAREHOUSE_DIR = TEST_DATA_DIR + "/warehouse";
 
   protected HiveConf hiveConf = null;
-  protected Driver driver = null;
+  protected IDriver driver = null;
   protected HiveMetaStoreClient client = null;
 
   @BeforeClass
@@ -66,7 +67,7 @@ public abstract class HCatBaseTest {
   public void setUp() throws Exception {
     if (driver == null) {
       setUpHiveConf();
-      driver = new Driver(hiveConf);
+      driver = DriverFactory.newDriver(hiveConf);
       client = new HiveMetaStoreClient(hiveConf);
       SessionState.start(new CliSessionState(hiveConf));
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java
index 31857bf..6d47b83 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java
@@ -30,7 +30,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.Warehouse;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.NullWritable;
@@ -53,7 +54,7 @@ public class TestPassProperties {
   private static final String TEST_WAREHOUSE_DIR = TEST_DATA_DIR + "/warehouse";
   private static final String INPUT_FILE_NAME = TEST_DATA_DIR + "/input.data";
 
-  private static Driver driver;
+  private static IDriver driver;
   private static PigServer server;
   private static String[] input;
   private static HiveConf hiveConf;
@@ -67,7 +68,7 @@ public class TestPassProperties {
     hiveConf
     .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
         "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
     SessionState.start(new CliSessionState(hiveConf));
 
     new File(TEST_WAREHOUSE_DIR).mkdirs();

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/AbstractHCatLoaderTest.java
----------------------------------------------------------------------
diff --git a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/AbstractHCatLoaderTest.java b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/AbstractHCatLoaderTest.java
index 124a5bc..265b0fe 100644
--- a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/AbstractHCatLoaderTest.java
+++ b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/AbstractHCatLoaderTest.java
@@ -43,7 +43,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 import org.apache.hadoop.mapreduce.Job;
@@ -96,7 +96,7 @@ public abstract class AbstractHCatLoaderTest extends HCatBaseTest {
     dropTable(tablename, driver);
   }
 
-  static void dropTable(String tablename, Driver driver) throws IOException, CommandNeedRetryException {
+  static void dropTable(String tablename, IDriver driver) throws IOException, CommandNeedRetryException {
     driver.run("drop table if exists " + tablename);
   }
 
@@ -104,7 +104,7 @@ public abstract class AbstractHCatLoaderTest extends HCatBaseTest {
     createTable(tablename, schema, partitionedBy, driver, storageFormat);
   }
 
-  static void createTable(String tablename, String schema, String partitionedBy, Driver driver, String storageFormat)
+  static void createTable(String tablename, String schema, String partitionedBy, IDriver driver, String storageFormat)
       throws IOException, CommandNeedRetryException {
     String createTable;
     createTable = "create table " + tablename + "(" + schema + ") ";
@@ -125,7 +125,7 @@ public abstract class AbstractHCatLoaderTest extends HCatBaseTest {
    * Execute Hive CLI statement
    * @param cmd arbitrary statement to execute
    */
-  static void executeStatementOnDriver(String cmd, Driver driver) throws IOException, CommandNeedRetryException {
+  static void executeStatementOnDriver(String cmd, IDriver driver) throws IOException, CommandNeedRetryException {
     LOG.debug("Executing: " + cmd);
     CommandProcessorResponse cpr = driver.run(cmd);
     if(cpr.getResponseCode() != 0) {
@@ -725,7 +725,7 @@ public abstract class AbstractHCatLoaderTest extends HCatBaseTest {
       }
       assertTrue("Expected " + primitiveRows.length + "; found " + numTuplesRead, numTuplesRead == primitiveRows.length);
     }
-    private static void setupAllTypesTable(Driver driver) throws Exception {
+    private static void setupAllTypesTable(IDriver driver) throws Exception {
       String[] primitiveData = new String[primitiveRows.length];
       for (int i = 0; i < primitiveRows.length; i++) {
         Object[] rowData = primitiveRows[i];

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestE2EScenarios.java
----------------------------------------------------------------------
diff --git a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestE2EScenarios.java b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestE2EScenarios.java
index 95a8164..6f372c9 100644
--- a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestE2EScenarios.java
+++ b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestE2EScenarios.java
@@ -30,7 +30,8 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.InputSplit;
@@ -67,7 +68,7 @@ public class TestE2EScenarios {
 
   private static final String TEXTFILE_LOCN = TEST_DATA_DIR + "/textfile";
 
-  private static Driver driver;
+  private static IDriver driver;
 
   protected String storageFormat() {
     return "orc";
@@ -91,7 +92,7 @@ public class TestE2EScenarios {
     hiveConf
     .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
         "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
     SessionState.start(new CliSessionState(hiveConf));
 
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderComplexSchema.java
----------------------------------------------------------------------
diff --git a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderComplexSchema.java b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderComplexSchema.java
index 5c9bae4..5c6626d 100644
--- a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderComplexSchema.java
+++ b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderComplexSchema.java
@@ -32,7 +32,8 @@ import java.util.Set;
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.io.IOConstants;
 import org.apache.hadoop.hive.ql.io.StorageFormats;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
@@ -67,7 +68,7 @@ import static org.junit.Assume.assumeTrue;
 public class TestHCatLoaderComplexSchema {
 
   //private static MiniCluster cluster = MiniCluster.buildCluster();
-  private static Driver driver;
+  private static IDriver driver;
   //private static Properties props;
   private static final Logger LOG = LoggerFactory.getLogger(TestHCatLoaderComplexSchema.class);
 
@@ -114,7 +115,7 @@ public class TestHCatLoaderComplexSchema {
     .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
         "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
 
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
     SessionState.start(new CliSessionState(hiveConf));
     //props = new Properties();
     //props.setProperty("fs.default.name", cluster.getProperties().getProperty("fs.default.name"));

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderEncryption.java
----------------------------------------------------------------------
diff --git a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderEncryption.java b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderEncryption.java
index 496f3c8..d1f46e4 100644
--- a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderEncryption.java
+++ b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderEncryption.java
@@ -28,7 +28,8 @@ import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.io.StorageFormats;
 import org.apache.hadoop.hive.ql.processors.CommandProcessor;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory;
@@ -93,7 +94,7 @@ public class TestHCatLoaderEncryption {
   private HadoopShims.MiniDFSShim dfs = null;
   private HadoopShims.HdfsEncryptionShim hes = null;
   private final String[] testOnlyCommands = new String[]{"crypto"};
-  private Driver driver;
+  private IDriver driver;
   private Map<Integer, Pair<Integer, String>> basicInputData;
   private static List<HCatRecord> readRecords = new ArrayList<HCatRecord>();
 
@@ -115,7 +116,7 @@ public class TestHCatLoaderEncryption {
     dropTable(tablename, driver);
   }
 
-  static void dropTable(String tablename, Driver driver) throws IOException, CommandNeedRetryException {
+  static void dropTable(String tablename, IDriver driver) throws IOException, CommandNeedRetryException {
     driver.run("drop table if exists " + tablename);
   }
 
@@ -123,7 +124,7 @@ public class TestHCatLoaderEncryption {
     createTable(tablename, schema, partitionedBy, driver, storageFormat);
   }
 
-  static void createTable(String tablename, String schema, String partitionedBy, Driver driver, String storageFormat)
+  static void createTable(String tablename, String schema, String partitionedBy, IDriver driver, String storageFormat)
       throws IOException, CommandNeedRetryException {
     String createTable;
     createTable = "create table " + tablename + "(" + schema + ") ";
@@ -142,7 +143,7 @@ public class TestHCatLoaderEncryption {
    * Execute Hive CLI statement
    * @param cmd arbitrary statement to execute
    */
-  static void executeStatementOnDriver(String cmd, Driver driver) throws IOException, CommandNeedRetryException {
+  static void executeStatementOnDriver(String cmd, IDriver driver) throws IOException, CommandNeedRetryException {
     LOG.debug("Executing: " + cmd);
     CommandProcessorResponse cpr = driver.run(cmd);
     if(cpr.getResponseCode() != 0) {
@@ -177,7 +178,7 @@ public class TestHCatLoaderEncryption {
           "_" + salt.getAndIncrement() + "/dfs/");
     }
 
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
 
     initEncryptionShim(hiveConf);
     String encryptedTablePath =  TEST_WAREHOUSE_DIR + "/encryptedTable";
@@ -393,7 +394,7 @@ public class TestHCatLoaderEncryption {
     }
   }
 
-  static void createTableInSpecifiedPath(String tableName, String schema, String path, Driver driver) throws IOException, CommandNeedRetryException {
+  static void createTableInSpecifiedPath(String tableName, String schema, String path, IDriver driver) throws IOException, CommandNeedRetryException {
     String createTableStr;
     createTableStr = "create table " + tableName + "(" + schema + ") location \'" + path + "\'";
     executeStatementOnDriver(createTableStr, driver);

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorerMulti.java
----------------------------------------------------------------------
diff --git a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorerMulti.java b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorerMulti.java
index e2c9b2e..c402ba8 100644
--- a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorerMulti.java
+++ b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorerMulti.java
@@ -31,7 +31,8 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.io.StorageFormats;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
@@ -60,7 +61,7 @@ public class TestHCatStorerMulti {
 
   private static final String BASIC_TABLE = "junit_unparted_basic";
   private static final String PARTITIONED_TABLE = "junit_parted_basic";
-  private static Driver driver;
+  private static IDriver driver;
 
   private static Map<Integer, Pair<Integer, String>> basicInputData;
 
@@ -104,7 +105,7 @@ public class TestHCatStorerMulti {
       hiveConf
       .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
           "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
-      driver = new Driver(hiveConf);
+      driver = DriverFactory.newDriver(hiveConf);
       SessionState.start(new CliSessionState(hiveConf));
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestMsgBusConnection.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestMsgBusConnection.java b/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestMsgBusConnection.java
index 9f7c207..7c5d6d9 100644
--- a/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestMsgBusConnection.java
+++ b/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestMsgBusConnection.java
@@ -37,7 +37,8 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.hcatalog.common.HCatConstants;
 import org.apache.hive.hcatalog.messaging.HCatEventMessage;
@@ -45,7 +46,7 @@ import org.apache.hive.hcatalog.messaging.jms.MessagingUtils;
 
 public class TestMsgBusConnection extends TestCase {
 
-  private Driver driver;
+  private IDriver driver;
   private BrokerService broker;
   private MessageConsumer consumer;
   private static final int TIMEOUT = 2000;
@@ -74,7 +75,7 @@ public class TestMsgBusConnection extends TestCase {
     "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
     hiveConf.set(HCatConstants.HCAT_MSGBUS_TOPIC_PREFIX, "planetlab.hcat");
     SessionState.start(new CliSessionState(hiveConf));
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
   }
 
   private void connectClient() throws JMSException {

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestNotificationListener.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestNotificationListener.java b/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestNotificationListener.java
index 3a33403..e432a3a 100644
--- a/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestNotificationListener.java
+++ b/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestNotificationListener.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.PartitionEventType;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.hcatalog.common.HCatConstants;
 import org.apache.hive.hcatalog.mapreduce.HCatBaseTest;
@@ -114,7 +114,7 @@ public class TestNotificationListener extends HCatBaseTest implements MessageLis
     .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
         "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
     SessionState.start(new CliSessionState(hiveConf));
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
     client = new HiveMetaStoreClient(hiveConf);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
index bccf60c..19ed03a 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
@@ -41,7 +41,8 @@ import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.hcatalog.common.HCatUtil;
 
@@ -451,7 +452,7 @@ public class HiveEndPoint {
       if(SessionState.get() == null) {
         localSession = SessionState.start(new CliSessionState(conf));
       }
-      Driver driver = new Driver(conf);
+      IDriver driver = DriverFactory.newDriver(conf);
 
       try {
         if (LOG.isDebugEnabled()) {
@@ -488,7 +489,7 @@ public class HiveEndPoint {
       }
     }
 
-    private static boolean runDDL(Driver driver, String sql) throws QueryFailedException {
+    private static boolean runDDL(IDriver driver, String sql) throws QueryFailedException {
       int retryCount = 1; // # of times to retry if first attempt fails
       for (int attempt=0; attempt<=retryCount; ++attempt) {
         try {

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
index 101ba1a..07621d4 100644
--- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
@@ -67,7 +67,8 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService;
 import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.BucketCodec;
 import org.apache.hadoop.hive.ql.io.IOConstants;
@@ -153,7 +154,7 @@ public class TestStreaming {
   private static final String COL2 = "msg";
 
   private final HiveConf conf;
-  private Driver driver;
+  private IDriver driver;
   private final IMetaStoreClient msClient;
 
   final String metaStoreURI = null;
@@ -221,7 +222,7 @@ public class TestStreaming {
   @Before
   public void setup() throws Exception {
     SessionState.start(new CliSessionState(conf));
-    driver = new Driver(conf);
+    driver = DriverFactory.newDriver(conf);
     driver.setMaxRows(200002);//make sure Driver returns all results
     // drop and recreate the necessary databases and tables
     dropDB(msClient, dbName);
@@ -2078,7 +2079,7 @@ public class TestStreaming {
 
   ///////// -------- UTILS ------- /////////
   // returns Path of the partition created (if any) else Path of table
-  public static Path createDbAndTable(Driver driver, String databaseName,
+  public static Path createDbAndTable(IDriver driver, String databaseName,
                                       String tableName, List<String> partVals,
                                       String[] colNames, String[] colTypes,
                                       String[] bucketCols,
@@ -2105,14 +2106,14 @@ public class TestStreaming {
     return new Path(tableLoc);
   }
 
-  private static Path addPartition(Driver driver, String tableName, List<String> partVals, String[] partNames) throws QueryFailedException, CommandNeedRetryException, IOException {
+  private static Path addPartition(IDriver driver, String tableName, List<String> partVals, String[] partNames) throws QueryFailedException, CommandNeedRetryException, IOException {
     String partSpec = getPartsSpec(partNames, partVals);
     String addPart = "alter table " + tableName + " add partition ( " + partSpec  + " )";
     runDDL(driver, addPart);
     return getPartitionPath(driver, tableName, partSpec);
   }
 
-  private static Path getPartitionPath(Driver driver, String tableName, String partSpec) throws CommandNeedRetryException, IOException {
+  private static Path getPartitionPath(IDriver driver, String tableName, String partSpec) throws CommandNeedRetryException, IOException {
     ArrayList<String> res = queryTable(driver, "describe extended " + tableName + " PARTITION (" + partSpec + ")");
     String partInfo = res.get(res.size() - 1);
     int start = partInfo.indexOf("location:") + "location:".length();
@@ -2179,7 +2180,7 @@ public class TestStreaming {
     return " partitioned by (" + getTablePartsStr(partNames) + " )";
   }
 
-  private static boolean runDDL(Driver driver, String sql) throws QueryFailedException {
+  private static boolean runDDL(IDriver driver, String sql) throws QueryFailedException {
     LOG.debug(sql);
     System.out.println(sql);
     int retryCount = 1; // # of times to retry if first attempt fails
@@ -2202,7 +2203,7 @@ public class TestStreaming {
   }
 
 
-  public static ArrayList<String> queryTable(Driver driver, String query)
+  public static ArrayList<String> queryTable(IDriver driver, String query)
           throws CommandNeedRetryException, IOException {
     CommandProcessorResponse cpr = driver.run(query);
     if(cpr.getResponseCode() != 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/commands/TestCommands.java
----------------------------------------------------------------------
diff --git a/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/commands/TestCommands.java b/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/commands/TestCommands.java
index b959321..75cbc83 100644
--- a/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/commands/TestCommands.java
+++ b/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/commands/TestCommands.java
@@ -26,7 +26,8 @@ import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.hcatalog.HcatTestUtils;
@@ -67,7 +68,7 @@ public class TestCommands {
   private static Logger LOG = LoggerFactory.getLogger(CommandTestUtils.class.getName());
 
   private static HiveConf hconf;
-  private static Driver driver;
+  private static IDriver driver;
   private static HCatClient client;
   private static String TEST_PATH;
 
@@ -86,7 +87,7 @@ public class TestCommands {
     FileSystem fs = FileSystem.get(testPath.toUri(),hconf);
     fs.mkdirs(testPath);
 
-    driver = new Driver(hconf);
+    driver = DriverFactory.newDriver(hconf);
     SessionState.start(new CliSessionState(hconf));
     client = HCatClient.create(hconf);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/TestPigHBaseStorageHandler.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/TestPigHBaseStorageHandler.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/TestPigHBaseStorageHandler.java
index 120b4af..4778e1f 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/TestPigHBaseStorageHandler.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/TestPigHBaseStorageHandler.java
@@ -48,7 +48,8 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.hcatalog.common.HCatUtil;
@@ -63,7 +64,7 @@ import org.junit.Test;
 public class TestPigHBaseStorageHandler extends SkeletonHBaseTest {
 
   private static HiveConf   hcatConf;
-  private static Driver driver;
+  private static IDriver driver;
   private static String mypath;
 
   private final byte[] FAMILY     = Bytes.toBytes("testFamily");
@@ -93,7 +94,7 @@ public class TestPigHBaseStorageHandler extends SkeletonHBaseTest {
       }
     }
 
-    driver = new Driver(hcatConf);
+    driver = DriverFactory.newDriver(hcatConf);
     SessionState.start(new CliSessionState(hcatConf));
 
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
index 308ab0d..c3b8e8c 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
@@ -94,7 +94,8 @@ import org.apache.hadoop.hive.metastore.messaging.EventMessage.EventType;
 import org.apache.hadoop.hive.metastore.messaging.InsertMessage;
 import org.apache.hadoop.hive.metastore.messaging.MessageDeserializer;
 import org.apache.hadoop.hive.metastore.messaging.MessageFactory;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.hcatalog.api.repl.ReplicationV1CompatRule;
 import org.apache.hive.hcatalog.data.Pair;
@@ -118,7 +119,7 @@ public class TestDbNotificationListener {
   private static final int CLEANUP_SLEEP_TIME = 10;
   private static Map<String, String> emptyParameters = new HashMap<String, String>();
   private static IMetaStoreClient msClient;
-  private static Driver driver;
+  private static IDriver driver;
   private static MessageDeserializer md = null;
   private int startTime;
   private long firstEventId;
@@ -255,7 +256,7 @@ public class TestDbNotificationListener {
         "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
     SessionState.start(new CliSessionState(conf));
     msClient = new HiveMetaStoreClient(conf);
-    driver = new Driver(conf);
+    driver = DriverFactory.newDriver(conf);
     md = MessageFactory.getInstance().getDeserializer();
 
     bcompat = new ReplicationV1CompatRule(msClient, conf, testsToSkipForReplV1BackwardCompatTesting );

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/mapreduce/TestSequenceFileReadWrite.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/mapreduce/TestSequenceFileReadWrite.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/mapreduce/TestSequenceFileReadWrite.java
index d2dbe8f..b7888c6 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/mapreduce/TestSequenceFileReadWrite.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/mapreduce/TestSequenceFileReadWrite.java
@@ -32,7 +32,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.Warehouse;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.NullWritable;
@@ -59,7 +60,7 @@ public class TestSequenceFileReadWrite {
   private File dataDir;
   private String warehouseDir;
   private String inputFileName;
-  private Driver driver;
+  private IDriver driver;
   private PigServer server;
   private String[] input;
   private HiveConf hiveConf;
@@ -79,7 +80,7 @@ public class TestSequenceFileReadWrite {
     hiveConf
     .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
         "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
     SessionState.start(new CliSessionState(hiveConf));
 
     if(!(new File(warehouseDir).mkdirs())) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java
index 91fc706..2382582 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java
@@ -36,18 +36,17 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.PartitionSpec;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-
-import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Lists;
+
 public class TestFilterHooks {
   private static final Logger LOG = LoggerFactory.getLogger(TestFilterHooks.class);
 
@@ -165,7 +164,7 @@ public class TestFilterHooks {
   private static final String INDEX1 = "idx1";
   private static HiveConf hiveConf;
   private static HiveMetaStoreClient msc;
-  private static Driver driver;
+  private static IDriver driver;
 
   @BeforeClass
   public static void setUp() throws Exception {
@@ -183,7 +182,7 @@ public class TestFilterHooks {
 
     SessionState.start(new CliSessionState(hiveConf));
     msc = new HiveMetaStoreClient(hiveConf);
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
 
     driver.run("drop database if exists " + DBNAME1  + " cascade");
     driver.run("drop database if exists " + DBNAME2  + " cascade");

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
index 7cf351f..7b3a896 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
@@ -34,14 +34,15 @@ import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.thrift.TException;
 
 public class TestMarkPartition extends TestCase{
 
   protected HiveConf hiveConf;
-  private Driver driver;
+  private IDriver driver;
 
   @Override
   protected void setUp() throws Exception {
@@ -61,7 +62,7 @@ public class TestMarkPartition extends TestCase{
   TException, NoSuchObjectException, UnknownDBException, UnknownTableException,
   InvalidPartitionException, UnknownPartitionException, InterruptedException {
     HiveMetaStoreClient msc = new HiveMetaStoreClient(hiveConf);
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
     driver.run("drop database if exists hive2215 cascade");
     driver.run("create database hive2215");
     driver.run("use hive2215");

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java
index e7b899b..51c3ccd 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java
@@ -22,8 +22,6 @@ import java.io.IOException;
 import java.net.ServerSocket;
 import java.util.concurrent.TimeUnit;
 
-import junit.framework.TestCase;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -33,7 +31,8 @@ import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+
+import junit.framework.TestCase;
 
 
 public class TestMetaStoreAuthorization extends TestCase {

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java
index e44cfca..1ca18b9 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java
@@ -20,15 +20,14 @@ package org.apache.hadoop.hive.metastore;
 
 
 
-import junit.framework.TestCase;
-
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+
+import junit.framework.TestCase;
 
 /**
  * TestMetaStoreEventListener. Test case for
@@ -37,7 +36,7 @@ import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 public class TestMetaStoreEndFunctionListener extends TestCase {
   private HiveConf hiveConf;
   private HiveMetaStoreClient msc;
-  private Driver driver;
+  private IDriver driver;
 
   @Override
   protected void setUp() throws Exception {
@@ -58,7 +57,7 @@ public class TestMetaStoreEndFunctionListener extends TestCase {
     hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
     SessionState.start(new CliSessionState(hiveConf));
     msc = new HiveMetaStoreClient(hiveConf);
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
index e803106..260c947 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hive.metastore;
 
-import com.google.common.collect.Lists;
-
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -27,7 +25,6 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import junit.framework.TestCase;
 
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -63,12 +60,15 @@ import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreEventContext;
 import org.apache.hadoop.hive.metastore.events.PreLoadPartitionDoneEvent;
-import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.processors.SetProcessor;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
+import com.google.common.collect.Lists;
+
+import junit.framework.TestCase;
+
 /**
  * TestMetaStoreEventListener. Test case for
  * {@link org.apache.hadoop.hive.metastore.MetaStoreEventListener} and
@@ -77,7 +77,7 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 public class TestMetaStoreEventListener extends TestCase {
   private HiveConf hiveConf;
   private HiveMetaStoreClient msc;
-  private Driver driver;
+  private IDriver driver;
 
   private static final String dbName = "hive2038";
   private static final String tblName = "tmptbl";
@@ -107,7 +107,7 @@ public class TestMetaStoreEventListener extends TestCase {
     hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
     SessionState.start(new CliSessionState(hiveConf));
     msc = new HiveMetaStoreClient(hiveConf);
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
 
     driver.run("drop database if exists " + dbName + " cascade");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java
index 9623fed..cc2c5f9 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java
@@ -20,16 +20,15 @@ package org.apache.hadoop.hive.metastore;
 
 import java.util.List;
 
-import junit.framework.TestCase;
-
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.events.ListenerEvent;
-import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
+import junit.framework.TestCase;
+
 /**
  * Ensure that the status of MetaStore events depend on the RawStore's commit status.
  */
@@ -37,7 +36,7 @@ public class TestMetaStoreEventListenerOnlyOnCommit extends TestCase {
 
   private HiveConf hiveConf;
   private HiveMetaStoreClient msc;
-  private Driver driver;
+  private IDriver driver;
 
   @Override
   protected void setUp() throws Exception {
@@ -61,7 +60,7 @@ public class TestMetaStoreEventListenerOnlyOnCommit extends TestCase {
     hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
     SessionState.start(new CliSessionState(hiveConf));
     msc = new HiveMetaStoreClient(hiveConf);
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
 
     DummyListener.notifyList.clear();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreInitListener.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreInitListener.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreInitListener.java
index 4982313..025cc40 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreInitListener.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreInitListener.java
@@ -18,15 +18,14 @@
 
 package org.apache.hadoop.hive.metastore;
 
-import junit.framework.TestCase;
-
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
+import junit.framework.TestCase;
+
 /**
  * TestMetaStoreInitListener. Test case for
  * {@link org.apache.hadoop.hive.metastore.MetaStoreInitListener}
@@ -34,7 +33,7 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 public class TestMetaStoreInitListener extends TestCase {
   private HiveConf hiveConf;
   private HiveMetaStoreClient msc;
-  private Driver driver;
+  private IDriver driver;
 
   @Override
   protected void setUp() throws Exception {
@@ -51,7 +50,7 @@ public class TestMetaStoreInitListener extends TestCase {
     hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
     SessionState.start(new CliSessionState(hiveConf));
     msc = new HiveMetaStoreClient(hiveConf);
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreListenersError.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreListenersError.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreListenersError.java
index 11ebf4d..99b67bb 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreListenersError.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreListenersError.java
@@ -18,12 +18,12 @@
 
 package org.apache.hadoop.hive.metastore;
 
-import junit.framework.Assert;
-import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
 
 /**
  * Test for unwrapping InvocationTargetException, which is thrown from

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java
index 6f5a963..928de6d 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java
@@ -21,9 +21,8 @@ import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.metrics.Metrics;
 import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
-import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -37,7 +36,7 @@ public class TestMetaStoreMetrics {
 
 
   private static HiveConf hiveConf;
-  private static Driver driver;
+  private static IDriver driver;
 
   @BeforeClass
   public static void before() throws Exception {
@@ -56,7 +55,7 @@ public class TestMetaStoreMetrics {
 
     //Increments one HMS connection (Hive.get())
     SessionState.start(new CliSessionState(hiveConf));
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
   }
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetastoreVersion.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetastoreVersion.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetastoreVersion.java
index 0a034d3..7525c57 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetastoreVersion.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetastoreVersion.java
@@ -20,25 +20,25 @@ package org.apache.hadoop.hive.metastore;
 import java.io.File;
 import java.lang.reflect.Field;
 
-import junit.framework.TestCase;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hive.common.util.HiveStringUtils;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.ObjectStore;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hive.common.util.HiveStringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import junit.framework.TestCase;
 
 public class TestMetastoreVersion extends TestCase {
   private static final Logger LOG = LoggerFactory.getLogger(TestMetastoreVersion.class);
   protected HiveConf hiveConf;
-  private Driver driver;
+  private IDriver driver;
   private String metaStoreRoot;
   private String testMetastoreDB;
   private IMetaStoreSchemaInfo metastoreSchemaInfo;
@@ -120,7 +120,7 @@ public class TestMetastoreVersion extends TestCase {
     System.setProperty(HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION_RECORD_VERSION.toString(), "true");
     hiveConf = new HiveConf(this.getClass());
     SessionState.start(new CliSessionState(hiveConf));
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
     driver.run("show tables");
 
     // correct version stored by Metastore during startup
@@ -137,13 +137,13 @@ public class TestMetastoreVersion extends TestCase {
     System.setProperty(HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION.toString(), "false");
     hiveConf = new HiveConf(this.getClass());
     SessionState.start(new CliSessionState(hiveConf));
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
     driver.run("show tables");
 
     ObjectStore.setSchemaVerified(false);
     hiveConf.setBoolVar(HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION, true);
     setVersion(hiveConf, metastoreSchemaInfo.getHiveSchemaVersion());
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
     CommandProcessorResponse proc = driver.run("show tables");
     assertTrue(proc.getResponseCode() == 0);
   }
@@ -156,7 +156,7 @@ public class TestMetastoreVersion extends TestCase {
     System.setProperty(HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION.toString(), "false");
     hiveConf = new HiveConf(this.getClass());
     SessionState.start(new CliSessionState(hiveConf));
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
     driver.run("show tables");
 
     ObjectStore.setSchemaVerified(false);
@@ -164,7 +164,7 @@ public class TestMetastoreVersion extends TestCase {
     hiveConf = new HiveConf(this.getClass());
     setVersion(hiveConf, "fooVersion");
     SessionState.start(new CliSessionState(hiveConf));
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
     CommandProcessorResponse proc = driver.run("show tables");
     assertTrue(proc.getResponseCode() != 0);
   }
@@ -178,14 +178,14 @@ public class TestMetastoreVersion extends TestCase {
     System.setProperty(HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION.toString(), "false");
     hiveConf = new HiveConf(this.getClass());
     SessionState.start(new CliSessionState(hiveConf));
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
     driver.run("show tables");
 
     System.setProperty(HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION.toString(), "true");
     hiveConf = new HiveConf(this.getClass());
     setVersion(hiveConf, "3.9000.0");
     SessionState.start(new CliSessionState(hiveConf));
-    driver = new Driver(hiveConf);
+    driver = DriverFactory.newDriver(hiveConf);
     CommandProcessorResponse proc = driver.run("show tables");
     assertEquals(0, proc.getResponseCode());
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java
index 1fd84ac..d43c239 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java
@@ -76,7 +76,7 @@ public class TestAcidOnTez {
   @Rule
   public TestName testName = new TestName();
   private HiveConf hiveConf;
-  private Driver d;
+  private IDriver d;
   private static enum Table {
     ACIDTBL("acidTbl"),
     ACIDTBLPART("acidTblPart"),
@@ -118,7 +118,7 @@ public class TestAcidOnTez {
       throw new RuntimeException("Could not create " + TEST_WAREHOUSE_DIR);
     }
     SessionState.start(new SessionState(hiveConf));
-    d = new Driver(hiveConf);
+    d = DriverFactory.newDriver(hiveConf);
     dropTables();
     runStatementOnDriver("create table " + Table.ACIDTBL + "(a int, b int) clustered by (a) into " + BUCKET_COUNT + " buckets stored as orc " + getTblProperties());
     runStatementOnDriver("create table " + Table.ACIDTBLPART + "(a int, b int) partitioned by (p string) clustered by (a) into " + BUCKET_COUNT + " buckets stored as orc " + getTblProperties());
@@ -791,7 +791,7 @@ ekoifman:apache-hive-3.0.0-SNAPSHOT-bin ekoifman$ tree  ~/dev/hiverwgit/itests/h
     }
 
     SessionState.start(conf);
-    d = new Driver(conf);
+    d = DriverFactory.newDriver(conf);
   }
 
   // Ideally test like this should be a qfile test. However, the explain output from qfile is always
@@ -886,7 +886,7 @@ ekoifman:apache-hive-3.0.0-SNAPSHOT-bin ekoifman$ tree  ~/dev/hiverwgit/itests/h
    */
   private List<String> runStatementOnDriver(String stmt, HiveConf conf)
       throws Exception {
-    Driver driver = new Driver(conf);
+    IDriver driver = DriverFactory.newDriver(conf);
     driver.setMaxRows(10000);
     CommandProcessorResponse cpr = driver.run(stmt);
     if(cpr.getResponseCode() != 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestCreateUdfEntities.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestCreateUdfEntities.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestCreateUdfEntities.java
index 34f4ed0..1894ba0 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestCreateUdfEntities.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestCreateUdfEntities.java
@@ -28,15 +28,15 @@ import org.junit.Before;
 import org.junit.Test;
 
 public class TestCreateUdfEntities {
-  private Driver driver;
+  private IDriver driver;
   private String funcName = "print_test";
 
   @Before
   public void setUp() throws Exception {
 
-    HiveConf conf = new HiveConf(Driver.class);
+    HiveConf conf = new HiveConf(IDriver.class);
     SessionState.start(conf);
-    driver = new Driver(conf);
+    driver = DriverFactory.newDriver(conf);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java
index 179eed9..cde728e 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java
@@ -64,7 +64,7 @@ public class TestDDLWithRemoteMetastoreSecondNamenode extends TestCase {
   private static Hive db;
   private static FileSystem fs, fs2;
   private static HiveConf jobConf;
-  private static Driver driver;
+  private static IDriver driver;
   private static int tests = 0;
   private static Boolean isInitialized = false;
 
@@ -100,7 +100,7 @@ public class TestDDLWithRemoteMetastoreSecondNamenode extends TestCase {
       fs2Uri = fs2.getUri().toString();
       jobConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, fs2Uri);
 
-      driver = new Driver(jobConf);
+      driver = DriverFactory.newDriver(jobConf);
 
       fs = FileSystem.get(conf);
       if (fs.exists(tmppath) && !fs.getFileStatus(tmppath).isDir()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java
index d73cd64..3d4f24d 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java
@@ -35,7 +35,8 @@ import org.apache.hadoop.hive.common.LogUtils.LogInitializationException;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.Warehouse;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.history.HiveHistory.Keys;
 import org.apache.hadoop.hive.ql.history.HiveHistory.QueryInfo;
 import org.apache.hadoop.hive.ql.history.HiveHistory.TaskInfo;
@@ -143,7 +144,7 @@ public class TestHiveHistory extends TestCase {
       SessionState.start(ss);
 
       String cmd = "select a.key+1 from src a";
-      Driver d = new Driver(conf);
+      IDriver d = DriverFactory.newDriver(conf);
       int ret = d.run(cmd).getResponseCode();
       if (ret != 0) {
         fail("Failed");

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestSemanticAnalyzerHookLoading.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestSemanticAnalyzerHookLoading.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestSemanticAnalyzerHookLoading.java
index 2170ca3..7e59241 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestSemanticAnalyzerHookLoading.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestSemanticAnalyzerHookLoading.java
@@ -25,7 +25,8 @@ import junit.framework.TestCase;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.Warehouse;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
@@ -37,7 +38,7 @@ public class TestSemanticAnalyzerHookLoading extends TestCase {
     conf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname, DummySemanticAnalyzerHook.class.getName());
     conf.set(ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
     SessionState.start(conf);
-    Driver driver = new Driver(conf);
+    IDriver driver = DriverFactory.newDriver(conf);
 
     driver.run("drop table testDL");
     CommandProcessorResponse resp = driver.run("create table testDL (a int) as select * from tbl2");

http://git-wip-us.apache.org/repos/asf/hive/blob/0e2dab91/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
index 0e08007..2bb06d5 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
@@ -51,7 +51,8 @@ import org.apache.hadoop.hive.metastore.messaging.event.filters.EventBoundaryFil
 import org.apache.hadoop.hive.metastore.messaging.event.filters.MessageFormatFilter;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.exec.repl.ReplDumpWork;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -101,12 +102,12 @@ public class TestReplicationScenarios {
 
   private static HiveConf hconf;
   private static int msPort;
-  private static Driver driver;
+  private static IDriver driver;
   private static HiveMetaStoreClient metaStoreClient;
   private static String proxySettingName;
   static HiveConf hconfMirror;
   static int msPortMirror;
-  static Driver driverMirror;
+  static IDriver driverMirror;
   static HiveMetaStoreClient metaStoreClientMirror;
 
   @Rule
@@ -159,7 +160,7 @@ public class TestReplicationScenarios {
     FileSystem fs = FileSystem.get(testPath.toUri(),hconf);
     fs.mkdirs(testPath);
 
-    driver = new Driver(hconf);
+    driver = DriverFactory.newDriver(hconf);
     SessionState.start(new CliSessionState(hconf));
     metaStoreClient = new HiveMetaStoreClient(hconf);
 
@@ -170,7 +171,7 @@ public class TestReplicationScenarios {
     hconfMirror = new HiveConf(hconf);
     hconfMirror.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:"
         + msPortMirror);
-    driverMirror = new Driver(hconfMirror);
+    driverMirror = DriverFactory.newDriver(hconfMirror);
     metaStoreClientMirror = new HiveMetaStoreClient(hconfMirror);
 
     ObjectStore.setTwoMetastoreTesting(true);
@@ -586,7 +587,7 @@ public class TestReplicationScenarios {
             public void run() {
               try {
                 LOG.info("Entered new thread");
-                Driver driver2 = new Driver(hconf);
+                IDriver driver2 = DriverFactory.newDriver(hconf);
                 SessionState.start(new CliSessionState(hconf));
                 CommandProcessorResponse ret = driver2.run("ALTER TABLE " + dbName + ".ptned PARTITION (b=1) RENAME TO PARTITION (b=10)");
                 success = (ret.getException() == null);
@@ -661,7 +662,7 @@ public class TestReplicationScenarios {
             public void run() {
               try {
                 LOG.info("Entered new thread");
-                Driver driver2 = new Driver(hconf);
+                IDriver driver2 = DriverFactory.newDriver(hconf);
                 SessionState.start(new CliSessionState(hconf));
                 CommandProcessorResponse ret = driver2.run("DROP TABLE " + dbName + ".ptned");
                 success = (ret.getException() == null);
@@ -3358,7 +3359,7 @@ public class TestReplicationScenarios {
     verifyRun("SELECT count(*) from " + dbName + "_dupe.unptned", new String[]{"2"}, driverMirror);
   }
 
-  private static String createDB(String name, Driver myDriver) {
+  private static String createDB(String name, IDriver myDriver) {
     LOG.info("Testing " + name);
     String dbName = name + "_" + tid;
     run("CREATE DATABASE " + dbName, myDriver);
@@ -3542,10 +3543,10 @@ public class TestReplicationScenarios {
   }
 
 
-  private String getResult(int rowNum, int colNum, Driver myDriver) throws IOException {
+  private String getResult(int rowNum, int colNum, IDriver myDriver) throws IOException {
     return getResult(rowNum,colNum,false, myDriver);
   }
-  private String getResult(int rowNum, int colNum, boolean reuse, Driver myDriver) throws IOException {
+  private String getResult(int rowNum, int colNum, boolean reuse, IDriver myDriver) throws IOException {
     if (!reuse) {
       lastResults = new ArrayList<String>();
       try {
@@ -3566,7 +3567,7 @@ public class TestReplicationScenarios {
    * Unless for Null Values it actually returns in UpperCase and hence explicitly lowering case
    * before assert.
    */
-  private void verifyResults(String[] data, Driver myDriver) throws IOException {
+  private void verifyResults(String[] data, IDriver myDriver) throws IOException {
     List<String> results = getOutput(myDriver);
     LOG.info("Expecting {}", data);
     LOG.info("Got {}", results);
@@ -3576,7 +3577,7 @@ public class TestReplicationScenarios {
     }
   }
 
-  private List<String> getOutput(Driver myDriver) throws IOException {
+  private List<String> getOutput(IDriver myDriver) throws IOException {
     List<String> results = new ArrayList<>();
     try {
       myDriver.getResults(results);
@@ -3587,7 +3588,7 @@ public class TestReplicationScenarios {
     return results;
   }
 
-  private void printOutput(Driver myDriver) throws IOException {
+  private void printOutput(IDriver myDriver) throws IOException {
     for (String s : getOutput(myDriver)){
       LOG.info(s);
     }
@@ -3648,23 +3649,23 @@ public class TestReplicationScenarios {
     }
   }
 
-  private void verifySetup(String cmd, String[] data, Driver myDriver) throws  IOException {
+  private void verifySetup(String cmd, String[] data, IDriver myDriver) throws  IOException {
     if (VERIFY_SETUP_STEPS){
       run(cmd, myDriver);
       verifyResults(data, myDriver);
     }
   }
 
-  private void verifyRun(String cmd, String data, Driver myDriver) throws IOException {
+  private void verifyRun(String cmd, String data, IDriver myDriver) throws IOException {
     verifyRun(cmd, new String[] { data }, myDriver);
   }
 
-  private void verifyRun(String cmd, String[] data, Driver myDriver) throws IOException {
+  private void verifyRun(String cmd, String[] data, IDriver myDriver) throws IOException {
     run(cmd, myDriver);
     verifyResults(data, myDriver);
   }
 
-  private void verifyFail(String cmd, Driver myDriver) throws RuntimeException {
+  private void verifyFail(String cmd, IDriver myDriver) throws RuntimeException {
     boolean success = false;
     try {
       success = run(cmd, false, myDriver);
@@ -3677,7 +3678,7 @@ public class TestReplicationScenarios {
     assertFalse(success);
   }
 
-  private void verifyRunWithPatternMatch(String cmd, String key, String pattern, Driver myDriver) throws IOException {
+  private void verifyRunWithPatternMatch(String cmd, String key, String pattern, IDriver myDriver) throws IOException {
     run(cmd, myDriver);
     List<String> results = getOutput(myDriver);
     assertTrue(results.size() > 0);
@@ -3692,7 +3693,7 @@ public class TestReplicationScenarios {
     assertTrue(success);
   }
 
-  private static void run(String cmd, Driver myDriver) throws RuntimeException {
+  private static void run(String cmd, IDriver myDriver) throws RuntimeException {
     try {
     run(cmd,false, myDriver); // default arg-less run simply runs, and does not care about failure
     } catch (AssertionError ae){
@@ -3702,7 +3703,7 @@ public class TestReplicationScenarios {
     }
   }
 
-  private static boolean run(String cmd, boolean errorOnFail, Driver myDriver) throws RuntimeException {
+  private static boolean run(String cmd, boolean errorOnFail, IDriver myDriver) throws RuntimeException {
     boolean success = false;
     try {
       CommandProcessorResponse ret = myDriver.run(cmd);