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 2019/12/03 09:30:35 UTC

[hive] 02/02: HIVE-22521: Both Driver and SessionState has a userName (Zoltan Haindrich reviewed by László Bodor)

This is an automated email from the ASF dual-hosted git repository.

kgyrtkirk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git

commit 27a15244a7d7d8ba7c09f580d2b862ca9419057f
Author: Zoltan Haindrich <ki...@rxd.hu>
AuthorDate: Thu Nov 28 14:01:22 2019 +0000

    HIVE-22521: Both Driver and SessionState has a userName (Zoltan Haindrich reviewed by László Bodor)
    
    Signed-off-by: Zoltan Haindrich <zh...@cloudera.com>
---
 .../ql/exec/spark/TestSmallTableCacheEviction.java |  2 +-
 .../ql/exec/spark/TestSparkSessionTimeout.java     |  2 +-
 .../hive/ql/exec/spark/TestSparkStatistics.java    |  2 +-
 ql/src/java/org/apache/hadoop/hive/ql/Driver.java  | 26 +++++++++-------------
 .../org/apache/hadoop/hive/ql/DriverContext.java   |  8 +------
 .../org/apache/hadoop/hive/ql/DriverFactory.java   |  8 +++----
 .../org/apache/hadoop/hive/ql/DriverUtils.java     |  2 +-
 .../hadoop/hive/ql/parse/DDLSemanticAnalyzer.java  |  8 +++----
 .../apache/hadoop/hive/ql/reexec/ReExecDriver.java |  5 ++---
 .../scheduled/ScheduledQueryExecutionService.java  |  2 +-
 .../hadoop/hive/ql/txn/compactor/Worker.java       |  2 +-
 .../hive/ql/udf/generic/GenericUDTFGetSplits.java  |  2 +-
 .../org/apache/hadoop/hive/ql/TestTxnCommands.java |  2 +-
 .../apache/hadoop/hive/ql/TestTxnCommands2.java    |  2 +-
 .../apache/hadoop/hive/ql/TestTxnCommands3.java    |  2 +-
 .../hadoop/hive/ql/TxnCommandsBaseForTests.java    |  2 +-
 .../ql/exec/spark/TestLocalHiveSparkClient.java    |  2 +-
 .../hadoop/hive/ql/lockmgr/TestDbTxnManager2.java  |  4 ++--
 .../hive/service/cli/operation/SQLOperation.java   |  2 +-
 19 files changed, 36 insertions(+), 49 deletions(-)

diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSmallTableCacheEviction.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSmallTableCacheEviction.java
index 7bdb5e5..bf7f3dc 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSmallTableCacheEviction.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSmallTableCacheEviction.java
@@ -104,7 +104,7 @@ public class TestSmallTableCacheEviction {
     return new Driver(new QueryState.Builder()
             .withGenerateNewQueryId(true)
             .withHiveConf(conf).build(),
-            null, null);
+            null);
   }
 
   /**
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkSessionTimeout.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkSessionTimeout.java
index 51bd8a7..842aa9f 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkSessionTimeout.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkSessionTimeout.java
@@ -111,7 +111,7 @@ public class TestSparkSessionTimeout {
       driver = new Driver(new QueryState.Builder()
               .withGenerateNewQueryId(true)
               .withHiveConf(conf).build(),
-              null, null);
+              null);
 
       SparkSession sparkSession = SparkUtilities.getSparkSession(conf, SparkSessionManagerImpl
               .getInstance());
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java
index 2b72d09..9b433a2 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java
@@ -58,7 +58,7 @@ public class TestSparkStatistics {
       driver = new Driver(new QueryState.Builder()
               .withGenerateNewQueryId(true)
               .withHiveConf(conf).build(),
-              null, null);
+              null);
 
       driver.run("create table test (col int)");
       Assert.assertEquals(0, driver.compile("select * from test order by col", true));
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 1574982..a28bf16 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -235,32 +235,26 @@ public class Driver implements IDriver {
 
   @VisibleForTesting
   public Driver(HiveConf conf) {
-    this(new QueryState.Builder().withGenerateNewQueryId(true).withHiveConf(conf).build(), null);
+    this(new QueryState.Builder().withGenerateNewQueryId(true).withHiveConf(conf).build());
   }
 
   // Pass lineageState when a driver instantiates another Driver to run
   // or compile another query
   public Driver(HiveConf conf, Context ctx, LineageState lineageState) {
-    this(getNewQueryState(conf, lineageState), null, null);
+    this(getNewQueryState(conf, lineageState), null);
     this.ctx = ctx;
   }
 
-  // Pass lineageState when a driver instantiates another Driver to run
-  // or compile another query
-  public Driver(HiveConf conf, String userName, LineageState lineageState) {
-    this(getNewQueryState(conf, lineageState), userName, null);
-  }
-
-  public Driver(QueryState queryState, String userName) {
-    this(queryState, userName, null, null);
+  public Driver(QueryState queryState) {
+    this(queryState, null, null);
   }
 
-  public Driver(QueryState queryState, String userName, QueryInfo queryInfo) {
-    this(queryState, userName, queryInfo, null);
+  public Driver(QueryState queryState, QueryInfo queryInfo) {
+    this(queryState, queryInfo, null);
   }
 
-  public Driver(QueryState queryState, String userName, QueryInfo queryInfo, HiveTxnManager txnManager) {
-    driverContext = new DriverContext(queryState, queryInfo, userName, new HookRunner(queryState.getConf(), CONSOLE),
+  public Driver(QueryState queryState, QueryInfo queryInfo, HiveTxnManager txnManager) {
+    driverContext = new DriverContext(queryState, queryInfo, new HookRunner(queryState.getConf(), CONSOLE),
         txnManager);
   }
 
@@ -270,7 +264,7 @@ public class Driver implements IDriver {
    * @param lineageState a LineageState to be set in the new QueryState object
    * @return The new QueryState object
    */
-  private static QueryState getNewQueryState(HiveConf conf, LineageState lineageState) {
+  public static QueryState getNewQueryState(HiveConf conf, LineageState lineageState) {
     return new QueryState.Builder()
         .withGenerateNewQueryId(true)
         .withHiveConf(conf)
@@ -409,7 +403,7 @@ public class Driver implements IDriver {
       HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl();
       if (executeHooks) {
         hookCtx.setConf(driverContext.getConf());
-        hookCtx.setUserName(driverContext.getUserName());
+        hookCtx.setUserName(SessionState.get().getUserName());
         hookCtx.setIpAddress(SessionState.get().getUserIpAddress());
         hookCtx.setCommand(command);
         hookCtx.setHiveOperation(driverContext.getQueryState().getHiveOperation());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/DriverContext.java b/ql/src/java/org/apache/hadoop/hive/ql/DriverContext.java
index 1afcfc8..13e2f29 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/DriverContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/DriverContext.java
@@ -38,7 +38,6 @@ public class DriverContext {
   private final QueryState queryState;
   private final QueryInfo queryInfo;
   private final HiveConf conf;
-  private final String userName;
   private final HookRunner hookRunner;
 
   // Transaction manager the Driver has been initialized with (can be null).
@@ -72,12 +71,11 @@ public class DriverContext {
   private Context backupContext = null;
   private boolean retrial = false;
 
-  public DriverContext(QueryState queryState, QueryInfo queryInfo, String userName, HookRunner hookRunner,
+  public DriverContext(QueryState queryState, QueryInfo queryInfo, HookRunner hookRunner,
       HiveTxnManager initTxnManager) {
     this.queryState = queryState;
     this.queryInfo = queryInfo;
     this.conf = queryState.getConf();
-    this.userName = userName;
     this.hookRunner = hookRunner;
     this.initTxnManager = initTxnManager;
   }
@@ -98,10 +96,6 @@ public class DriverContext {
     return conf;
   }
 
-  public String getUserName() {
-    return userName;
-  }
-
   public HookRunner getHookRunner() {
     return hookRunner;
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/DriverFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/DriverFactory.java
index 963741c..9cca2a4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/DriverFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/DriverFactory.java
@@ -35,13 +35,13 @@ import com.google.common.base.Strings;
 public class DriverFactory {
 
   public static IDriver newDriver(HiveConf conf) {
-    return newDriver(getNewQueryState(conf), null, null);
+    return newDriver(getNewQueryState(conf), null);
   }
 
-  public static IDriver newDriver(QueryState queryState, String userName, QueryInfo queryInfo) {
+  public static IDriver newDriver(QueryState queryState, QueryInfo queryInfo) {
     boolean enabled = queryState.getConf().getBoolVar(ConfVars.HIVE_QUERY_REEXECUTION_ENABLED);
     if (!enabled) {
-      return new Driver(queryState, userName, queryInfo);
+      return new Driver(queryState, queryInfo);
     }
 
     String strategies = queryState.getConf().getVar(ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES);
@@ -54,7 +54,7 @@ public class DriverFactory {
       plugins.add(buildReExecPlugin(string));
     }
 
-    return new ReExecDriver(queryState, userName, queryInfo, plugins);
+    return new ReExecDriver(queryState, queryInfo, plugins);
   }
 
   private static IReExecutionPlugin buildReExecPlugin(String name) throws RuntimeException {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/DriverUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/DriverUtils.java
index 26e904a..aa8e64d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/DriverUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/DriverUtils.java
@@ -48,7 +48,7 @@ public class DriverUtils {
     boolean isOk = false;
     try {
       QueryState qs = new QueryState.Builder().withHiveConf(conf).withGenerateNewQueryId(true).nonIsolated().build();
-      Driver driver = new Driver(qs, user, null, null);
+      Driver driver = new Driver(qs, null, null);
       driver.setCompactionWriteIds(writeIds, compactorTxnId);
       try {
         try {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index 5ebf719..497b17f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -2341,10 +2341,10 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         }
         cmd.append(")");
       }
-      SessionState ss = SessionState.get();
-      // TODO: should this use getUserFromAuthenticator?
-      String uName = (ss == null? null: ss.getUserName());
-      Driver driver = new Driver(conf, uName, queryState.getLineageState());
+      // FIXME: is it ok to have a completely new querystate?
+      QueryState newQueryState = Driver.getNewQueryState(conf, queryState.getLineageState());
+      // FIXME: this driver instance is never closed
+      Driver driver = new Driver(newQueryState);
       int rc = driver.compile(cmd.toString(), false);
       if (rc != 0) {
         throw new SemanticException(ErrorMsg.NO_VALID_PARTN.getMsg());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReExecDriver.java b/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReExecDriver.java
index dea46c0..a32af75 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReExecDriver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReExecDriver.java
@@ -103,10 +103,9 @@ public class ReExecDriver implements IDriver {
     return executionIndex == 0;
   }
 
-  public ReExecDriver(QueryState queryState, String userName, QueryInfo queryInfo,
-      ArrayList<IReExecutionPlugin> plugins) {
+  public ReExecDriver(QueryState queryState, QueryInfo queryInfo, ArrayList<IReExecutionPlugin> plugins) {
     this.queryState = queryState;
-    coreDriver = new Driver(queryState, userName, queryInfo, null);
+    coreDriver = new Driver(queryState, queryInfo, null);
     coreDriver.getHookRunner().addSemanticAnalyzerHook(new HandleReOptimizationExplain());
     this.plugins = plugins;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/scheduled/ScheduledQueryExecutionService.java b/ql/src/java/org/apache/hadoop/hive/ql/scheduled/ScheduledQueryExecutionService.java
index 5dd105f..48bdc97 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/scheduled/ScheduledQueryExecutionService.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/scheduled/ScheduledQueryExecutionService.java
@@ -118,7 +118,7 @@ public class ScheduledQueryExecutionService implements Closeable {
         info.setState(QueryState.EXECUTING);
         reportQueryProgress();
         try (
-          IDriver driver = DriverFactory.newDriver(DriverFactory.getNewQueryState(conf), q.getUser(), null)) {
+          IDriver driver = DriverFactory.newDriver(DriverFactory.getNewQueryState(conf), null)) {
           info.setExecutorQueryId(driver.getQueryState().getQueryId());
           driver.run(q.getQuery());
           info.setState(QueryState.FINISHED);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
index 3270175..749cdb6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
@@ -342,7 +342,7 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread {
         //todo: use DriverUtils.runOnDriver() here
         QueryState queryState = new QueryState.Builder().withGenerateNewQueryId(true).withHiveConf(conf).build();
         SessionState localSession = null;
-        try (Driver d = new Driver(queryState, userName)) {
+        try (Driver d = new Driver(queryState)) {
           if (SessionState.get() == null) {
             localSession = new SessionState(conf);
             SessionState.start(localSession);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
index e95edbf..389f5cc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
@@ -306,7 +306,7 @@ public class GenericUDTFGetSplits extends GenericUDTF {
     // So initialize the new Driver with a new TxnManager so that it does not use the
     // Session TxnManager that is already in use.
     HiveTxnManager txnManager = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
-    Driver driver = new Driver(new QueryState.Builder().withHiveConf(conf).nonIsolated().build(), null, null, txnManager);
+    Driver driver = new Driver(new QueryState.Builder().withHiveConf(conf).nonIsolated().build(), null, txnManager);
     DriverCleanup driverCleanup = new DriverCleanup(driver, txnManager, splitsAppId.toString());
     boolean needsCleanup = true;
     try {
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
index 6e7b201..8421408 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
@@ -272,7 +272,7 @@ public class TestTxnCommands extends TxnCommandsBaseForTests {
         throw new RuntimeException(e);
       }
       QueryState qs = new QueryState.Builder().withHiveConf(hiveConf).nonIsolated().build();
-      try (Driver d = new Driver(qs, null)) {
+      try (Driver d = new Driver(qs)) {
         LOG.info("Ready to run the query: " + query);
         syncThreadStart(cdlIn, cdlOut);
         try {
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
index cbc72b4..c184ce5 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
@@ -160,7 +160,7 @@ public class TestTxnCommands2 {
     }
     SessionState ss = SessionState.start(hiveConf);
     ss.applyAuthorizationPolicy();
-    d = new Driver(new QueryState.Builder().withHiveConf(hiveConf).nonIsolated().build(), null);
+    d = new Driver(new QueryState.Builder().withHiveConf(hiveConf).nonIsolated().build());
     d.setMaxRows(10000);
     dropTables();
     runStatementOnDriver("create table " + Table.ACIDTBL + "(a int, b int) clustered by (a) into " + BUCKET_COUNT + " buckets stored as orc TBLPROPERTIES (" + tableProperties + ")");
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java
index 7b3ab28..908ceb4 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java
@@ -306,7 +306,7 @@ public class TestTxnCommands3 extends TxnCommandsBaseForTests {
     runStatementOnDriver("insert into T values(0,2)");//makes delta_1_1 in T1
     runStatementOnDriver("insert into T values(1,4)");//makes delta_2_2 in T2
 
-    Driver driver2 = new Driver(new QueryState.Builder().withHiveConf(hiveConf).build(), null);
+    Driver driver2 = new Driver(new QueryState.Builder().withHiveConf(hiveConf).build());
     driver2.setMaxRows(10000);
 
     HiveTxnManager txnMgr2 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(hiveConf);
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java b/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
index f01a07e..af14e62 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
@@ -114,7 +114,7 @@ public abstract class TxnCommandsBaseForTests {
     }
     SessionState ss = SessionState.start(hiveConf);
     ss.applyAuthorizationPolicy();
-    d = new Driver(new QueryState.Builder().withHiveConf(hiveConf).nonIsolated().build(), null);
+    d = new Driver(new QueryState.Builder().withHiveConf(hiveConf).nonIsolated().build());
     d.setMaxRows(10000);
     dropTables();
     runStatementOnDriver("create table " + Table.ACIDTBL + "(a int, b int) clustered by (a) into " + BUCKET_COUNT + " buckets stored as orc TBLPROPERTIES ('transactional'='true')");
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestLocalHiveSparkClient.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestLocalHiveSparkClient.java
index 94991d3..3f89b90 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestLocalHiveSparkClient.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestLocalHiveSparkClient.java
@@ -91,7 +91,7 @@ public class TestLocalHiveSparkClient {
     try {
       driver = new Driver(new QueryState.Builder()
           .withGenerateNewQueryId(true)
-          .withHiveConf(conf).build(), null, null);
+          .withHiveConf(conf).build(), null);
 
       SparkSession sparkSession = SparkUtilities.getSparkSession(conf,
           SparkSessionManagerImpl.getInstance());
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
index f9bbb28..23d860f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
@@ -100,8 +100,8 @@ public class TestDbTxnManager2 {
   public void setUp() throws Exception {
     SessionState.start(conf);
     ctx = new Context(conf);
-    driver = new Driver(new QueryState.Builder().withHiveConf(conf).nonIsolated().build(), null);
-    driver2 = new Driver(new QueryState.Builder().withHiveConf(conf).build(), null);
+    driver = new Driver(new QueryState.Builder().withHiveConf(conf).nonIsolated().build());
+    driver2 = new Driver(new QueryState.Builder().withHiveConf(conf).build());
     TxnDbUtil.cleanDb(conf);
     TxnDbUtil.prepDb(conf);
     SessionState ss = SessionState.get();
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 b87b670..468ce10 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
@@ -160,7 +160,7 @@ public class SQLOperation extends ExecuteStatementOperation {
   public void prepare(QueryState queryState) throws HiveSQLException {
     setState(OperationState.RUNNING);
     try {
-      driver = DriverFactory.newDriver(queryState, getParentSession().getUserName(), queryInfo);
+      driver = DriverFactory.newDriver(queryState, queryInfo);
 
       // Start the timer thread for cancelling the query when query timeout is reached
       // queryTimeout == 0 means no timeout