diff --git itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java index 205b0fd0dd..5ec89423bd 100644 --- itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java +++ itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java @@ -22,7 +22,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Driver; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatistic; @@ -68,12 +68,12 @@ public void testSparkStatistics() throws MalformedURLException, CommandProcessor SparkTask sparkTask = sparkTasks.get(0); - DriverContext driverCxt = new DriverContext(driver.getContext()); - driverCxt.prepare(driver.getPlan()); + DriverQueue driverQueue = new DriverQueue(driver.getContext()); + driverQueue.prepare(driver.getPlan()); - sparkTask.initialize(driver.getQueryState(), driver.getPlan(), driverCxt, driver.getContext() + sparkTask.initialize(driver.getQueryState(), driver.getPlan(), driverQueue, driver.getContext() .getOpContext()); - Assert.assertEquals(0, sparkTask.execute(driverCxt)); + Assert.assertEquals(0, sparkTask.execute(driverQueue)); Assert.assertNotNull(sparkTask.getSparkStatistics()); diff --git itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java index e0010b836e..5fce8afe5f 100644 --- itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java +++ itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java @@ -53,7 +53,7 @@ import org.apache.hadoop.hive.metastore.messaging.event.filters.DatabaseAndTableFilter; import org.apache.hadoop.hive.metastore.messaging.event.filters.EventBoundaryFilter; import org.apache.hadoop.hive.metastore.messaging.event.filters.MessageFormatFilter; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.metastore.messaging.json.JSONMessageEncoder; import org.apache.hadoop.hive.metastore.messaging.json.gzip.GzipJSONMessageEncoder; import org.apache.hadoop.hive.ql.DriverFactory; @@ -399,7 +399,7 @@ private Task getReplLoadRootTask(String replicadb, boolean isIncrementalDump, Tu null, null, isIncrementalDump, Long.valueOf(tuple.lastReplId), Collections.emptyList()); Task replLoadTask = TaskFactory.get(replLoadWork, confTemp); - replLoadTask.initialize(null, null, new DriverContext(driver.getContext()), null); + replLoadTask.initialize(null, null, new DriverQueue(driver.getContext()), null); replLoadTask.executeTask(null); Hive.closeCurrent(); return replLoadWork.getRootTask(); diff --git ql/src/java/org/apache/hadoop/hive/ql/Driver.java ql/src/java/org/apache/hadoop/hive/ql/Driver.java index 91910d1c0c..f30628e5bf 100644 --- ql/src/java/org/apache/hadoop/hive/ql/Driver.java +++ ql/src/java/org/apache/hadoop/hive/ql/Driver.java @@ -22,7 +22,6 @@ import java.io.DataInput; import java.io.IOException; import java.io.PrintStream; -import java.io.Serializable; import java.net.InetAddress; import java.util.ArrayList; import java.util.Collection; @@ -135,77 +134,33 @@ public class Driver implements IDriver { - static final private String CLASS_NAME = Driver.class.getName(); + private static final String CLASS_NAME = Driver.class.getName(); private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME); - static final private LogHelper console = new LogHelper(LOG); + private static final LogHelper console = new LogHelper(LOG); private static final int SHUTDOWN_HOOK_PRIORITY = 0; - private final QueryInfo queryInfo; private Runnable shutdownRunner = null; private int maxRows = 100; private ByteStream.Output bos = new ByteStream.Output(); - private final HiveConf conf; private DataInput resStream; private Context ctx; - private DriverContext driverCxt; - private QueryPlan plan; - private Schema schema; - - private FetchTask fetchTask; - private List hiveLocks = new ArrayList(); - - // A limit on the number of threads that can be launched - private int maxthreads; - - private String userName; + private final DriverContext driverContext; + private DriverQueue driverQueue; + private final List hiveLocks = new ArrayList(); // HS2 operation handle guid string private String operationId; - // For WebUI. Kept alive after queryPlan is freed. - private final QueryDisplay queryDisplay = new QueryDisplay(); private DriverState driverState = new DriverState(); - // Query specific info - private final QueryState queryState; - - // Query hooks that execute before compilation and after execution - private HookRunner hookRunner; - - // Transaction manager the Driver has been initialized with (can be null). - // If this is set then this Transaction manager will be used during query - // compilation/execution rather than using the current session's transaction manager. - // This might be needed in a situation where a Driver is nested within an already - // running Driver/query - the nested Driver requires a separate transaction manager - // so as not to conflict with the outer Driver/query which is using the session - // transaction manager. - private final HiveTxnManager initTxnMgr; - - // Transaction manager used for the query. This will be set at compile time based on - // either initTxnMgr or from the SessionState, in that order. - private HiveTxnManager queryTxnMgr; - private StatsSource statsSource; - - // Boolean to store information about whether valid txn list was generated - // for current query. - private boolean validTxnListsGenerated; - - private CacheUsage cacheUsage; - private CacheEntry usedCacheEntry; - private ValidWriteIdList compactionWriteIds = null; - private long compactorTxnId = 0; - - private Context backupContext = null; - private boolean retrial = false; - private boolean checkConcurrency() { - return conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY); + return driverContext.getConf().getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY); } @Override public Schema getSchema() { - return schema; + return driverContext.getSchema(); } public Schema getExplainSchema() { @@ -305,13 +260,9 @@ public Driver(QueryState queryState, String userName, QueryInfo queryInfo) { this(queryState, userName, queryInfo, null); } - public Driver(QueryState queryState, String userName, QueryInfo queryInfo, HiveTxnManager txnMgr) { - this.queryState = queryState; - this.conf = queryState.getConf(); - this.userName = userName; - this.hookRunner = new HookRunner(conf, console); - this.queryInfo = queryInfo; - this.initTxnMgr = txnMgr; + public Driver(QueryState queryState, String userName, QueryInfo queryInfo, HiveTxnManager txnManager) { + driverContext = new DriverContext(queryState, queryInfo, userName, new HookRunner(queryState.getConf(), console), + txnManager); } /** @@ -360,13 +311,13 @@ public void compile(String command, boolean resetTaskIds, boolean deferClose) th public Map getHiveVariable() { return SessionState.get().getHiveVariables(); } - }).substitute(conf, command); + }).substitute(driverContext.getConf(), command); String queryStr = command; try { // command should be redacted to avoid to logging sensitive data - queryStr = HookUtils.redactLogString(conf, command); + queryStr = HookUtils.redactLogString(driverContext.getConf(), command); } catch (Exception e) { LOG.warn("WARNING! Query command could not be redacted." + e); } @@ -384,8 +335,8 @@ public void compile(String command, boolean resetTaskIds, boolean deferClose) th DriverState.setDriverState(driverState); - final String queryId = Strings.isNullOrEmpty(queryState.getQueryId()) ? - QueryPlan.makeQueryId() : queryState.getQueryId(); + final String queryId = Strings.isNullOrEmpty(driverContext.getQueryState().getQueryId()) ? + QueryPlan.makeQueryId() : driverContext.getQueryState().getQueryId(); SparkSession ss = SessionState.get().getSparkSession(); if (ss != null) { @@ -396,12 +347,12 @@ public void compile(String command, boolean resetTaskIds, boolean deferClose) th setTriggerContext(queryId); } //save some info for webUI for use after plan is freed - this.queryDisplay.setQueryStr(queryStr); - this.queryDisplay.setQueryId(queryId); + driverContext.getQueryDisplay().setQueryStr(queryStr); + driverContext.getQueryDisplay().setQueryId(queryId); LOG.info("Compiling command(queryId=" + queryId + "): " + queryStr); - conf.setQueryString(queryStr); + driverContext.getConf().setQueryString(queryStr); // FIXME: sideeffect will leave the last query set at the session level if (SessionState.get() != null) { SessionState.get().getConf().setQueryString(queryStr); @@ -416,19 +367,19 @@ public void compile(String command, boolean resetTaskIds, boolean deferClose) th checkInterrupted("before parsing and analysing the query", null, null); if (ctx == null) { - ctx = new Context(conf); + ctx = new Context(driverContext.getConf()); setTriggerContext(queryId); } - ctx.setHiveTxnManager(queryTxnMgr); - ctx.setStatsSource(statsSource); + ctx.setHiveTxnManager(driverContext.getTxnManager()); + ctx.setStatsSource(driverContext.getStatsSource()); ctx.setCmd(command); ctx.setHDFSCleanup(true); perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.PARSE); // Trigger query hook before compilation - hookRunner.runBeforeParseHook(command); + driverContext.getHookRunner().runBeforeParseHook(command); ASTNode tree; try { @@ -437,11 +388,11 @@ public void compile(String command, boolean resetTaskIds, boolean deferClose) th parseError = true; throw e; } finally { - hookRunner.runAfterParseHook(command, parseError); + driverContext.getHookRunner().runAfterParseHook(command, parseError); } perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PARSE); - hookRunner.runBeforeCompileHook(command); + driverContext.getHookRunner().runBeforeCompileHook(command); // clear CurrentFunctionsInUse set, to capture new set of functions // that SemanticAnalyzer finds are in use SessionState.get().getCurrentFunctionsInUse().clear(); @@ -453,27 +404,27 @@ public void compile(String command, boolean resetTaskIds, boolean deferClose) th // because at that point we need access to the objects. Hive.get().getMSC().flushCache(); - backupContext = new Context(ctx); - boolean executeHooks = hookRunner.hasPreAnalyzeHooks(); + driverContext.setBackupContext(new Context(ctx)); + boolean executeHooks = driverContext.getHookRunner().hasPreAnalyzeHooks(); HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl(); if (executeHooks) { - hookCtx.setConf(conf); - hookCtx.setUserName(userName); + hookCtx.setConf(driverContext.getConf()); + hookCtx.setUserName(driverContext.getUserName()); hookCtx.setIpAddress(SessionState.get().getUserIpAddress()); hookCtx.setCommand(command); - hookCtx.setHiveOperation(queryState.getHiveOperation()); + hookCtx.setHiveOperation(driverContext.getQueryState().getHiveOperation()); - tree = hookRunner.runPreAnalyzeHooks(hookCtx, tree); + tree = driverContext.getHookRunner().runPreAnalyzeHooks(hookCtx, tree); } // Do semantic analysis and plan generation - BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(queryState, tree); + BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(driverContext.getQueryState(), tree); - if (!retrial) { - if ((queryState.getHiveOperation() != null) - && queryState.getHiveOperation().equals(HiveOperation.REPLDUMP)) { - setLastReplIdForDump(queryState.getConf()); + if (!driverContext.isRetrial()) { + if ((driverContext.getQueryState().getHiveOperation() != null) && + driverContext.getQueryState().getHiveOperation().equals(HiveOperation.REPLDUMP)) { + setLastReplIdForDump(driverContext.getQueryState().getConf()); } openTransaction(); generateValidTxnList(); @@ -483,14 +434,14 @@ public void compile(String command, boolean resetTaskIds, boolean deferClose) th if (executeHooks) { hookCtx.update(sem); - hookRunner.runPostAnalyzeHooks(hookCtx, sem.getAllRootTasks()); + driverContext.getHookRunner().runPostAnalyzeHooks(hookCtx, sem.getAllRootTasks()); } - LOG.info("Semantic Analysis Completed (retrial = {})", retrial); + LOG.info("Semantic Analysis Completed (retrial = {})", driverContext.isRetrial()); // Retrieve information about cache usage for the query. - if (conf.getBoolVar(HiveConf.ConfVars.HIVE_QUERY_RESULTS_CACHE_ENABLED)) { - cacheUsage = sem.getCacheUsage(); + if (driverContext.getConf().getBoolVar(HiveConf.ConfVars.HIVE_QUERY_RESULTS_CACHE_ENABLED)) { + driverContext.setCacheUsage(sem.getCacheUsage()); } // validate the plan @@ -500,32 +451,33 @@ public void compile(String command, boolean resetTaskIds, boolean deferClose) th checkInterrupted("after analyzing query.", null, null); // get the output schema - schema = getSchema(sem, conf); - plan = new QueryPlan(queryStr, sem, queryDisplay.getQueryStartTime(), queryId, - queryState.getHiveOperation(), schema); + driverContext.setSchema(getSchema(sem, driverContext.getConf())); + QueryPlan plan = new QueryPlan(queryStr, sem, driverContext.getQueryDisplay().getQueryStartTime(), queryId, + driverContext.getQueryState().getHiveOperation(), driverContext.getSchema()); // save the optimized plan and sql for the explain plan.setOptimizedCBOPlan(ctx.getCalcitePlan()); plan.setOptimizedQueryString(ctx.getOptimizedSql()); + driverContext.setPlan(plan); - conf.set("mapreduce.workflow.id", "hive_" + queryId); - conf.set("mapreduce.workflow.name", queryStr); + driverContext.getConf().set("mapreduce.workflow.id", "hive_" + queryId); + driverContext.getConf().set("mapreduce.workflow.name", queryStr); // initialize FetchTask right here if (plan.getFetchTask() != null) { - plan.getFetchTask().initialize(queryState, plan, null, ctx.getOpContext()); + plan.getFetchTask().initialize(driverContext.getQueryState(), plan, null, ctx.getOpContext()); } //do the authorization check if (!sem.skipAuthorization() && - HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)) { + HiveConf.getBoolVar(driverContext.getConf(), HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)) { try { perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.DO_AUTHORIZATION); // Authorization check for kill query will be in KillQueryImpl // As both admin or operation owner can perform the operation. // Which is not directly supported in authorizer - if (queryState.getHiveOperation() != HiveOperation.KILL_QUERY) { - CommandAuthorizer.doAuthorization(queryState.getHiveOperation(), sem, command); + if (driverContext.getQueryState().getHiveOperation() != HiveOperation.KILL_QUERY) { + CommandAuthorizer.doAuthorization(driverContext.getQueryState().getHiveOperation(), sem, command); } } catch (AuthorizationException authExp) { console.printError("Authorization failed:" + authExp.getMessage() + ". Use SHOW GRANT to get more details."); @@ -535,16 +487,16 @@ public void compile(String command, boolean resetTaskIds, boolean deferClose) th } } - if (conf.getBoolVar(ConfVars.HIVE_LOG_EXPLAIN_OUTPUT) - || conf.getBoolVar(ConfVars.HIVE_SERVER2_WEBUI_EXPLAIN_OUTPUT)) { + if (driverContext.getConf().getBoolVar(ConfVars.HIVE_LOG_EXPLAIN_OUTPUT) + || driverContext.getConf().getBoolVar(ConfVars.HIVE_SERVER2_WEBUI_EXPLAIN_OUTPUT)) { String explainOutput = getExplainOutput(sem, plan, tree); if (explainOutput != null) { - if (conf.getBoolVar(ConfVars.HIVE_LOG_EXPLAIN_OUTPUT)) { + if (driverContext.getConf().getBoolVar(ConfVars.HIVE_LOG_EXPLAIN_OUTPUT)) { LOG.info("EXPLAIN output for queryid " + queryId + " : " + explainOutput); } - if (conf.isWebUiQueryInfoCacheEnabled() - && conf.getBoolVar(ConfVars.HIVE_SERVER2_WEBUI_EXPLAIN_OUTPUT)) { - queryDisplay.setExplainPlan(explainOutput); + if (driverContext.getConf().isWebUiQueryInfoCacheEnabled() + && driverContext.getConf().getBoolVar(ConfVars.HIVE_SERVER2_WEBUI_EXPLAIN_OUTPUT)) { + driverContext.getQueryDisplay().setExplainPlan(explainOutput); } } } @@ -578,7 +530,7 @@ public void compile(String command, boolean resetTaskIds, boolean deferClose) th // before/after execution hook will never be executed. if (!parseError) { try { - hookRunner.runAfterCompilationHook(command, compileError); + driverContext.getHookRunner().runAfterCompilationHook(command, compileError); } catch (Exception e) { LOG.warn("Failed when invoking query after-compilation hook.", e); } @@ -586,7 +538,7 @@ public void compile(String command, boolean resetTaskIds, boolean deferClose) th double duration = perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.COMPILE)/1000.00; ImmutableMap compileHMSTimings = dumpMetaCallTimingWithoutEx("compilation"); - queryDisplay.setHmsTimings(QueryDisplay.Phase.COMPILATION, compileHMSTimings); + driverContext.getQueryDisplay().setHmsTimings(QueryDisplay.Phase.COMPILATION, compileHMSTimings); boolean isInterrupted = driverState.isAborted(); if (isInterrupted && !deferClose) { @@ -606,15 +558,15 @@ public void compile(String command, boolean resetTaskIds, boolean deferClose) th private void createTransactionManager() throws CommandProcessorException { try { // Initialize the transaction manager. This must be done before analyze is called. - if (initTxnMgr != null) { - queryTxnMgr = initTxnMgr; - } else { - queryTxnMgr = SessionState.get().initTxnMgr(conf); - } - if (queryTxnMgr instanceof Configurable) { - ((Configurable) queryTxnMgr).setConf(conf); + HiveTxnManager queryTxnManager = (driverContext.getInitTxnManager() != null) ? + driverContext.getInitTxnManager() : + SessionState.get().initTxnMgr(driverContext.getConf()); + + if (queryTxnManager instanceof Configurable) { + ((Configurable) queryTxnManager).setConf(driverContext.getConf()); } - queryState.setTxnManager(queryTxnMgr); + driverContext.setTxnManager(queryTxnManager); + driverContext.getQueryState().setTxnManager(queryTxnManager); // In case when user Ctrl-C twice to kill Hive CLI JVM, we want to release locks // if compile is being called multiple times, clear the old shutdownhook @@ -623,7 +575,7 @@ private void createTransactionManager() throws CommandProcessorException { @Override public void run() { try { - releaseLocksAndCommitOrRollback(false, queryTxnMgr); + releaseLocksAndCommitOrRollback(false, driverContext.getTxnManager()); } catch (LockException e) { LOG.warn("Exception when releasing locks in ShutdownHook for Driver: " + e.getMessage()); @@ -646,12 +598,12 @@ public void run() { // required. private boolean isValidTxnListState() throws LockException { // 1) Get valid txn list. - String txnString = conf.get(ValidTxnList.VALID_TXNS_KEY); + String txnString = driverContext.getConf().get(ValidTxnList.VALID_TXNS_KEY); if (txnString == null) { // Not a transactional op, nothing more to do return true; } - ValidTxnList currentTxnList = queryTxnMgr.getValidTxns(); + ValidTxnList currentTxnList = driverContext.getTxnManager().getValidTxns(); String currentTxnString = currentTxnList.toString(); if (currentTxnString.equals(txnString)) { // Still valid, nothing more to do @@ -693,15 +645,15 @@ private boolean isValidTxnListState() throws LockException { } } // 3) Get txn tables that are being written - String txnWriteIdListStr = conf.get(ValidTxnWriteIdList.VALID_TABLES_WRITEIDS_KEY); + String txnWriteIdListStr = driverContext.getConf().get(ValidTxnWriteIdList.VALID_TABLES_WRITEIDS_KEY); if (txnWriteIdListStr == null || txnWriteIdListStr.length() == 0) { // Nothing to check return true; } ValidTxnWriteIdList txnWriteIdList = new ValidTxnWriteIdList(txnWriteIdListStr); - List> writtenTables = getWrittenTableList(plan); + List> writtenTables = getWrittenTableList(driverContext.getPlan()); ValidTxnWriteIdList currentTxnWriteIds = - queryTxnMgr.getValidWriteIds( + driverContext.getTxnManager().getValidWriteIds( writtenTables.stream() .filter(e -> AcidUtils.isTransactionalTable(e.getRight())) .map(e -> e.getLeft()) @@ -741,10 +693,10 @@ private void setTriggerContext(final String queryId) { // query info is created by SQLOperation which will have start time of the operation. When JDBC Statement is not // used queryInfo will be null, in which case we take creation of Driver instance as query start time (which is also // the time when query display object is created) - if (queryInfo != null) { - queryStartTime = queryInfo.getBeginTime(); + if (driverContext.getQueryInfo() != null) { + queryStartTime = driverContext.getQueryInfo().getBeginTime(); } else { - queryStartTime = queryDisplay.getQueryStartTime(); + queryStartTime = driverContext.getQueryDisplay().getQueryStartTime(); } WmContext wmContext = new WmContext(queryStartTime, queryId); ctx.setWmContext(wmContext); @@ -767,9 +719,10 @@ private void setLastReplIdForDump(HiveConf conf) throws HiveException, TExceptio } private void openTransaction() throws LockException, CommandProcessorException { - if (checkConcurrency() && startImplicitTxn(queryTxnMgr) && !queryTxnMgr.isTxnOpen()) { + if (checkConcurrency() && startImplicitTxn(driverContext.getTxnManager()) && + !driverContext.getTxnManager().isTxnOpen()) { String userFromUGI = getUserFromUGI(); - queryTxnMgr.openTxn(ctx, userFromUGI); + driverContext.getTxnManager().openTxn(ctx, userFromUGI); } } @@ -778,12 +731,12 @@ private void generateValidTxnList() throws LockException { // compilation and processing. We only do this if 1) a transaction // was already opened and 2) the list has not been recorded yet, // e.g., by an explicit open transaction command. - validTxnListsGenerated = false; - String currentTxnString = conf.get(ValidTxnList.VALID_TXNS_KEY); - if (queryTxnMgr.isTxnOpen() && (currentTxnString == null || currentTxnString.isEmpty())) { + driverContext.setValidTxnListsGenerated(false); + String currentTxnString = driverContext.getConf().get(ValidTxnList.VALID_TXNS_KEY); + if (driverContext.getTxnManager().isTxnOpen() && (currentTxnString == null || currentTxnString.isEmpty())) { try { - recordValidTxns(queryTxnMgr); - validTxnListsGenerated = true; + recordValidTxns(driverContext.getTxnManager()); + driverContext.setValidTxnListsGenerated(true); } catch (LockException e) { LOG.error("Exception while acquiring valid txn list", e); throw e; @@ -794,11 +747,12 @@ private void generateValidTxnList() throws LockException { private boolean startImplicitTxn(HiveTxnManager txnManager) throws LockException { boolean shouldOpenImplicitTxn = !ctx.isExplainPlan(); //this is dumb. HiveOperation is not always set. see HIVE-16447/HIVE-16443 - switch (queryState.getHiveOperation() == null ? HiveOperation.QUERY : queryState.getHiveOperation()) { + HiveOperation hiveOperation = driverContext.getQueryState().getHiveOperation(); + switch (hiveOperation == null ? HiveOperation.QUERY : hiveOperation) { case COMMIT: case ROLLBACK: if(!txnManager.isTxnOpen()) { - throw new LockException(null, ErrorMsg.OP_NOT_ALLOWED_WITHOUT_TXN, queryState.getHiveOperation().getOperationName()); + throw new LockException(null, ErrorMsg.OP_NOT_ALLOWED_WITHOUT_TXN, hiveOperation.getOperationName()); } case SWITCHDATABASE: case SET_AUTOCOMMIT: @@ -863,18 +817,18 @@ private String getExplainOutput(BaseSemanticAnalyzer sem, QueryPlan plan, ASTNode astTree) throws IOException { String ret = null; ExplainTask task = new ExplainTask(); - task.initialize(queryState, plan, null, ctx.getOpContext()); + task.initialize(driverContext.getQueryState(), plan, null, ctx.getOpContext()); ByteArrayOutputStream baos = new ByteArrayOutputStream(); PrintStream ps = new PrintStream(baos); try { List> rootTasks = sem.getAllRootTasks(); - if (conf.getBoolVar(ConfVars.HIVE_SERVER2_WEBUI_SHOW_GRAPH)) { + if (driverContext.getConf().getBoolVar(ConfVars.HIVE_SERVER2_WEBUI_SHOW_GRAPH)) { JSONObject jsonPlan = task.getJSONPlan( null, rootTasks, sem.getFetchTask(), true, true, true, sem.getCboInfo(), plan.getOptimizedCBOPlan(), plan.getOptimizedQueryString()); if (jsonPlan.getJSONObject(ExplainTask.STAGE_DEPENDENCIES) != null && jsonPlan.getJSONObject(ExplainTask.STAGE_DEPENDENCIES).length() <= - conf.getIntVar(ConfVars.HIVE_SERVER2_WEBUI_MAX_GRAPH_SIZE)) { + driverContext.getConf().getIntVar(ConfVars.HIVE_SERVER2_WEBUI_MAX_GRAPH_SIZE)) { ret = jsonPlan.toString(); } else { ret = null; @@ -893,7 +847,7 @@ private String getExplainOutput(BaseSemanticAnalyzer sem, QueryPlan plan, @Override public HiveConf getConf() { - return conf; + return driverContext.getConf(); } /** @@ -901,7 +855,7 @@ public HiveConf getConf() { */ @Override public QueryPlan getPlan() { - return plan; + return driverContext.getPlan(); } /** @@ -909,33 +863,33 @@ public QueryPlan getPlan() { */ @Override public FetchTask getFetchTask() { - return fetchTask; + return driverContext.getFetchTask(); } // Write the current set of valid transactions into the conf file private void recordValidTxns(HiveTxnManager txnMgr) throws LockException { - String oldTxnString = conf.get(ValidTxnList.VALID_TXNS_KEY); + String oldTxnString = driverContext.getConf().get(ValidTxnList.VALID_TXNS_KEY); if ((oldTxnString != null) && (oldTxnString.length() > 0)) { throw new IllegalStateException("calling recordValidTxn() more than once in the same " + JavaUtils.txnIdToString(txnMgr.getCurrentTxnId())); } ValidTxnList txnList = txnMgr.getValidTxns(); String txnStr = txnList.toString(); - conf.set(ValidTxnList.VALID_TXNS_KEY, txnStr); + driverContext.getConf().set(ValidTxnList.VALID_TXNS_KEY, txnStr); LOG.debug("Encoding valid txns info " + txnStr + " txnid:" + txnMgr.getCurrentTxnId()); } // Write the current set of valid write ids for the operated acid tables into the conf file so // that it can be read by the input format. private ValidTxnWriteIdList recordValidWriteIds(HiveTxnManager txnMgr) throws LockException { - String txnString = conf.get(ValidTxnList.VALID_TXNS_KEY); + String txnString = driverContext.getConf().get(ValidTxnList.VALID_TXNS_KEY); if ((txnString == null) || (txnString.isEmpty())) { throw new IllegalStateException("calling recordValidWritsIdss() without initializing ValidTxnList " + JavaUtils.txnIdToString(txnMgr.getCurrentTxnId())); } - List txnTables = getTransactionalTableList(plan); + List txnTables = getTransactionalTableList(driverContext.getPlan()); ValidTxnWriteIdList txnWriteIds = null; - if (compactionWriteIds != null) { + if (driverContext.getCompactionWriteIds() != null) { /** * This is kludgy: here we need to read with Compactor's snapshot/txn * rather than the snapshot of the current {@code txnMgr}, in effect @@ -947,21 +901,21 @@ private ValidTxnWriteIdList recordValidWriteIds(HiveTxnManager txnMgr) throws Lo throw new LockException("Unexpected tables in compaction: " + txnTables); } String fullTableName = txnTables.get(0); - txnWriteIds = new ValidTxnWriteIdList(compactorTxnId); - txnWriteIds.addTableValidWriteIdList(compactionWriteIds); + txnWriteIds = new ValidTxnWriteIdList(driverContext.getCompactorTxnId()); + txnWriteIds.addTableValidWriteIdList(driverContext.getCompactionWriteIds()); } else { txnWriteIds = txnMgr.getValidWriteIds(txnTables, txnString); } String writeIdStr = txnWriteIds.toString(); - conf.set(ValidTxnWriteIdList.VALID_TABLES_WRITEIDS_KEY, writeIdStr); - if (plan.getFetchTask() != null) { + driverContext.getConf().set(ValidTxnWriteIdList.VALID_TABLES_WRITEIDS_KEY, writeIdStr); + if (driverContext.getPlan().getFetchTask() != null) { /** * This is needed for {@link HiveConf.ConfVars.HIVEFETCHTASKCONVERSION} optimization which * initializes JobConf in FetchOperator before recordValidTxns() but this has to be done * after locks are acquired to avoid race conditions in ACID. * This case is supported only for single source query. */ - Operator source = plan.getFetchTask().getWork().getSource(); + Operator source = driverContext.getPlan().getFetchTask().getWork().getSource(); if (source instanceof TableScanOperator) { TableScanOperator tsOp = (TableScanOperator)source; String fullTableName = AcidUtils.getFullTableName(tsOp.getConf().getDatabaseName(), @@ -972,7 +926,7 @@ private ValidTxnWriteIdList recordValidWriteIds(HiveTxnManager txnMgr) throws Lo + " is missing from the ValidWriteIdList config: " + writeIdStr); } if (writeIdList != null) { - plan.getFetchTask().setValidWriteIdList(writeIdList.toString()); + driverContext.getPlan().getFetchTask().setValidWriteIdList(writeIdList.toString()); } } } @@ -1048,7 +1002,7 @@ private String getUserFromUGI() throws CommandProcessorException { // Don't use the userName member, as it may or may not have been set. Get the value from // conf, which calls into getUGI to figure out who the process is running as. try { - return conf.getUser(); + return driverContext.getConf().getUser(); } catch (IOException e) { String errorMessage = "FAILED: Error in determining user while acquiring locks: " + e.getMessage(); console.printError(errorMessage, "\n" + StringUtils.stringifyException(e)); @@ -1069,7 +1023,7 @@ private void acquireLocks() throws CommandProcessorException { PerfLogger perfLogger = SessionState.getPerfLogger(); perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.ACQUIRE_READ_WRITE_LOCKS); - if(!queryTxnMgr.isTxnOpen() && queryTxnMgr.supportsAcid()) { + if(!driverContext.getTxnManager().isTxnOpen() && driverContext.getTxnManager().supportsAcid()) { /*non acid txn managers don't support txns but fwd lock requests to lock managers acid txn manager requires all locks to be associated with a txn so if we end up here w/o an open txn it's because we are processing something like "use @@ -1080,15 +1034,15 @@ private void acquireLocks() throws CommandProcessorException { String userFromUGI = getUserFromUGI(); // Set the table write id in all of the acid file sinks - if (!plan.getAcidSinks().isEmpty()) { - List acidSinks = new ArrayList<>(plan.getAcidSinks()); + if (!driverContext.getPlan().getAcidSinks().isEmpty()) { + List acidSinks = new ArrayList<>(driverContext.getPlan().getAcidSinks()); //sorting makes tests easier to write since file names and ROW__IDs depend on statementId //so this makes (file name -> data) mapping stable acidSinks.sort((FileSinkDesc fsd1, FileSinkDesc fsd2) -> fsd1.getDirName().compareTo(fsd2.getDirName())); for (FileSinkDesc desc : acidSinks) { TableDesc tableInfo = desc.getTableInfo(); - long writeId = queryTxnMgr.getTableWriteId(Utilities.getDatabaseName(tableInfo.getTableName()), + long writeId = driverContext.getTxnManager().getTableWriteId(Utilities.getDatabaseName(tableInfo.getTableName()), Utilities.getTableName(tableInfo.getTableName())); desc.setTableWriteId(writeId); @@ -1098,48 +1052,49 @@ private void acquireLocks() throws CommandProcessorException { * Insert ... Select ... Union All Select ... using * {@link org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator#UNION_SUDBIR_PREFIX} */ - desc.setStatementId(queryTxnMgr.getStmtIdAndIncrement()); + desc.setStatementId(driverContext.getTxnManager().getStmtIdAndIncrement()); String unionAllSubdir = "/" + AbstractFileMergeOperator.UNION_SUDBIR_PREFIX; if(desc.getInsertOverwrite() && desc.getDirName().toString().contains(unionAllSubdir) && desc.isFullAcidTable()) { - throw new UnsupportedOperationException("QueryId=" + plan.getQueryId() + + throw new UnsupportedOperationException("QueryId=" + driverContext.getPlan().getQueryId() + " is not supported due to OVERWRITE and UNION ALL. Please use truncate + insert"); } } } - if (plan.getAcidAnalyzeTable() != null) { + if (driverContext.getPlan().getAcidAnalyzeTable() != null) { // Allocate write ID for the table being analyzed. - Table t = plan.getAcidAnalyzeTable().getTable(); - queryTxnMgr.getTableWriteId(t.getDbName(), t.getTableName()); + Table t = driverContext.getPlan().getAcidAnalyzeTable().getTable(); + driverContext.getTxnManager().getTableWriteId(t.getDbName(), t.getTableName()); } - DDLDescWithWriteId acidDdlDesc = plan.getAcidDdlDesc(); + DDLDescWithWriteId acidDdlDesc = driverContext.getPlan().getAcidDdlDesc(); boolean hasAcidDdl = acidDdlDesc != null && acidDdlDesc.mayNeedWriteId(); if (hasAcidDdl) { String fqTableName = acidDdlDesc.getFullTableName(); - long writeId = queryTxnMgr.getTableWriteId( + long writeId = driverContext.getTxnManager().getTableWriteId( Utilities.getDatabaseName(fqTableName), Utilities.getTableName(fqTableName)); acidDdlDesc.setWriteId(writeId); } /*It's imperative that {@code acquireLocks()} is called for all commands so that HiveTxnManager can transition its state machine correctly*/ - queryTxnMgr.acquireLocks(plan, ctx, userFromUGI, driverState); + driverContext.getTxnManager().acquireLocks(driverContext.getPlan(), ctx, userFromUGI, driverState); final List locks = ctx.getHiveLocks(); - LOG.info("Operation {} obtained {} locks", plan.getOperation(), + LOG.info("Operation {} obtained {} locks", driverContext.getPlan().getOperation(), ((locks == null) ? 0 : locks.size())); // This check is for controlling the correctness of the current state - if (queryTxnMgr.recordSnapshot(plan) && !validTxnListsGenerated) { + if (driverContext.getTxnManager().recordSnapshot(driverContext.getPlan()) && + !driverContext.isValidTxnListsGenerated()) { throw new IllegalStateException( "Need to record valid WriteID list but there is no valid TxnID list (" + - JavaUtils.txnIdToString(queryTxnMgr.getCurrentTxnId()) + - ", queryId:" + plan.getQueryId() + ")"); + JavaUtils.txnIdToString(driverContext.getTxnManager().getCurrentTxnId()) + + ", queryId:" + driverContext.getPlan().getQueryId() + ")"); } - if (plan.hasAcidResourcesInQuery() || hasAcidDdl) { - recordValidWriteIds(queryTxnMgr); + if (driverContext.getPlan().hasAcidResourcesInQuery() || hasAcidDdl) { + recordValidWriteIds(driverContext.getTxnManager()); } } catch (Exception e) { @@ -1152,7 +1107,7 @@ private void acquireLocks() throws CommandProcessorException { } public void releaseLocksAndCommitOrRollback(boolean commit) throws LockException { - releaseLocksAndCommitOrRollback(commit, queryTxnMgr); + releaseLocksAndCommitOrRollback(commit, driverContext.getTxnManager()); } /** @@ -1168,20 +1123,21 @@ public void releaseLocksAndCommitOrRollback(boolean commit, HiveTxnManager txnMa HiveTxnManager txnMgr; if (txnManager == null) { // Default to driver's txn manager if no txn manager specified - txnMgr = queryTxnMgr; + txnMgr = driverContext.getTxnManager(); } else { txnMgr = txnManager; } // If we've opened a transaction we need to commit or rollback rather than explicitly // releasing the locks. - conf.unset(ValidTxnList.VALID_TXNS_KEY); - conf.unset(ValidTxnWriteIdList.VALID_TABLES_WRITEIDS_KEY); + driverContext.getConf().unset(ValidTxnList.VALID_TXNS_KEY); + driverContext.getConf().unset(ValidTxnWriteIdList.VALID_TABLES_WRITEIDS_KEY); if(!checkConcurrency()) { return; } if (txnMgr.isTxnOpen()) { if (commit) { - if(conf.getBoolVar(ConfVars.HIVE_IN_TEST) && conf.getBoolVar(ConfVars.HIVETESTMODEROLLBACKTXN)) { + if (driverContext.getConf().getBoolVar(ConfVars.HIVE_IN_TEST) && + driverContext.getConf().getBoolVar(ConfVars.HIVETESTMODEROLLBACKTXN)) { txnMgr.rollbackTxn(); } else { @@ -1211,7 +1167,7 @@ public void releaseLocksAndCommitOrRollback(boolean commit, HiveTxnManager txnMa */ public void releaseResources() { releasePlan(); - releaseDriverContext(); + releaseDriverQueue(); } @Override @@ -1291,16 +1247,16 @@ public CommandProcessorResponse compileAndRespond(String command, boolean cleanu if (cleanupTxnList) { // Valid txn list might be generated for a query compiled using this // command, thus we need to reset it - conf.unset(ValidTxnList.VALID_TXNS_KEY); + driverContext.getConf().unset(ValidTxnList.VALID_TXNS_KEY); } } } public void lockAndRespond() throws CommandProcessorException { // Assumes the query has already been compiled - if (plan == null) { + if (driverContext.getPlan() == null) { throw new IllegalStateException( - "No previously compiled query for driver - queryId=" + queryState.getQueryId()); + "No previously compiled query for driver - queryId=" + driverContext.getQueryState().getQueryId()); } if (requiresLock()) { @@ -1322,7 +1278,7 @@ private void compileInternal(String command, boolean deferClose) throws CommandP PerfLogger perfLogger = SessionState.getPerfLogger(true); perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.WAIT_COMPILE); - try (CompileLock compileLock = CompileLockFactory.newInstance(conf, command)) { + try (CompileLock compileLock = CompileLockFactory.newInstance(driverContext.getConf(), command)) { boolean success = compileLock.tryAcquire(); perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.WAIT_COMPILE); @@ -1349,8 +1305,8 @@ private void compileInternal(String command, boolean deferClose) throws CommandP //Save compile-time PerfLogging for WebUI. //Execution-time Perf logs are done by either another thread's PerfLogger //or a reset PerfLogger. - queryDisplay.setPerfLogStarts(QueryDisplay.Phase.COMPILATION, perfLogger.getStartTimes()); - queryDisplay.setPerfLogEnds(QueryDisplay.Phase.COMPILATION, perfLogger.getEndTimes()); + driverContext.getQueryDisplay().setPerfLogStarts(QueryDisplay.Phase.COMPILATION, perfLogger.getStartTimes()); + driverContext.getQueryDisplay().setPerfLogEnds(QueryDisplay.Phase.COMPILATION, perfLogger.getEndTimes()); } private void runInternal(String command, boolean alreadyCompiled) throws CommandProcessorException { @@ -1377,11 +1333,11 @@ private void runInternal(String command, boolean alreadyCompiled) throws Command // the method has been returned by an error or not. boolean isFinishedWithError = true; try { - HiveDriverRunHookContext hookContext = new HiveDriverRunHookContextImpl(conf, + HiveDriverRunHookContext hookContext = new HiveDriverRunHookContextImpl(driverContext.getConf(), alreadyCompiled ? ctx.getCmd() : command); // Get all the driver run hooks and pre-execute them. try { - hookRunner.runPreDriverHooks(hookContext); + driverContext.getHookRunner().runPreDriverHooks(hookContext); } catch (Exception e) { String errorMessage = "FAILED: Hive Internal Error: " + Utilities.getNameMessage(e); console.printError(errorMessage + "\n" + StringUtils.stringifyException(e)); @@ -1393,7 +1349,7 @@ private void runInternal(String command, boolean alreadyCompiled) throws Command compileInternal(command, true); } else { // Since we're reusing the compiled plan, we need to update its start time for current run - plan.setQueryStartTime(queryDisplay.getQueryStartTime()); + driverContext.getPlan().setQueryStartTime(driverContext.getQueryDisplay().getQueryStartTime()); } //Reset the PerfLogger so that it doesn't retain any previous values. @@ -1403,7 +1359,7 @@ private void runInternal(String command, boolean alreadyCompiled) throws Command // the reason that we set the txn manager for the cxt here is because each // query has its own ctx object. The txn mgr is shared across the // same instance of Driver, which can run multiple queries. - ctx.setHiveTxnManager(queryTxnMgr); + ctx.setHiveTxnManager(driverContext.getTxnManager()); checkInterrupted("at acquiring the lock.", null, null); @@ -1418,13 +1374,14 @@ private void runInternal(String command, boolean alreadyCompiled) throws Command // Currently, we acquire a snapshot, we compile the query wrt that snapshot, // and then, we acquire locks. If snapshot is still valid, we continue as usual. // But if snapshot is not valid, we recompile the query. - retrial = true; - backupContext.addRewrittenStatementContext(ctx); - backupContext.setHiveLocks(ctx.getHiveLocks()); - ctx = backupContext; - conf.set(ValidTxnList.VALID_TXNS_KEY, queryTxnMgr.getValidTxns().toString()); - if (plan.hasAcidResourcesInQuery()) { - recordValidWriteIds(queryTxnMgr); + driverContext.setRetrial(true); + driverContext.getBackupContext().addRewrittenStatementContext(ctx); + driverContext.getBackupContext().setHiveLocks(ctx.getHiveLocks()); + ctx = driverContext.getBackupContext(); + driverContext.getConf().set(ValidTxnList.VALID_TXNS_KEY, + driverContext.getTxnManager().getValidTxns().toString()); + if (driverContext.getPlan().hasAcidResourcesInQuery()) { + recordValidWriteIds(driverContext.getTxnManager()); } if (!alreadyCompiled) { @@ -1432,7 +1389,7 @@ private void runInternal(String command, boolean alreadyCompiled) throws Command compileInternal(command, true); } else { // Since we're reusing the compiled plan, we need to update its start time for current run - plan.setQueryStartTime(queryDisplay.getQueryStartTime()); + driverContext.getPlan().setQueryStartTime(driverContext.getQueryDisplay().getQueryStartTime()); } if (!isValidTxnListState()) { @@ -1446,7 +1403,7 @@ private void runInternal(String command, boolean alreadyCompiled) throws Command // the reason that we set the txn manager for the cxt here is because each // query has its own ctx object. The txn mgr is shared across the // same instance of Driver, which can run multiple queries. - ctx.setHiveTxnManager(queryTxnMgr); + ctx.setHiveTxnManager(driverContext.getTxnManager()); } } catch (LockException e) { throw handleHiveException(e, 13); @@ -1462,12 +1419,14 @@ private void runInternal(String command, boolean alreadyCompiled) throws Command //if needRequireLock is false, the release here will do nothing because there is no lock try { //since set autocommit starts an implicit txn, close it - if(queryTxnMgr.isImplicitTransactionOpen() || plan.getOperation() == HiveOperation.COMMIT) { + if (driverContext.getTxnManager().isImplicitTransactionOpen() || + driverContext.getPlan().getOperation() == HiveOperation.COMMIT) { releaseLocksAndCommitOrRollback(true); } - else if(plan.getOperation() == HiveOperation.ROLLBACK) { + else if(driverContext.getPlan().getOperation() == HiveOperation.ROLLBACK) { releaseLocksAndCommitOrRollback(false); - } else if (!queryTxnMgr.isTxnOpen() && queryState.getHiveOperation() == HiveOperation.REPLLOAD) { + } else if (!driverContext.getTxnManager().isTxnOpen() && + driverContext.getQueryState().getHiveOperation() == HiveOperation.REPLLOAD) { // repl load during migration, commits the explicit txn and start some internal txns. Call // releaseLocksAndCommitOrRollback to do the clean up. releaseLocksAndCommitOrRollback(false); @@ -1478,12 +1437,12 @@ else if(plan.getOperation() == HiveOperation.ROLLBACK) { throw handleHiveException(e, 12); } - queryDisplay.setPerfLogStarts(QueryDisplay.Phase.EXECUTION, perfLogger.getStartTimes()); - queryDisplay.setPerfLogEnds(QueryDisplay.Phase.EXECUTION, perfLogger.getEndTimes()); + driverContext.getQueryDisplay().setPerfLogStarts(QueryDisplay.Phase.EXECUTION, perfLogger.getStartTimes()); + driverContext.getQueryDisplay().setPerfLogEnds(QueryDisplay.Phase.EXECUTION, perfLogger.getEndTimes()); // Take all the driver run hooks and post-execute them. try { - hookRunner.runPostDriverHooks(hookContext); + driverContext.getHookRunner().runPostDriverHooks(hookContext); } catch (Exception e) { String errorMessage = "FAILED: Hive Internal Error: " + Utilities.getNameMessage(e); console.printError(errorMessage + "\n" + StringUtils.stringifyException(e)); @@ -1494,7 +1453,7 @@ else if(plan.getOperation() == HiveOperation.ROLLBACK) { if (driverState.isAborted()) { closeInProcess(true); } else { - // only release the related resources ctx, driverContext as normal + // only release the related resources ctx, driverQueue as normal releaseResources(); } @@ -1544,11 +1503,11 @@ private boolean requiresLock() { if (isExplicitLockOperation()) { return false; } - if (!HiveConf.getBoolVar(conf, ConfVars.HIVE_LOCK_MAPRED_ONLY)) { + if (!HiveConf.getBoolVar(driverContext.getConf(), ConfVars.HIVE_LOCK_MAPRED_ONLY)) { return true; } Queue> taskQueue = new LinkedList>(); - taskQueue.addAll(plan.getRootTasks()); + taskQueue.addAll(driverContext.getPlan().getRootTasks()); while (taskQueue.peek() != null) { Task tsk = taskQueue.remove(); if (tsk.requireLock()) { @@ -1567,7 +1526,7 @@ private boolean requiresLock() { } private boolean isExplicitLockOperation() { - HiveOperation currentOpt = plan.getOperation(); + HiveOperation currentOpt = driverContext.getPlan().getOperation(); if (currentOpt != null) { switch (currentOpt) { case LOCKDB: @@ -1585,7 +1544,7 @@ private boolean isExplicitLockOperation() { private CommandProcessorException createProcessorException(int ret, String errorMessage, String sqlState, Throwable downstreamError) { SessionState.getPerfLogger().cleanupPerfLogMetrics(); - queryDisplay.setErrorMessage(errorMessage); + driverContext.getQueryDisplay().setErrorMessage(errorMessage); if (downstreamError != null && downstreamError instanceof HiveException) { ErrorMsg em = ((HiveException)downstreamError).getCanonicalErrorMsg(); if (em != null) { @@ -1598,60 +1557,60 @@ private CommandProcessorException createProcessorException(int ret, String error private void useFetchFromCache(CacheEntry cacheEntry) { // Change query FetchTask to use new location specified in results cache. FetchTask fetchTaskFromCache = (FetchTask) TaskFactory.get(cacheEntry.getFetchWork()); - fetchTaskFromCache.initialize(queryState, plan, null, ctx.getOpContext()); - plan.setFetchTask(fetchTaskFromCache); - cacheUsage = new CacheUsage(CacheUsage.CacheStatus.QUERY_USING_CACHE, cacheEntry); + fetchTaskFromCache.initialize(driverContext.getQueryState(), driverContext.getPlan(), null, ctx.getOpContext()); + driverContext.getPlan().setFetchTask(fetchTaskFromCache); + driverContext.setCacheUsage(new CacheUsage(CacheUsage.CacheStatus.QUERY_USING_CACHE, cacheEntry)); } private void preExecutionCacheActions() throws Exception { - if (cacheUsage != null) { - if (cacheUsage.getStatus() == CacheUsage.CacheStatus.CAN_CACHE_QUERY_RESULTS && - plan.getFetchTask() != null) { + if (driverContext.getCacheUsage() != null) { + if (driverContext.getCacheUsage().getStatus() == CacheUsage.CacheStatus.CAN_CACHE_QUERY_RESULTS && + driverContext.getPlan().getFetchTask() != null) { ValidTxnWriteIdList txnWriteIdList = null; - if (plan.hasAcidResourcesInQuery()) { - txnWriteIdList = AcidUtils.getValidTxnWriteIdList(conf); + if (driverContext.getPlan().hasAcidResourcesInQuery()) { + txnWriteIdList = AcidUtils.getValidTxnWriteIdList(driverContext.getConf()); } // The results of this query execution might be cacheable. // Add a placeholder entry in the cache so other queries know this result is pending. CacheEntry pendingCacheEntry = - QueryResultsCache.getInstance().addToCache(cacheUsage.getQueryInfo(), txnWriteIdList); + QueryResultsCache.getInstance().addToCache(driverContext.getCacheUsage().getQueryInfo(), txnWriteIdList); if (pendingCacheEntry != null) { // Update cacheUsage to reference the pending entry. - this.cacheUsage.setCacheEntry(pendingCacheEntry); + this.driverContext.getCacheUsage().setCacheEntry(pendingCacheEntry); } } } } private void postExecutionCacheActions() throws Exception { - if (cacheUsage != null) { - if (cacheUsage.getStatus() == CacheUsage.CacheStatus.QUERY_USING_CACHE) { + if (driverContext.getCacheUsage() != null) { + if (driverContext.getCacheUsage().getStatus() == CacheUsage.CacheStatus.QUERY_USING_CACHE) { // Using a previously cached result. - CacheEntry cacheEntry = cacheUsage.getCacheEntry(); + CacheEntry cacheEntry = driverContext.getCacheUsage().getCacheEntry(); // Reader count already incremented during cache lookup. // Save to usedCacheEntry to ensure reader is released after query. - this.usedCacheEntry = cacheEntry; - } else if (cacheUsage.getStatus() == CacheUsage.CacheStatus.CAN_CACHE_QUERY_RESULTS && - cacheUsage.getCacheEntry() != null && - plan.getFetchTask() != null) { + driverContext.setUsedCacheEntry(cacheEntry); + } else if (driverContext.getCacheUsage().getStatus() == CacheUsage.CacheStatus.CAN_CACHE_QUERY_RESULTS && + driverContext.getCacheUsage().getCacheEntry() != null && + driverContext.getPlan().getFetchTask() != null) { // Save results to the cache for future queries to use. PerfLogger perfLogger = SessionState.getPerfLogger(); perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SAVE_TO_RESULTS_CACHE); ValidTxnWriteIdList txnWriteIdList = null; - if (plan.hasAcidResourcesInQuery()) { - txnWriteIdList = AcidUtils.getValidTxnWriteIdList(conf); + if (driverContext.getPlan().hasAcidResourcesInQuery()) { + txnWriteIdList = AcidUtils.getValidTxnWriteIdList(driverContext.getConf()); } - CacheEntry cacheEntry = cacheUsage.getCacheEntry(); + CacheEntry cacheEntry = driverContext.getCacheUsage().getCacheEntry(); boolean savedToCache = QueryResultsCache.getInstance().setEntryValid( cacheEntry, - plan.getFetchTask().getWork()); + driverContext.getPlan().getFetchTask().getWork()); LOG.info("savedToCache: {} ({})", savedToCache, cacheEntry); if (savedToCache) { - useFetchFromCache(cacheUsage.getCacheEntry()); + useFetchFromCache(driverContext.getCacheUsage().getCacheEntry()); // setEntryValid() already increments the reader count. Set usedCacheEntry so it gets released. - this.usedCacheEntry = cacheUsage.getCacheEntry(); + driverContext.setUsedCacheEntry(driverContext.getCacheUsage().getCacheEntry()); } perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SAVE_TO_RESULTS_CACHE); @@ -1663,20 +1622,20 @@ private void execute() throws CommandProcessorException { PerfLogger perfLogger = SessionState.getPerfLogger(); perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.DRIVER_EXECUTE); - boolean noName = Strings.isNullOrEmpty(conf.get(MRJobConfig.JOB_NAME)); + boolean noName = Strings.isNullOrEmpty(driverContext.getConf().get(MRJobConfig.JOB_NAME)); int maxlen; - if ("spark".equals(conf.getVar(ConfVars.HIVE_EXECUTION_ENGINE))) { - maxlen = conf.getIntVar(HiveConf.ConfVars.HIVESPARKJOBNAMELENGTH); + if ("spark".equals(driverContext.getConf().getVar(ConfVars.HIVE_EXECUTION_ENGINE))) { + maxlen = driverContext.getConf().getIntVar(HiveConf.ConfVars.HIVESPARKJOBNAMELENGTH); } else { - maxlen = conf.getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH); + maxlen = driverContext.getConf().getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH); } Metrics metrics = MetricsFactory.getInstance(); - String queryId = plan.getQueryId(); + String queryId = driverContext.getPlan().getQueryId(); // Get the query string from the conf file as the compileInternal() method might // hide sensitive information during query redaction. - String queryStr = conf.getQueryString(); + String queryStr = driverContext.getConf().getQueryString(); driverState.lock(); try { @@ -1693,8 +1652,6 @@ private void execute() throws CommandProcessorException { driverState.unlock(); } - maxthreads = HiveConf.getIntVar(conf, HiveConf.ConfVars.EXECPARALLETHREADNUMBER); - HookContext hookContext = null; // Whether there's any error occurred during query execution. Used for query lifetime hook. @@ -1706,31 +1663,33 @@ private void execute() throws CommandProcessorException { // so clear timing in this thread's Hive object before proceeding. Hive.get().clearMetaCallTiming(); - plan.setStarted(); + driverContext.getPlan().setStarted(); if (SessionState.get() != null) { SessionState.get().getHiveHistory().startQuery(queryStr, queryId); - SessionState.get().getHiveHistory().logPlanProgress(plan); + SessionState.get().getHiveHistory().logPlanProgress(driverContext.getPlan()); } resStream = null; SessionState ss = SessionState.get(); // TODO: should this use getUserFromAuthenticator? - hookContext = new PrivateHookContext(plan, queryState, ctx.getPathToCS(), SessionState.get().getUserName(), + hookContext = new PrivateHookContext(driverContext.getPlan(), driverContext.getQueryState(), ctx.getPathToCS(), + SessionState.get().getUserName(), ss.getUserIpAddress(), InetAddress.getLocalHost().getHostAddress(), operationId, - ss.getSessionId(), Thread.currentThread().getName(), ss.isHiveServerQuery(), perfLogger, queryInfo, ctx); + ss.getSessionId(), Thread.currentThread().getName(), ss.isHiveServerQuery(), perfLogger, + driverContext.getQueryInfo(), ctx); hookContext.setHookType(HookContext.HookType.PRE_EXEC_HOOK); - hookRunner.runPreHooks(hookContext); + driverContext.getHookRunner().runPreHooks(hookContext); // Trigger query hooks before query execution. - hookRunner.runBeforeExecutionHook(queryStr, hookContext); + driverContext.getHookRunner().runBeforeExecutionHook(queryStr, hookContext); - setQueryDisplays(plan.getRootTasks()); - int mrJobs = Utilities.getMRTasks(plan.getRootTasks()).size(); - int jobs = mrJobs + Utilities.getTezTasks(plan.getRootTasks()).size() - + Utilities.getSparkTasks(plan.getRootTasks()).size(); + setQueryDisplays(driverContext.getPlan().getRootTasks()); + int mrJobs = Utilities.getMRTasks(driverContext.getPlan().getRootTasks()).size(); + int jobs = mrJobs + Utilities.getTezTasks(driverContext.getPlan().getRootTasks()).size() + + Utilities.getSparkTasks(driverContext.getPlan().getRootTasks()).size(); if (jobs > 0) { logMrWarning(mrJobs); console.printInfo("Query ID = " + queryId); @@ -1739,7 +1698,7 @@ private void execute() throws CommandProcessorException { if (SessionState.get() != null) { SessionState.get().getHiveHistory().setQueryProperty(queryId, Keys.QUERY_NUM_TASKS, String.valueOf(jobs)); - SessionState.get().getHiveHistory().setIdToTableMap(plan.getIdToTableNameMap()); + SessionState.get().getHiveHistory().setIdToTableMap(driverContext.getPlan().getIdToTableNameMap()); } String jobname = Utilities.abbreviate(queryStr, maxlen - 6); @@ -1751,22 +1710,22 @@ private void execute() throws CommandProcessorException { checkInterrupted("before running tasks.", hookContext, perfLogger); - DriverContext driverCxt = new DriverContext(ctx); - driverCxt.prepare(plan); + DriverQueue driverQueue = new DriverQueue(ctx); + driverQueue.prepare(driverContext.getPlan()); ctx.setHDFSCleanup(true); - this.driverCxt = driverCxt; // for canceling the query (should be bound to session?) + this.driverQueue = driverQueue; // for canceling the query (should be bound to session?) SessionState.get().setMapRedStats(new LinkedHashMap<>()); SessionState.get().setStackTraces(new HashMap<>()); SessionState.get().setLocalMapRedErrors(new HashMap<>()); // Add root Tasks to runnable - for (Task tsk : plan.getRootTasks()) { + for (Task tsk : driverContext.getPlan().getRootTasks()) { // This should never happen, if it does, it's a bug with the potential to produce // incorrect results. assert tsk.getParentTasks() == null || tsk.getParentTasks().isEmpty(); - driverCxt.addToRunnable(tsk); + driverQueue.addToRunnable(tsk); if (metrics != null) { tsk.updateTaskMetrics(metrics); @@ -1777,18 +1736,19 @@ private void execute() throws CommandProcessorException { perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.RUN_TASKS); // Loop while you either have tasks running, or tasks queued up - while (driverCxt.isRunning()) { + while (driverQueue.isRunning()) { // Launch upto maxthreads tasks Task task; - while ((task = driverCxt.getRunnable(maxthreads)) != null) { - TaskRunner runner = launchTask(task, queryId, noName, jobname, jobs, driverCxt); + int maxthreads = HiveConf.getIntVar(driverContext.getConf(), HiveConf.ConfVars.EXECPARALLETHREADNUMBER); + while ((task = driverQueue.getRunnable(maxthreads)) != null) { + TaskRunner runner = launchTask(task, queryId, noName, jobname, jobs, driverQueue); if (!runner.isRunning()) { break; } } // poll the Tasks to see which one completed - TaskRunner tskRun = driverCxt.pollFinished(); + TaskRunner tskRun = driverQueue.pollFinished(); if (tskRun == null) { continue; } @@ -1798,14 +1758,14 @@ private void execute() throws CommandProcessorException { Briefly for replication the graph is huge and so memory pressure is going to be huge if we keep a lot of references around. */ - String opName = plan.getOperationName(); + String opName = driverContext.getPlan().getOperationName(); boolean isReplicationOperation = opName.equals(HiveOperation.REPLDUMP.getOperationName()) || opName.equals(HiveOperation.REPLLOAD.getOperationName()); if (!isReplicationOperation) { hookContext.addCompleteTask(tskRun); } - queryDisplay.setTaskResult(tskRun.getTask().getId(), tskRun.getTaskResult()); + driverContext.getQueryDisplay().setTaskResult(tskRun.getTask().getId(), tskRun.getTaskResult()); Task tsk = tskRun.getTask(); TaskResult result = tskRun.getTaskResult(); @@ -1822,14 +1782,14 @@ private void execute() throws CommandProcessorException { console.printError(errorMessage); // add backup task to runnable - if (DriverContext.isLaunchable(backupTask)) { - driverCxt.addToRunnable(backupTask); + if (DriverQueue.isLaunchable(backupTask)) { + driverQueue.addToRunnable(backupTask); } continue; } else { String errorMessage = getErrorMsgAndDetail(exitVal, result.getTaskError(), tsk); - if (driverCxt.isShutdown()) { + if (driverQueue.isShutdown()) { errorMessage = "FAILED: Operation cancelled. " + errorMessage; } invokeFailureHooks(perfLogger, hookContext, @@ -1847,7 +1807,7 @@ private void execute() throws CommandProcessorException { } console.printError(errorMessage); - driverCxt.shutdown(); + driverQueue.shutdown(); // in case we decided to run everything in local mode, restore the // the jobtracker setting to its initial value ctx.restoreOriginalTracker(); @@ -1855,7 +1815,7 @@ private void execute() throws CommandProcessorException { } } - driverCxt.finished(tskRun); + driverQueue.finished(tskRun); if (SessionState.get() != null) { SessionState.get().getHiveHistory().setTaskProperty(queryId, tsk.getId(), @@ -1865,8 +1825,8 @@ private void execute() throws CommandProcessorException { if (tsk.getChildTasks() != null) { for (Task child : tsk.getChildTasks()) { - if (DriverContext.isLaunchable(child)) { - driverCxt.addToRunnable(child); + if (DriverQueue.isLaunchable(child)) { + driverQueue.addToRunnable(child); } } } @@ -1879,7 +1839,7 @@ private void execute() throws CommandProcessorException { // the jobtracker setting to its initial value ctx.restoreOriginalTracker(); - if (driverCxt.isShutdown()) { + if (driverQueue.isShutdown()) { String errorMessage = "FAILED: Operation cancelled"; invokeFailureHooks(perfLogger, hookContext, errorMessage, null); console.printError(errorMessage); @@ -1890,27 +1850,27 @@ private void execute() throws CommandProcessorException { // Some incomplete outputs may be added at the beginning, for eg: for dynamic partitions. // remove them HashSet remOutputs = new LinkedHashSet(); - for (WriteEntity output : plan.getOutputs()) { + for (WriteEntity output : driverContext.getPlan().getOutputs()) { if (!output.isComplete()) { remOutputs.add(output); } } for (WriteEntity output : remOutputs) { - plan.getOutputs().remove(output); + driverContext.getPlan().getOutputs().remove(output); } hookContext.setHookType(HookContext.HookType.POST_EXEC_HOOK); - hookRunner.runPostExecHooks(hookContext); + driverContext.getHookRunner().runPostExecHooks(hookContext); if (SessionState.get() != null) { SessionState.get().getHiveHistory().setQueryProperty(queryId, Keys.QUERY_RET_CODE, String.valueOf(0)); SessionState.get().getHiveHistory().printRowCount(queryId); } - releasePlan(plan); + releasePlan(driverContext.getPlan()); } catch (CommandProcessorException cpe) { executionError = true; throw cpe; @@ -1938,7 +1898,7 @@ private void execute() throws CommandProcessorException { } finally { // Trigger query hooks after query completes its execution. try { - hookRunner.runAfterExecutionHook(queryStr, hookContext, executionError); + driverContext.getHookRunner().runAfterExecutionHook(queryStr, hookContext, executionError); } catch (Exception e) { LOG.warn("Failed when invoking query after execution hook", e); } @@ -1947,12 +1907,12 @@ private void execute() throws CommandProcessorException { SessionState.get().getHiveHistory().endQuery(queryId); } if (noName) { - conf.set(MRJobConfig.JOB_NAME, ""); + driverContext.getConf().set(MRJobConfig.JOB_NAME, ""); } double duration = perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.DRIVER_EXECUTE)/1000.00; ImmutableMap executionHMSTimings = dumpMetaCallTimingWithoutEx("execution"); - queryDisplay.setHmsTimings(QueryDisplay.Phase.EXECUTION, executionHMSTimings); + driverContext.getQueryDisplay().setHmsTimings(QueryDisplay.Phase.EXECUTION, executionHMSTimings); Map stats = SessionState.get().getMapRedStats(); if (stats != null && !stats.isEmpty()) { @@ -1968,7 +1928,7 @@ private void execute() throws CommandProcessorException { numModifiedRows = addWithOverflowCheck(numModifiedRows, entry.getValue().getNumModifiedRows()); } } - queryState.setNumModifiedRows(numModifiedRows); + driverContext.getQueryState().setNumModifiedRows(numModifiedRows); console.printInfo("Total MapReduce CPU Time Spent: " + Utilities.formatMsecToStr(totalCpu)); } SparkSession ss = SessionState.get().getSparkSession(); @@ -2034,7 +1994,7 @@ private void setQueryDisplays(List> tasks) { if (visited.contains(task)) { continue; } - task.setQueryDisplay(queryDisplay); + task.setQueryDisplay(driverContext.getQueryDisplay()); if (task.getDependentTasks() != null) { childTasks.addAll(task.getDependentTasks()); } @@ -2044,7 +2004,7 @@ private void setQueryDisplays(List> tasks) { } private void logMrWarning(int mrJobs) { - if (mrJobs <= 0 || !("mr".equals(HiveConf.getVar(conf, ConfVars.HIVE_EXECUTION_ENGINE)))) { + if (mrJobs <= 0 || !("mr".equals(HiveConf.getVar(driverContext.getConf(), ConfVars.HIVE_EXECUTION_ENGINE)))) { return; } String warning = HiveConf.generateMrDeprecationWarning(); @@ -2078,7 +2038,7 @@ private void invokeFailureHooks(PerfLogger perfLogger, hookContext.setErrorMessage(errorMessage); hookContext.setException(exception); // Get all the failure execution hooks and execute them. - hookRunner.runFailureHooks(hookContext); + driverContext.getHookRunner().runFailureHooks(hookContext); } /** @@ -2094,27 +2054,27 @@ private void invokeFailureHooks(PerfLogger perfLogger, * name of the task, if it is a map-reduce job * @param jobs * number of map-reduce jobs - * @param cxt + * @param driverQueue * the driver context */ private TaskRunner launchTask(Task tsk, String queryId, boolean noName, - String jobname, int jobs, DriverContext cxt) throws HiveException { + String jobname, int jobs, DriverQueue driverQueue) throws HiveException { if (SessionState.get() != null) { SessionState.get().getHiveHistory().startTask(queryId, tsk, tsk.getClass().getName()); } if (tsk.isMapRedTask() && !(tsk instanceof ConditionalTask)) { if (noName) { - conf.set(MRJobConfig.JOB_NAME, jobname + " (" + tsk.getId() + ")"); + driverContext.getConf().set(MRJobConfig.JOB_NAME, jobname + " (" + 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); + driverContext.getConf().set(DagUtils.MAPREDUCE_WORKFLOW_NODE_NAME, tsk.getId()); + Utilities.setWorkflowAdjacencies(driverContext.getConf(), driverContext.getPlan()); + driverQueue.incCurJobNo(1); + console.printInfo("Launching Job " + driverQueue.getCurJobNo() + " out of " + jobs); } - tsk.initialize(queryState, plan, cxt, ctx.getOpContext()); - TaskRunner tskRun = new TaskRunner(tsk, cxt); + tsk.initialize(driverContext.getQueryState(), driverContext.getPlan(), driverQueue, ctx.getOpContext()); + TaskRunner tskRun = new TaskRunner(tsk, driverQueue); - cxt.launching(tskRun); + driverQueue.launching(tskRun); // Launch Task if (HiveConf.getBoolVar(tsk.getConf(), HiveConf.ConfVars.EXECPARALLEL) && tsk.canExecuteInParallel()) { // Launch it in the parallel mode, as a separate thread only for MR tasks @@ -2133,7 +2093,7 @@ private TaskRunner launchTask(Task tsk, String queryId, boolean noName, @Override public boolean isFetchingTable() { - return fetchTask != null; + return driverContext.getFetchTask() != null; } @SuppressWarnings("unchecked") @@ -2149,11 +2109,11 @@ public boolean getResults(List res) throws IOException { * indeed written using ThriftJDBCBinarySerDe, read one row from the output sequence file, * since it is a blob of row batches. */ - if (fetchTask.getWork().isUsingThriftJDBCBinarySerDe()) { + if (driverContext.getFetchTask().getWork().isUsingThriftJDBCBinarySerDe()) { maxRows = 1; } - fetchTask.setMaxRows(maxRows); - return fetchTask.fetch(res); + driverContext.getFetchTask().setMaxRows(maxRows); + return driverContext.getFetchTask().fetch(res); } if (resStream == null) { @@ -2209,26 +2169,26 @@ public void resetFetch() throws IOException { } if (isFetchingTable()) { try { - fetchTask.clearFetch(); + driverContext.getFetchTask().clearFetch(); } catch (Exception e) { throw new IOException("Error closing the current fetch task", e); } // FetchTask should not depend on the plan. - fetchTask.initialize(queryState, null, null, ctx.getOpContext()); + driverContext.getFetchTask().initialize(driverContext.getQueryState(), null, null, ctx.getOpContext()); } else { ctx.resetStream(); resStream = null; } } - // DriverContext could be released in the query and close processes at same + // DriverQueue could be released in the query and close processes at same // time, which needs to be thread protected. - private void releaseDriverContext() { + private void releaseDriverQueue() { driverState.lock(); try { - if (driverCxt != null) { - driverCxt.shutdown(); - driverCxt = null; + if (driverQueue != null) { + driverQueue.shutdown(); + driverQueue = null; } } catch (Exception e) { LOG.debug("Exception while shutting down the task runner", e); @@ -2239,14 +2199,15 @@ private void releaseDriverContext() { private void releasePlan() { try { - if (plan != null) { - fetchTask = plan.getFetchTask(); + if (driverContext.getPlan() != null) { + FetchTask fetchTask = driverContext.getPlan().getFetchTask(); if (fetchTask != null) { - fetchTask.setDriverContext(null); + fetchTask.setDriverQueue(null); fetchTask.setQueryPlan(null); } + driverContext.setFetchTask(fetchTask); } - plan = null; + driverContext.setPlan(null); } catch (Exception e) { LOG.debug("Exception while clearing the Fetch task", e); } @@ -2257,8 +2218,8 @@ private void releaseContext() { if (ctx != null) { boolean deleteResultDir = true; // don't let context delete result dirs and scratch dirs if result was cached - if(this.cacheUsage != null - && this.cacheUsage.getStatus() == CacheUsage.CacheStatus.QUERY_USING_CACHE) { + if (driverContext.getCacheUsage() != null + && driverContext.getCacheUsage().getStatus() == CacheUsage.CacheStatus.QUERY_USING_CACHE) { deleteResultDir = false; } @@ -2287,9 +2248,9 @@ private void releaseResStream() { private void releaseFetchTask() { try { - if (fetchTask != null) { - fetchTask.clearFetch(); - fetchTask = null; + if (driverContext.getFetchTask() != null) { + driverContext.getFetchTask().clearFetch(); + driverContext.setFetchTask(null); } } catch (Exception e) { LOG.debug(" Exception while clearing the FetchTask ", e); @@ -2299,35 +2260,35 @@ private void releaseFetchTask() { private boolean hasBadCacheAttempt() { // Check if the query results were cacheable, and created a pending cache entry. // If we successfully saved the results, the usage would have changed to QUERY_USING_CACHE. - return (cacheUsage != null && - cacheUsage.getStatus() == CacheUsage.CacheStatus.CAN_CACHE_QUERY_RESULTS && - cacheUsage.getCacheEntry() != null); + return (driverContext.getCacheUsage() != null && + driverContext.getCacheUsage().getStatus() == CacheUsage.CacheStatus.CAN_CACHE_QUERY_RESULTS && + driverContext.getCacheUsage().getCacheEntry() != null); } private void releaseCachedResult() { // Assumes the reader count has been incremented automatically by the results cache by either // lookup or creating the cache entry. - if (usedCacheEntry != null) { - usedCacheEntry.releaseReader(); - usedCacheEntry = null; + if (driverContext.getUsedCacheEntry() != null) { + driverContext.getUsedCacheEntry().releaseReader(); + driverContext.setUsedCacheEntry(null); } else if (hasBadCacheAttempt()) { // This query create a pending cache entry but it was never saved with real results, cleanup. // This step is required, as there may be queries waiting on this pending cache entry. // Removing/invalidating this entry will notify the waiters that this entry cannot be used. try { - QueryResultsCache.getInstance().removeEntry(cacheUsage.getCacheEntry()); + QueryResultsCache.getInstance().removeEntry(driverContext.getCacheUsage().getCacheEntry()); } catch (Exception err) { - LOG.error("Error removing failed cache entry " + cacheUsage.getCacheEntry(), err); + LOG.error("Error removing failed cache entry " + driverContext.getCacheUsage().getCacheEntry(), err); } } - cacheUsage = null; + driverContext.setCacheUsage(null); } // Close and release resources within a running query process. Since it runs under // driver state COMPILING, EXECUTING or INTERRUPT, it would not have race condition // with the releases probably running in the other closing thread. private int closeInProcess(boolean destroyed) { - releaseDriverContext(); + releaseDriverQueue(); releasePlan(); releaseCachedResult(); releaseFetchTask(); @@ -2352,7 +2313,7 @@ private int closeInProcess(boolean destroyed) { public void close() { driverState.lock(); try { - releaseDriverContext(); + releaseDriverQueue(); if (driverState.isCompiling() || driverState.isExecuting()) { driverState.abort(); } @@ -2398,7 +2359,7 @@ public void destroy() { @Override public QueryDisplay getQueryDisplay() { - return queryDisplay; + return driverContext.getQueryDisplay(); } /** @@ -2412,36 +2373,37 @@ public void setOperationId(String opId) { @Override public QueryState getQueryState() { - return queryState; + return driverContext.getQueryState(); } public HookRunner getHookRunner() { - return hookRunner; + return driverContext.getHookRunner(); } public void setStatsSource(StatsSource runtimeStatsSource) { - this.statsSource = runtimeStatsSource; + driverContext.setStatsSource(runtimeStatsSource); } public StatsSource getStatsSource() { - return statsSource; + return driverContext.getStatsSource(); } @Override public boolean hasResultSet() { // TODO explain should use a FetchTask for reading - for (Task task : plan.getRootTasks()) { + for (Task task : driverContext.getPlan().getRootTasks()) { if (task.getClass() == ExplainTask.class) { return true; } } - return plan.getFetchTask() != null && plan.getResultSchema() != null && plan.getResultSchema().isSetFieldSchemas(); + return driverContext.getPlan().getFetchTask() != null && driverContext.getPlan().getResultSchema() != null && + driverContext.getPlan().getResultSchema().isSetFieldSchemas(); } - void setCompactionWriteIds(ValidWriteIdList val, long compactorTxnId) { - this.compactionWriteIds = val; - this.compactorTxnId = compactorTxnId; + void setCompactionWriteIds(ValidWriteIdList compactionWriteIds, long compactorTxnId) { + driverContext.setCompactionWriteIds(compactionWriteIds); + driverContext.setCompactorTxnId(compactorTxnId); } } diff --git ql/src/java/org/apache/hadoop/hive/ql/DriverContext.java ql/src/java/org/apache/hadoop/hive/ql/DriverContext.java index 1b8260aa68..ee8eeb5886 100644 --- ql/src/java/org/apache/hadoop/hive/ql/DriverContext.java +++ ql/src/java/org/apache/hadoop/hive/ql/DriverContext.java @@ -18,220 +18,189 @@ package org.apache.hadoop.hive.ql; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.LinkedBlockingQueue; - -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; -import org.apache.hadoop.hive.ql.exec.NodeUtils; -import org.apache.hadoop.hive.ql.exec.NodeUtils.Function; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.StatsTask; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.TaskRunner; -import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.ReduceWork; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.common.ValidWriteIdList; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.Schema; +import org.apache.hadoop.hive.ql.cache.results.CacheUsage; +import org.apache.hadoop.hive.ql.cache.results.QueryResultsCache.CacheEntry; +import org.apache.hadoop.hive.ql.exec.FetchTask; +import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager; +import org.apache.hadoop.hive.ql.plan.mapper.StatsSource; /** - * DriverContext. - * + * Context for the procedure managed by the Driver. */ public class DriverContext { + // For WebUI. Kept alive after queryPlan is freed. + private final QueryDisplay queryDisplay = new QueryDisplay(); + + 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). + // If this is set then this Transaction manager will be used during query + // compilation/execution rather than using the current session's transaction manager. + // This might be needed in a situation where a Driver is nested within an already + // running Driver/query - the nested Driver requires a separate transaction manager + // so as not to conflict with the outer Driver/query which is using the session + // transaction manager. + private final HiveTxnManager initTxnManager; + + private QueryPlan plan; + private Schema schema; + + private FetchTask fetchTask; + // Transaction manager used for the query. This will be set at compile time based on + // either initTxnMgr or from the SessionState, in that order. + private HiveTxnManager txnManager; + private StatsSource statsSource; + + // Boolean to store information about whether valid txn list was generated + // for current query. + private boolean validTxnListsGenerated; + + private CacheUsage cacheUsage; + private CacheEntry usedCacheEntry; + private ValidWriteIdList compactionWriteIds = null; + private long compactorTxnId = 0; + + private Context backupContext = null; + private boolean retrial = false; + + public DriverContext(QueryState queryState, QueryInfo queryInfo, String userName, HookRunner hookRunner, + HiveTxnManager initTxnManager) { + this.queryState = queryState; + this.queryInfo = queryInfo; + this.conf = queryState.getConf(); + this.userName = userName; + this.hookRunner = hookRunner; + this.initTxnManager = initTxnManager; + } + + public QueryDisplay getQueryDisplay() { + return queryDisplay; + } + + public QueryState getQueryState() { + return queryState; + } + + public QueryInfo getQueryInfo() { + return queryInfo; + } + + public HiveConf getConf() { + return conf; + } + + public String getUserName() { + return userName; + } + + public HookRunner getHookRunner() { + return hookRunner; + } + + public HiveTxnManager getInitTxnManager() { + return initTxnManager; + } + + public QueryPlan getPlan() { + return plan; + } + + public void setPlan(QueryPlan plan) { + this.plan = plan; + } + + public Schema getSchema() { + return schema; + } - private static final Logger LOG = LoggerFactory.getLogger(Driver.class.getName()); - private static final SessionState.LogHelper console = new SessionState.LogHelper(LOG); + public void setSchema(Schema schema) { + this.schema = schema; + } - private static final int SLEEP_TIME = 2000; + public FetchTask getFetchTask() { + return fetchTask; + } - private Queue> runnable; - private Queue running; + public void setFetchTask(FetchTask fetchTask) { + this.fetchTask = fetchTask; + } - // how many jobs have been started - private int curJobNo; + public HiveTxnManager getTxnManager() { + return txnManager; + } - private Context ctx; - private boolean shutdown; + public void setTxnManager(HiveTxnManager txnManager) { + this.txnManager = txnManager; + } - final Map statsTasks = new HashMap<>(1); + public StatsSource getStatsSource() { + return statsSource; + } + + public void setStatsSource(StatsSource statsSource) { + this.statsSource = statsSource; + } + + public boolean isValidTxnListsGenerated() { + return validTxnListsGenerated; + } + + public void setValidTxnListsGenerated(boolean validTxnListsGenerated) { + this.validTxnListsGenerated = validTxnListsGenerated; + } + + public CacheUsage getCacheUsage() { + return cacheUsage; + } + + public void setCacheUsage(CacheUsage cacheUsage) { + this.cacheUsage = cacheUsage; + } - public DriverContext() { + public CacheEntry getUsedCacheEntry() { + return usedCacheEntry; } - public DriverContext(Context ctx) { - this.runnable = new ConcurrentLinkedQueue>(); - this.running = new LinkedBlockingQueue(); - this.ctx = ctx; + public void setUsedCacheEntry(CacheEntry usedCacheEntry) { + this.usedCacheEntry = usedCacheEntry; } - public synchronized boolean isShutdown() { - return shutdown; + public ValidWriteIdList getCompactionWriteIds() { + return compactionWriteIds; } - public synchronized boolean isRunning() { - return !shutdown && (!running.isEmpty() || !runnable.isEmpty()); + public void setCompactionWriteIds(ValidWriteIdList compactionWriteIds) { + this.compactionWriteIds = compactionWriteIds; } - public synchronized void remove(Task task) { - runnable.remove(task); + public long getCompactorTxnId() { + return compactorTxnId; } - public synchronized void launching(TaskRunner runner) throws HiveException { - checkShutdown(); - running.add(runner); + public void setCompactorTxnId(long compactorTxnId) { + this.compactorTxnId = compactorTxnId; } - public synchronized Task getRunnable(int maxthreads) throws HiveException { - checkShutdown(); - if (runnable.peek() != null && running.size() < maxthreads) { - return runnable.remove(); - } - return null; + public Context getBackupContext() { + return backupContext; } - public synchronized void releaseRunnable() { - //release the waiting poller. - notify(); + public void setBackupContext(Context backupContext) { + this.backupContext = backupContext; } - /** - * Polls running tasks to see if a task has ended. - * - * @return The result object for any completed/failed task - */ - public synchronized TaskRunner pollFinished() throws InterruptedException { - while (!shutdown) { - Iterator it = running.iterator(); - while (it.hasNext()) { - TaskRunner runner = it.next(); - if (runner != null && !runner.isRunning()) { - it.remove(); - return runner; - } - } - wait(SLEEP_TIME); - } - return null; + public boolean isRetrial() { + return retrial; } - private void checkShutdown() throws HiveException { - if (shutdown) { - throw new HiveException("FAILED: Operation cancelled"); - } - } - /** - * Cleans up remaining tasks in case of failure - */ - public synchronized void shutdown() { - LOG.debug("Shutting down query " + ctx.getCmd()); - shutdown = true; - for (TaskRunner runner : running) { - if (runner.isRunning()) { - Task task = runner.getTask(); - LOG.warn("Shutting down task : " + task); - try { - task.shutdown(); - } catch (Exception e) { - console.printError("Exception on shutting down task " + task.getId() + ": " + e); - } - Thread thread = runner.getRunner(); - if (thread != null) { - thread.interrupt(); - } - } - } - running.clear(); - } - - /** - * Checks if a task can be launched. - * - * @param tsk - * the task to be checked - * @return true if the task is launchable, false otherwise - */ - - public static boolean isLaunchable(Task tsk) { - // A launchable task is one that hasn't been queued, hasn't been - // initialized, and is runnable. - return tsk.isNotInitialized() && tsk.isRunnable(); - } - - public synchronized boolean addToRunnable(Task tsk) throws HiveException { - if (runnable.contains(tsk)) { - return false; - } - checkShutdown(); - runnable.add(tsk); - tsk.setQueued(); - return true; - } - - public int getCurJobNo() { - return curJobNo; - } - - public Context getCtx() { - return ctx; - } - - public void incCurJobNo(int amount) { - this.curJobNo = this.curJobNo + amount; - } - - public void prepare(QueryPlan plan) { - // extract stats keys from StatsTask - List> rootTasks = plan.getRootTasks(); - NodeUtils.iterateTask(rootTasks, StatsTask.class, new Function() { - @Override - public void apply(StatsTask statsTask) { - if (statsTask.getWork().isAggregating()) { - statsTasks.put(statsTask.getWork().getAggKey(), statsTask); - } - } - }); - } - - public void prepare(TaskRunner runner) { - } - - public void finished(TaskRunner runner) { - if (statsTasks.isEmpty() || !(runner.getTask() instanceof MapRedTask)) { - return; - } - MapRedTask mapredTask = (MapRedTask) runner.getTask(); - - MapWork mapWork = mapredTask.getWork().getMapWork(); - ReduceWork reduceWork = mapredTask.getWork().getReduceWork(); - List operators = new ArrayList(mapWork.getAliasToWork().values()); - if (reduceWork != null) { - operators.add(reduceWork.getReducer()); - } - final List statKeys = new ArrayList(1); - NodeUtils.iterate(operators, FileSinkOperator.class, new Function() { - @Override - public void apply(FileSinkOperator fsOp) { - if (fsOp.getConf().isGatherStats()) { - statKeys.add(fsOp.getConf().getStatsAggPrefix()); - } - } - }); - for (String statKey : statKeys) { - if (statsTasks.containsKey(statKey)) { - statsTasks.get(statKey).getWork().setSourceTask(mapredTask); - } else { - LOG.debug("There is no correspoing statTask for: " + statKey); - } - } + public void setRetrial(boolean retrial) { + this.retrial = retrial; } } diff --git ql/src/java/org/apache/hadoop/hive/ql/DriverQueue.java ql/src/java/org/apache/hadoop/hive/ql/DriverQueue.java new file mode 100644 index 0000000000..f61a938b10 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/DriverQueue.java @@ -0,0 +1,229 @@ +/* + * 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.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.LinkedBlockingQueue; + +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.NodeUtils; +import org.apache.hadoop.hive.ql.exec.NodeUtils.Function; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.StatsTask; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.exec.TaskRunner; +import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.MapWork; +import org.apache.hadoop.hive.ql.plan.ReduceWork; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Handles the queue of tasks that should be executed by the driver. + */ +public class DriverQueue { + + private static final Logger LOG = LoggerFactory.getLogger(Driver.class.getName()); + private static final SessionState.LogHelper CONSOLE = new SessionState.LogHelper(LOG); + + private static final int SLEEP_TIME = 2000; + + private final Queue> runnable = new ConcurrentLinkedQueue>();; + private final Queue running = new LinkedBlockingQueue(); + private final Map statsTasks = new HashMap<>(1); + private final Context ctx; + + // how many jobs have been started + private int curJobNo; + private boolean shutdown; + + public DriverQueue() { + this(null); + } + + public DriverQueue(Context ctx) { + this.ctx = ctx; + } + + public synchronized boolean isShutdown() { + return shutdown; + } + + public synchronized boolean isRunning() { + return !shutdown && (!running.isEmpty() || !runnable.isEmpty()); + } + + public synchronized void remove(Task task) { + runnable.remove(task); + } + + public synchronized void launching(TaskRunner runner) throws HiveException { + checkShutdown(); + running.add(runner); + } + + public synchronized Task getRunnable(int maxthreads) throws HiveException { + checkShutdown(); + if (runnable.peek() != null && running.size() < maxthreads) { + return runnable.remove(); + } + return null; + } + + public synchronized void releaseRunnable() { + //release the waiting poller. + notify(); + } + + /** + * Polls running tasks to see if a task has ended. + * + * @return The result object for any completed/failed task + */ + public synchronized TaskRunner pollFinished() throws InterruptedException { + while (!shutdown) { + Iterator it = running.iterator(); + while (it.hasNext()) { + TaskRunner runner = it.next(); + if (runner != null && !runner.isRunning()) { + it.remove(); + return runner; + } + } + wait(SLEEP_TIME); + } + return null; + } + + private void checkShutdown() throws HiveException { + if (shutdown) { + throw new HiveException("FAILED: Operation cancelled"); + } + } + /** + * Cleans up remaining tasks in case of failure + */ + public synchronized void shutdown() { + LOG.debug("Shutting down query " + ctx.getCmd()); + shutdown = true; + for (TaskRunner runner : running) { + if (runner.isRunning()) { + Task task = runner.getTask(); + LOG.warn("Shutting down task : " + task); + try { + task.shutdown(); + } catch (Exception e) { + CONSOLE.printError("Exception on shutting down task " + task.getId() + ": " + e); + } + Thread thread = runner.getRunner(); + if (thread != null) { + thread.interrupt(); + } + } + } + running.clear(); + } + + /** + * Checks if a task can be launched. + * + * @param tsk + * the task to be checked + * @return true if the task is launchable, false otherwise + */ + + public static boolean isLaunchable(Task tsk) { + // A launchable task is one that hasn't been queued, hasn't been + // initialized, and is runnable. + return tsk.isNotInitialized() && tsk.isRunnable(); + } + + public synchronized boolean addToRunnable(Task tsk) throws HiveException { + if (runnable.contains(tsk)) { + return false; + } + checkShutdown(); + runnable.add(tsk); + tsk.setQueued(); + return true; + } + + public int getCurJobNo() { + return curJobNo; + } + + public Context getCtx() { + return ctx; + } + + public void incCurJobNo(int amount) { + this.curJobNo = this.curJobNo + amount; + } + + public void prepare(QueryPlan plan) { + // extract stats keys from StatsTask + List> rootTasks = plan.getRootTasks(); + NodeUtils.iterateTask(rootTasks, StatsTask.class, new Function() { + @Override + public void apply(StatsTask statsTask) { + if (statsTask.getWork().isAggregating()) { + statsTasks.put(statsTask.getWork().getAggKey(), statsTask); + } + } + }); + } + + public void finished(TaskRunner runner) { + if (statsTasks.isEmpty() || !(runner.getTask() instanceof MapRedTask)) { + return; + } + MapRedTask mapredTask = (MapRedTask) runner.getTask(); + + MapWork mapWork = mapredTask.getWork().getMapWork(); + ReduceWork reduceWork = mapredTask.getWork().getReduceWork(); + List operators = new ArrayList(mapWork.getAliasToWork().values()); + if (reduceWork != null) { + operators.add(reduceWork.getReducer()); + } + final List statKeys = new ArrayList(1); + NodeUtils.iterate(operators, FileSinkOperator.class, new Function() { + @Override + public void apply(FileSinkOperator fsOp) { + if (fsOp.getConf().isGatherStats()) { + statKeys.add(fsOp.getConf().getStatsAggPrefix()); + } + } + }); + for (String statKey : statKeys) { + if (statsTasks.containsKey(statKey)) { + statsTasks.get(statKey).getWork().setSourceTask(mapredTask); + } else { + LOG.debug("There is no correspoing statTask for: " + statKey); + } + } + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLOperationContext.java ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLOperationContext.java index aedbfb079b..5050317b01 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLOperationContext.java +++ ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLOperationContext.java @@ -19,7 +19,7 @@ package org.apache.hadoop.hive.ql.ddl; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.metadata.Hive; @@ -34,7 +34,7 @@ public class DDLOperationContext { private final Hive db; private final HiveConf conf; - private final DriverContext driverContext; + private final DriverQueue driverQueue; private final MetaDataFormatter formatter; private final DDLTask task; private final DDLWork work; @@ -42,11 +42,11 @@ private final QueryPlan queryPlan; private final LogHelper console; - public DDLOperationContext(HiveConf conf, DriverContext driverContext, DDLTask task, DDLWork work, + public DDLOperationContext(HiveConf conf, DriverQueue driverQueue, DDLTask task, DDLWork work, QueryState queryState, QueryPlan queryPlan, LogHelper console) throws HiveException { this.db = Hive.get(conf); this.conf = conf; - this.driverContext = driverContext; + this.driverQueue = driverQueue; this.formatter = MetaDataFormatUtils.getFormatter(conf); this.task = task; this.work = work; @@ -63,8 +63,8 @@ public HiveConf getConf() { return conf; } - public DriverContext getDriverContext() { - return driverContext; + public DriverQueue getDriverQueue() { + return driverQueue; } public MetaDataFormatter getFormatter() { diff --git ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLTask.java ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLTask.java index 831d86c563..743896fb97 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLTask.java @@ -27,7 +27,7 @@ import java.util.Set; import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.Task; @@ -66,14 +66,14 @@ public boolean requireLock() { } @Override - public void initialize(QueryState queryState, QueryPlan queryPlan, DriverContext ctx, + public void initialize(QueryState queryState, QueryPlan queryPlan, DriverQueue ctx, CompilationOpContext opContext) { super.initialize(queryState, queryPlan, ctx, opContext); } @Override - public int execute(DriverContext driverContext) { - if (driverContext.getCtx().getExplainAnalyze() == AnalyzeState.RUNNING) { + public int execute(DriverQueue driverQueue) { + if (driverQueue.getCtx().getExplainAnalyze() == AnalyzeState.RUNNING) { return 0; } @@ -81,7 +81,7 @@ public int execute(DriverContext driverContext) { DDLDesc ddlDesc = work.getDDLDesc(); if (DESC_TO_OPARATION.containsKey(ddlDesc.getClass())) { - DDLOperationContext context = new DDLOperationContext(conf, driverContext, this, (DDLWork)work, queryState, + DDLOperationContext context = new DDLOperationContext(conf, driverQueue, this, (DDLWork)work, queryState, queryPlan, console); Class ddlOpertaionClass = DESC_TO_OPARATION.get(ddlDesc.getClass()); Constructor constructor = diff --git ql/src/java/org/apache/hadoop/hive/ql/ddl/database/lock/LockDatabaseOperation.java ql/src/java/org/apache/hadoop/hive/ql/ddl/database/lock/LockDatabaseOperation.java index 776e15e377..e882b30610 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ddl/database/lock/LockDatabaseOperation.java +++ ql/src/java/org/apache/hadoop/hive/ql/ddl/database/lock/LockDatabaseOperation.java @@ -34,7 +34,7 @@ public LockDatabaseOperation(DDLOperationContext context, LockDatabaseDesc desc) @Override public int execute() throws HiveException { - Context ctx = context.getDriverContext().getCtx(); + Context ctx = context.getDriverQueue().getCtx(); HiveTxnManager txnManager = ctx.getHiveTxnManager(); return txnManager.lockDatabase(context.getDb(), desc); } diff --git ql/src/java/org/apache/hadoop/hive/ql/ddl/database/unlock/UnlockDatabaseOperation.java ql/src/java/org/apache/hadoop/hive/ql/ddl/database/unlock/UnlockDatabaseOperation.java index 398fb84ac3..98e4527679 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ddl/database/unlock/UnlockDatabaseOperation.java +++ ql/src/java/org/apache/hadoop/hive/ql/ddl/database/unlock/UnlockDatabaseOperation.java @@ -34,7 +34,7 @@ public UnlockDatabaseOperation(DDLOperationContext context, UnlockDatabaseDesc d @Override public int execute() throws HiveException { - Context ctx = context.getDriverContext().getCtx(); + Context ctx = context.getDriverQueue().getCtx(); HiveTxnManager txnManager = ctx.getHiveTxnManager(); return txnManager.unlockDatabase(context.getDb(), desc); } diff --git ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/LockTableOperation.java ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/LockTableOperation.java index db307e6d46..da8ebbea4f 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/LockTableOperation.java +++ ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/LockTableOperation.java @@ -34,7 +34,7 @@ public LockTableOperation(DDLOperationContext context, LockTableDesc desc) { @Override public int execute() throws HiveException { - Context ctx = context.getDriverContext().getCtx(); + Context ctx = context.getDriverQueue().getCtx(); HiveTxnManager txnManager = ctx.getHiveTxnManager(); return txnManager.lockTable(context.getDb(), desc); } diff --git ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksOperation.java ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksOperation.java index 69414f24e9..d561d7f91f 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksOperation.java +++ ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksOperation.java @@ -57,7 +57,7 @@ public ShowLocksOperation(DDLOperationContext context, ShowLocksDesc desc) { @Override public int execute() throws HiveException { - Context ctx = context.getDriverContext().getCtx(); + Context ctx = context.getDriverQueue().getCtx(); HiveTxnManager txnManager = ctx.getHiveTxnManager(); HiveLockManager lockMgr = txnManager.getLockManager(); diff --git ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/UnlockTableOperation.java ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/UnlockTableOperation.java index 3759b72a11..ab8d501493 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/UnlockTableOperation.java +++ ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/UnlockTableOperation.java @@ -34,7 +34,7 @@ public UnlockTableOperation(DDLOperationContext context, UnlockTableDesc desc) { @Override public int execute() throws HiveException { - Context ctx = context.getDriverContext().getCtx(); + Context ctx = context.getDriverQueue().getCtx(); HiveTxnManager txnManager = ctx.getHiveTxnManager(); return txnManager.unlockTable(context.getDb(), desc); } diff --git ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/TruncateTableOperation.java ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/TruncateTableOperation.java index 12443802ea..6d54ae6e1b 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/TruncateTableOperation.java +++ ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/TruncateTableOperation.java @@ -24,7 +24,7 @@ import org.apache.hadoop.hive.common.FileUtils; import org.apache.hadoop.hive.ql.ddl.DDLOperationContext; import org.apache.hadoop.hive.ql.ddl.DDLUtils; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.ddl.DDLOperation; import org.apache.hadoop.hive.ql.io.rcfile.truncate.ColumnTruncateTask; @@ -48,13 +48,13 @@ public int execute() throws HiveException { truncateWork.setListBucketingCtx(desc.getLbCtx()); truncateWork.setMapperCannotSpanPartns(true); - DriverContext driverCxt = new DriverContext(); + DriverQueue driverQueue = new DriverQueue(); ColumnTruncateTask taskExec = new ColumnTruncateTask(); - taskExec.initialize(context.getQueryState(), null, driverCxt, null); + taskExec.initialize(context.getQueryState(), null, driverQueue, null); taskExec.setWork(truncateWork); taskExec.setQueryPlan(context.getQueryPlan()); - int ret = taskExec.execute(driverCxt); + int ret = taskExec.execute(driverQueue); if (taskExec.getException() != null) { context.getTask().setException(taskExec.getException()); } diff --git ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableArchiveOperation.java ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableArchiveOperation.java index 315857bc69..51e0974c38 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableArchiveOperation.java +++ ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableArchiveOperation.java @@ -175,7 +175,7 @@ private boolean isRecovery(Path intermediateArchivedDir, Path intermediateOrigin private Path createArchiveInTmpDir(Table table, PartSpecInfo partitionSpecInfo, Path originalDir) throws HiveException { // First create the archive in a tmp dir so that if the job fails, the bad files don't pollute the filesystem - Path tmpPath = new Path(context.getDriverContext().getCtx().getExternalTmpPath(originalDir), "partlevel"); + Path tmpPath = new Path(context.getDriverQueue().getCtx().getExternalTmpPath(originalDir), "partlevel"); // Create the Hadoop archive context.getConsole().printInfo("Creating " + ARCHIVE_NAME + " for " + originalDir.toString() + " in " + tmpPath); diff --git ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableConcatenateOperation.java ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableConcatenateOperation.java index 718c21da75..bbe2e255f7 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableConcatenateOperation.java +++ ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableConcatenateOperation.java @@ -25,7 +25,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.ddl.DDLOperation; import org.apache.hadoop.hive.ql.ddl.DDLOperationContext; import org.apache.hadoop.hive.ql.exec.Operator; @@ -55,7 +55,7 @@ public AlterTableConcatenateOperation(DDLOperationContext context, AlterTableCon @Override public int execute() throws HiveException { - CompilationOpContext opContext = context.getDriverContext().getCtx().getOpContext(); + CompilationOpContext opContext = context.getDriverQueue().getCtx().getOpContext(); MergeFileWork mergeWork = getMergeFileWork(opContext); Task task = getTask(mergeWork); @@ -123,9 +123,9 @@ private FileMergeDesc getFileMergeDesc() { } private int executeTask(CompilationOpContext opContext, Task task) { - DriverContext driverCxt = new DriverContext(); - task.initialize(context.getQueryState(), context.getQueryPlan(), driverCxt, opContext); - int ret = task.execute(driverCxt); + DriverQueue driverQueue = new DriverQueue(); + task.initialize(context.getQueryState(), context.getQueryPlan(), driverQueue, opContext); + int ret = task.execute(driverQueue); if (task.getException() != null) { context.getTask().setException(task.getException()); } diff --git ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableUnarchiveOperation.java ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableUnarchiveOperation.java index 0fbaea0b28..7af27a7564 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableUnarchiveOperation.java +++ ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableUnarchiveOperation.java @@ -97,7 +97,7 @@ public int execute() throws HiveException, URISyntaxException { throw new HiveException("Haven't found any archive where it should be"); } - Path tmpPath = context.getDriverContext().getCtx().getExternalTmpPath(originalDir); + Path tmpPath = context.getDriverQueue().getCtx().getExternalTmpPath(originalDir); FileSystem fs = null; try { diff --git ql/src/java/org/apache/hadoop/hive/ql/ddl/view/MaterializedViewUpdateOperation.java ql/src/java/org/apache/hadoop/hive/ql/ddl/view/MaterializedViewUpdateOperation.java index ad6e163a4f..90c155d7b4 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ddl/view/MaterializedViewUpdateOperation.java +++ ql/src/java/org/apache/hadoop/hive/ql/ddl/view/MaterializedViewUpdateOperation.java @@ -40,7 +40,7 @@ public MaterializedViewUpdateOperation(DDLOperationContext context, Materialized @Override public int execute() throws HiveException { - if (context.getDriverContext().getCtx().getExplainAnalyze() == AnalyzeState.RUNNING) { + if (context.getDriverQueue().getCtx().getExplainAnalyze() == AnalyzeState.RUNNING) { return 0; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java index 10a2947b16..c75fcd3ef3 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java @@ -46,7 +46,7 @@ import org.apache.hadoop.hive.metastore.columnstats.cache.LongColumnStatsDataInspector; import org.apache.hadoop.hive.metastore.columnstats.cache.StringColumnStatsDataInspector; import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils; @@ -75,9 +75,9 @@ .getLogger(ColumnStatsUpdateTask.class); @Override - public void initialize(QueryState queryState, QueryPlan queryPlan, DriverContext ctx, + public void initialize(QueryState queryState, QueryPlan queryPlan, DriverQueue driverQueue, CompilationOpContext opContext) { - super.initialize(queryState, queryPlan, ctx, opContext); + super.initialize(queryState, queryPlan, driverQueue, opContext); } private ColumnStatistics constructColumnStatsFromInput() @@ -343,7 +343,7 @@ private int persistColumnStats(Hive db) throws HiveException, MetaException, IOE } @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { try { Hive db = getHive(); return persistColumnStats(db); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/ConditionalTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/ConditionalTask.java index ecdf368b52..86a29f2a40 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/ConditionalTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ConditionalTask.java @@ -21,7 +21,7 @@ import java.io.Serializable; import java.util.List; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.ConditionalResolver; import org.apache.hadoop.hive.ql.plan.ConditionalWork; @@ -75,12 +75,12 @@ public boolean hasReduce() { } @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { resTasks = resolver.getTasks(conf, resolverCtx); resolved = true; try { - resolveTask(driverContext); + resolveTask(driverQueue); } catch (Exception e) { setException(e); return 1; @@ -88,13 +88,13 @@ public int execute(DriverContext driverContext) { return 0; } - private void resolveTask(DriverContext driverContext) throws HiveException { + private void resolveTask(DriverQueue driverQueue) throws HiveException { for (Task tsk : getListTasks()) { if (!resTasks.contains(tsk)) { - driverContext.remove(tsk); + driverQueue.remove(tsk); console.printInfo(tsk.getId() + " is filtered out by condition resolver."); if (tsk.isMapRedTask()) { - driverContext.incCurJobNo(1); + driverQueue.incCurJobNo(1); } //recursively remove this task from its children's parent task tsk.removeFromChildrenTasks(); @@ -106,7 +106,7 @@ private void resolveTask(DriverContext driverContext) throws HiveException { } } // resolved task - if (driverContext.addToRunnable(tsk)) { + if (driverQueue.addToRunnable(tsk)) { console.printInfo(tsk.getId() + " is selected by condition resolver."); } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/CopyTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/CopyTask.java index 1a8e5e79e1..fb5bcb968a 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/CopyTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/CopyTask.java @@ -31,7 +31,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hive.common.FileUtils; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.parse.repl.dump.io.FileOperations; import org.apache.hadoop.hive.ql.plan.CopyWork; @@ -51,7 +51,7 @@ public CopyTask() { } @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { Path[] from = work.getFromPaths(), to = work.getToPaths(); for (int i = 0; i < from.length; ++i) { int result = copyOnePath(from[i], to[i]); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/DependencyCollectionTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/DependencyCollectionTask.java index 95b8205249..17fc8c22f4 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/DependencyCollectionTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/DependencyCollectionTask.java @@ -20,7 +20,7 @@ import java.io.Serializable; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.plan.DependencyCollectionWork; import org.apache.hadoop.hive.ql.plan.api.StageType; @@ -39,7 +39,7 @@ public DependencyCollectionTask() { } @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { return 0; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainSQRewriteTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainSQRewriteTask.java index 1f9e9aa1b9..79460d8616 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainSQRewriteTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainSQRewriteTask.java @@ -29,7 +29,7 @@ import org.antlr.runtime.TokenRewriteStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.parse.ASTNode; import org.apache.hadoop.hive.ql.parse.HiveParser; import org.apache.hadoop.hive.ql.parse.QB; @@ -52,7 +52,7 @@ public StageType getType() { } @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { PrintStream out = null; try { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java index a0f3e15b63..4d421e7ef1 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java @@ -46,7 +46,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.LockComponent; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.metadata.Table; @@ -398,7 +398,7 @@ private JSONObject getLocks(PrintStream out, ExplainWork work) { } @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { PrintStream out = null; try { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java index ae9d040ef8..d6a1e987ec 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java @@ -17,7 +17,7 @@ Licensed to the Apache Software Foundation (ASF) under one */ package org.apache.hadoop.hive.ql.exec; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.parse.SemanticException; @@ -44,7 +44,7 @@ public String getName() { } @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { try { // Also creates the root directory TableExport.Paths exportPaths = new TableExport.Paths( diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java index 93b115841b..a120c981c6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java @@ -25,7 +25,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.mr.ExecMapper; @@ -62,7 +62,7 @@ public void setValidWriteIdList(String writeIdStr) { } @Override - public void initialize(QueryState queryState, QueryPlan queryPlan, DriverContext ctx, + public void initialize(QueryState queryState, QueryPlan queryPlan, DriverQueue ctx, CompilationOpContext opContext) { super.initialize(queryState, queryPlan, ctx, opContext); work.initializeForFetch(opContext); @@ -109,7 +109,7 @@ public void initialize(QueryState queryState, QueryPlan queryPlan, DriverContext } @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { assert false; return 0; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java index 695d08bbe2..4873352e36 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java @@ -30,7 +30,7 @@ import org.apache.hadoop.hive.metastore.api.Order; import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.ddl.DDLUtils; import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; @@ -229,7 +229,7 @@ private void releaseLocks(LoadTableDesc ltd) throws HiveException { return; } - Context ctx = driverContext.getCtx(); + Context ctx = driverQueue.getCtx(); if (ctx.getHiveTxnManager().supportsAcid()) { //Acid LM doesn't maintain getOutputLockObjects(); this 'if' just makes logic more explicit return; @@ -302,7 +302,7 @@ public TaskInformation(Task task, String path) { } @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) { Utilities.FILE_OP_LOGGER.trace("Executing MoveWork " + System.identityHashCode(work) + " with " + work.getLoadFileWork() + "; " + work.getLoadTableWork() + "; " @@ -310,7 +310,7 @@ public int execute(DriverContext driverContext) { } try { - if (driverContext.getCtx().getExplainAnalyze() == AnalyzeState.RUNNING) { + if (driverQueue.getCtx().getExplainAnalyze() == AnalyzeState.RUNNING) { return 0; } Hive db = getHive(); @@ -404,7 +404,7 @@ public int execute(DriverContext driverContext) { throw new HiveException("MoveTask : Write id is not set in the config by open txn task for migration"); } tbd.setWriteId(writeId); - tbd.setStmtId(driverContext.getCtx().getHiveTxnManager().getStmtIdAndIncrement()); + tbd.setStmtId(driverQueue.getCtx().getHiveTxnManager().getStmtIdAndIncrement()); } boolean isFullAcidOp = work.getLoadTableWork().getWriteType() != AcidUtils.Operation.NOT_ACID diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java index 6b3635d05a..539dc21cc4 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java @@ -45,7 +45,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.FileUtils; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.parse.LoadSemanticAnalyzer; import org.apache.hadoop.hive.ql.plan.api.StageType; import org.apache.hadoop.util.StringUtils; @@ -133,7 +133,7 @@ private void renameFileCopiedFromCmPath(Path toPath, FileSystem dstFs, List processors = new ArrayList<>(); @Override - public void initialize(QueryState queryState, QueryPlan queryPlan, DriverContext ctx, + public void initialize(QueryState queryState, QueryPlan queryPlan, DriverQueue ctx, CompilationOpContext opContext) { super.initialize(queryState, queryPlan, ctx, opContext); @@ -86,8 +86,8 @@ public void initialize(QueryState queryState, QueryPlan queryPlan, DriverContext @Override - public int execute(DriverContext driverContext) { - if (driverContext.getCtx().getExplainAnalyze() == AnalyzeState.RUNNING) { + public int execute(DriverQueue driverQueue) { + if (driverQueue.getCtx().getExplainAnalyze() == AnalyzeState.RUNNING) { return 0; } if (work.isAggregating() && work.isFooterScan()) { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java index 8eea9cfbea..14b0d71460 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java @@ -21,7 +21,7 @@ import org.apache.hadoop.hive.common.metrics.common.Metrics; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.QueryDisplay; import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.QueryState; @@ -61,7 +61,7 @@ protected transient QueryState queryState; protected transient LogHelper console; protected transient QueryPlan queryPlan; - protected transient DriverContext driverContext; + protected transient DriverQueue driverQueue; protected transient boolean clonedConf = false; protected transient String jobID; protected Task backupTask; @@ -142,7 +142,7 @@ public String getStatusMessage() { protected List> parentTasks; /** * this can be set by the Task, to provide more info about the failure in TaskResult - * where the Driver can find it. This is checked if {@link Task#execute(org.apache.hadoop.hive.ql.DriverContext)} + * where the Driver can find it. This is checked if {@link Task#execute(org.apache.hadoop.hive.ql.DriverQueue)} * returns non-0 code. */ private Throwable exception; @@ -156,7 +156,7 @@ public TaskHandle getTaskHandle() { return taskHandle; } - public void initialize(QueryState queryState, QueryPlan queryPlan, DriverContext driverContext, + public void initialize(QueryState queryState, QueryPlan queryPlan, DriverQueue driverQueue, CompilationOpContext opContext) { this.queryPlan = queryPlan; setInitialized(); @@ -164,7 +164,7 @@ public void initialize(QueryState queryState, QueryPlan queryPlan, DriverContext if (null == this.conf) { this.conf = queryState.getConf(); } - this.driverContext = driverContext; + this.driverQueue = driverQueue; console = new LogHelper(LOG); } public void setQueryDisplay(QueryDisplay queryDisplay) { @@ -209,7 +209,7 @@ public int executeTask(HiveHistory hiveHistory) { if (conf != null) { LOG.debug("Task getting executed using mapred tag : " + conf.get(MRJobConfig.JOB_TAGS)); } - int retval = execute(driverContext); + int retval = execute(driverQueue); this.setDone(); if (hiveHistory != null) { hiveHistory.logPlanProgress(queryPlan); @@ -225,7 +225,7 @@ public int executeTask(HiveHistory hiveHistory) { * * @return status of executing the task */ - public abstract int execute(DriverContext driverContext); + public abstract int execute(DriverQueue driverQueue); public boolean isRootTask() { return rootTask; @@ -594,12 +594,12 @@ public QueryPlan getQueryPlan() { return queryPlan; } - public DriverContext getDriverContext() { - return driverContext; + public DriverQueue getDriverQueue() { + return driverQueue; } - public void setDriverContext(DriverContext driverContext) { - this.driverContext = driverContext; + public void setDriverQueue(DriverQueue driverQueue) { + this.driverQueue = driverQueue; } public void setQueryPlan(QueryPlan queryPlan) { @@ -657,6 +657,6 @@ public QueryState getQueryState() { } public HiveTxnManager getTxnMgr() { - return driverContext.getCtx().getHiveTxnManager(); + return driverQueue.getCtx().getHiveTxnManager(); } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/TaskRunner.java ql/src/java/org/apache/hadoop/hive/ql/exec/TaskRunner.java index a5554c3004..39981e875e 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/TaskRunner.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/TaskRunner.java @@ -22,7 +22,7 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.hive.common.LogUtils; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.session.SessionState; import org.slf4j.Logger; @@ -48,13 +48,13 @@ protected Long initialValue() { private static transient final Logger LOG = LoggerFactory.getLogger(TaskRunner.class); - private final DriverContext driverCtx; + private final DriverQueue driverQueue; - public TaskRunner(Task tsk, DriverContext ctx) { + public TaskRunner(Task tsk, DriverQueue driverQueue) { this.tsk = tsk; this.result = new TaskResult(); - ss = SessionState.get(); - driverCtx = ctx; + this.ss = SessionState.get(); + this.driverQueue = driverQueue; } public Task getTask() { @@ -108,7 +108,7 @@ public void runSequential() { LOG.error("Error in executeTask", t); } result.setExitVal(exitVal); - driverCtx.releaseRunnable(); + driverQueue.releaseRunnable(); if (tsk.getException() != null) { result.setTaskError(tsk.getException()); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java index ab1b52e07e..c7b95780b6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java @@ -53,7 +53,7 @@ import org.apache.hadoop.hive.conf.HiveConfUtil; import org.apache.hadoop.hive.ql.CompilationOpContext; import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.QueryState; @@ -172,9 +172,9 @@ protected static String getResource(HiveConf conf, SessionState.ResourceType res * Initialization when invoked from QL. */ @Override - public void initialize(QueryState queryState, QueryPlan queryPlan, DriverContext driverContext, + public void initialize(QueryState queryState, QueryPlan queryPlan, DriverQueue driverQueue, CompilationOpContext opContext) { - super.initialize(queryState, queryPlan, driverContext, opContext); + super.initialize(queryState, queryPlan, driverQueue, opContext); job = new JobConf(conf, ExecDriver.class); @@ -219,19 +219,19 @@ public boolean checkFatalErrors(Counters ctrs, StringBuilder errMsg) { */ @SuppressWarnings({"deprecation", "unchecked"}) @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { IOPrepareCache ioPrepareCache = IOPrepareCache.get(); ioPrepareCache.clear(); boolean success = true; - Context ctx = driverContext.getCtx(); + Context ctx = driverQueue.getCtx(); boolean ctxCreated = false; Path emptyScratchDir; JobClient jc = null; - if (driverContext.isShutdown()) { + if (driverQueue.isShutdown()) { LOG.warn("Task was cancelled"); return 5; } @@ -410,14 +410,14 @@ public int execute(DriverContext driverContext) { HiveConfUtil.updateJobCredentialProviders(job); // Finally SUBMIT the JOB! - if (driverContext.isShutdown()) { + if (driverQueue.isShutdown()) { LOG.warn("Task was cancelled"); return 5; } rj = jc.submitJob(job); - if (driverContext.isShutdown()) { + if (driverQueue.isShutdown()) { LOG.warn("Task was cancelled"); killJob(); return 5; @@ -766,12 +766,12 @@ public static void main(String[] args) throws IOException, HiveException { memoryMXBean = ManagementFactory.getMemoryMXBean(); MapredLocalWork plan = SerializationUtilities.deserializePlan(pathData, MapredLocalWork.class); MapredLocalTask ed = new MapredLocalTask(plan, conf, isSilent); - ret = ed.executeInProcess(new DriverContext()); + ret = ed.executeInProcess(new DriverQueue()); } else { MapredWork plan = SerializationUtilities.deserializePlan(pathData, MapredWork.class); ExecDriver ed = new ExecDriver(plan, conf, isSilent); - ret = ed.execute(new DriverContext()); + ret = ed.execute(new DriverQueue()); } if (ret != 0) { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java index 0f594a1625..f47b310e8c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hive.common.metrics.common.MetricsConstant; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.MapRedStats; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.SerializationUtilities; @@ -92,9 +92,9 @@ public MapRedTask() { } @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { - Context ctx = driverContext.getCtx(); + Context ctx = driverQueue.getCtx(); boolean ctxCreated = false; try { @@ -111,7 +111,7 @@ public int execute(DriverContext driverContext) { conf.getBoolVar(HiveConf.ConfVars.LOCALMODEAUTO)) { if (inputSummary == null) { - inputSummary = Utilities.getInputSummary(driverContext.getCtx(), work.getMapWork(), null); + inputSummary = Utilities.getInputSummary(driverQueue.getCtx(), work.getMapWork(), null); } // set the values of totalInputFileSize and totalInputNumFiles, estimating them @@ -156,7 +156,7 @@ public int execute(DriverContext driverContext) { } // we are not running this mapred task via child jvm // so directly invoke ExecDriver - int ret = super.execute(driverContext); + int ret = super.execute(driverQueue); // restore the previous properties for framework name, RM address etc. if (this.isLocalMode()) { @@ -442,7 +442,7 @@ private void setNumberOfReducers() throws IOException { + reducers); } else { if (inputSummary == null) { - inputSummary = Utilities.getInputSummary(driverContext.getCtx(), work.getMapWork(), null); + inputSummary = Utilities.getInputSummary(driverQueue.getCtx(), work.getMapWork(), null); } int reducers = Utilities.estimateNumberOfReducers(conf, inputSummary, work.getMapWork(), work.isFinalMapRed()); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java index 4bc7568f9d..4f85979050 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java @@ -50,7 +50,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.ql.CompilationOpContext; import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.BucketMatcher; @@ -132,9 +132,9 @@ public void updateTaskMetrics(Metrics metrics) { } @Override - public void initialize(QueryState queryState, QueryPlan queryPlan, DriverContext driverContext, + public void initialize(QueryState queryState, QueryPlan queryPlan, DriverQueue driverQueue, CompilationOpContext opContext) { - super.initialize(queryState, queryPlan, driverContext, opContext); + super.initialize(queryState, queryPlan, driverQueue, opContext); job = new JobConf(conf, ExecDriver.class); execContext = new ExecMapperContext(job); //we don't use the HadoopJobExecHooks for local tasks @@ -153,21 +153,21 @@ public boolean requireLock() { } @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { if (conf.getBoolVar(HiveConf.ConfVars.SUBMITLOCALTASKVIACHILD)) { // send task off to another jvm - return executeInChildVM(driverContext); + return executeInChildVM(driverQueue); } else { // execute in process - return executeInProcess(driverContext); + return executeInProcess(driverQueue); } } - public int executeInChildVM(DriverContext driverContext) { + public int executeInChildVM(DriverQueue driverQueue) { // execute in child jvm try { // generate the cmd line to run in the child jvm - Context ctx = driverContext.getCtx(); + Context ctx = driverQueue.getCtx(); String hiveJar = conf.getJar(); String hadoopExec = conf.getVar(HiveConf.ConfVars.HADOOPBIN); @@ -370,7 +370,7 @@ public int executeInChildVM(DriverContext driverContext) { } } - public int executeInProcess(DriverContext driverContext) { + public int executeInProcess(DriverQueue driverQueue) { // check the local work if (work == null) { return -1; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ExternalTableCopyTaskBuilder.java ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ExternalTableCopyTaskBuilder.java index 1af92271f3..ae95993cad 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ExternalTableCopyTaskBuilder.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ExternalTableCopyTaskBuilder.java @@ -20,7 +20,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.FileUtils; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.TaskFactory; @@ -163,7 +163,7 @@ private int handleException(Exception e, Path sourcePath, Path targetPath, } @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { String distCpDoAsUser = conf.getVar(HiveConf.ConfVars.HIVE_DISTCP_DOAS_USER); Path sourcePath = work.fullyQualifiedSourcePath; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java index 32cb38ae90..3344f927d3 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java @@ -38,7 +38,7 @@ import org.apache.hadoop.hive.metastore.messaging.event.filters.AndFilter; import org.apache.hadoop.hive.metastore.messaging.event.filters.EventBoundaryFilter; import org.apache.hadoop.hive.metastore.messaging.event.filters.ReplEventFilter; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils; @@ -117,7 +117,7 @@ public String getName() { } @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { try { Hive hiveDb = getHive(); Path dumpRoot = new Path(conf.getVar(HiveConf.ConfVars.REPLDIR), getNextDumpDir()); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplLoadTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplLoadTask.java index 9a541d2873..ef9d429c7d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplLoadTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplLoadTask.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.InvalidInputException; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.ddl.DDLWork; import org.apache.hadoop.hive.ql.ddl.database.alter.poperties.AlterDatabaseSetPropertiesDesc; @@ -97,7 +97,7 @@ public StageType getType() { } @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { Task rootTask = work.getRootTask(); if (rootTask != null) { rootTask.setChildTasks(null); @@ -105,17 +105,17 @@ public int execute(DriverContext driverContext) { work.setRootTask(this); this.parentTasks = null; if (work.isIncrementalLoad()) { - return executeIncrementalLoad(driverContext); + return executeIncrementalLoad(driverQueue); } else { - return executeBootStrapLoad(driverContext); + return executeBootStrapLoad(driverQueue); } } - private int executeBootStrapLoad(DriverContext driverContext) { + private int executeBootStrapLoad(DriverQueue driverQueue) { try { int maxTasks = conf.getIntVar(HiveConf.ConfVars.REPL_APPROX_MAX_LOAD_TASKS); Context context = new Context(work.dumpDirectory, conf, getHive(), - work.sessionStateLineageState, driverContext.getCtx()); + work.sessionStateLineageState, driverQueue.getCtx()); TaskTracker loadTaskTracker = new TaskTracker(maxTasks); /* for now for simplicity we are doing just one directory ( one database ), come back to use @@ -286,7 +286,7 @@ a database ( directory ) LOG.info("Root Tasks / Total Tasks : {} / {} ", childTasks.size(), loadTaskTracker.numberOfTasks()); // Populate the driver context with the scratch dir info from the repl context, so that the temp dirs will be cleaned up later - driverContext.getCtx().getFsScratchDirs().putAll(context.pathInfo.getFsScratchDirs()); + driverQueue.getCtx().getFsScratchDirs().putAll(context.pathInfo.getFsScratchDirs()); } catch (RuntimeException e) { LOG.error("replication failed with run time exception", e); throw e; @@ -484,7 +484,7 @@ private void createBuilderTask(List> rootTasks) { DAGTraversal.traverse(rootTasks, new AddDependencyToLeaves(loadTask)); } - private int executeIncrementalLoad(DriverContext driverContext) { + private int executeIncrementalLoad(DriverQueue driverQueue) { try { // If user has requested to cleanup any bootstrap dump, then just do it before incremental load. if (work.needCleanTablesFromBootstrap) { @@ -503,7 +503,7 @@ private int executeIncrementalLoad(DriverContext driverContext) { if (work.hasBootstrapLoadTasks()) { LOG.debug("Current incremental dump have tables to be bootstrapped. Switching to bootstrap " + "mode after applying all events."); - return executeBootStrapLoad(driverContext); + return executeBootStrapLoad(driverQueue); } } @@ -517,7 +517,7 @@ private int executeIncrementalLoad(DriverContext driverContext) { if (work.getPathsToCopyIterator().hasNext()) { childTasks.addAll(new ExternalTableCopyTaskBuilder(work, conf).tasks(tracker)); } else { - childTasks.add(builder.build(driverContext, getHive(), LOG, tracker)); + childTasks.add(builder.build(driverQueue, getHive(), LOG, tracker)); } // If there are no more events to be applied, add a task to update the last.repl.id of the diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplStateLogTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplStateLogTask.java index 845aad14e3..e7a03ff455 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplStateLogTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplStateLogTask.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hive.ql.exec.repl; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.plan.api.StageType; @@ -34,7 +34,7 @@ private static final long serialVersionUID = 1L; @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { work.replStateLog(); return 0; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java index ed75df88f0..1a6fa4caeb 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.ReplLastIdInfo; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.ddl.DDLWork; import org.apache.hadoop.hive.ql.ddl.database.alter.poperties.AlterDatabaseSetPropertiesDesc; import org.apache.hadoop.hive.ql.ddl.misc.ReplRemoveFirstIncLoadPendFlagDesc; @@ -89,7 +89,7 @@ public IncrementalLoadTasksBuilder(String dbName, String loadPath, numIteration = 0; } - public Task build(DriverContext driverContext, Hive hive, Logger log, + public Task build(DriverQueue driverQueue, Hive hive, Logger log, TaskTracker tracker) throws Exception { Task evTaskRoot = TaskFactory.get(new DependencyCollectionWork()); Task taskChainTail = evTaskRoot; @@ -132,7 +132,7 @@ public IncrementalLoadTasksBuilder(String dbName, String loadPath, // entire chain MessageHandler.Context context = new MessageHandler.Context(dbName, location, - taskChainTail, eventDmd, conf, hive, driverContext.getCtx(), this.log); + taskChainTail, eventDmd, conf, hive, driverQueue.getCtx(), this.log); List> evTasks = analyzeEventLoad(context); if ((evTasks != null) && (!evTasks.isEmpty())) { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java index 8a44c1b0a7..a61448bdd0 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java @@ -20,21 +20,21 @@ import java.io.Closeable; import java.io.Serializable; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; import org.apache.hadoop.hive.ql.plan.SparkWork; import org.apache.spark.SparkConf; public interface HiveSparkClient extends Serializable, Closeable { /** - * HiveSparkClient should generate Spark RDD graph by given sparkWork and driverContext, + * HiveSparkClient should generate Spark RDD graph by given sparkWork and driverQueue, * and submit RDD graph to Spark cluster. - * @param driverContext + * @param driverQueue * @param sparkWork * @return SparkJobRef could be used to track spark job progress and metrics. * @throws Exception */ - SparkJobRef execute(DriverContext driverContext, SparkWork sparkWork) throws Exception; + SparkJobRef execute(DriverQueue driverQueue, SparkWork sparkWork) throws Exception; /** * @return spark configuration diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java index 6c6122a88c..2f7193fb36 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConfUtil; import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; import org.apache.hadoop.hive.ql.exec.spark.status.impl.JobMetricsListener; @@ -124,8 +124,8 @@ public int getDefaultParallelism() throws Exception { } @Override - public SparkJobRef execute(DriverContext driverContext, SparkWork sparkWork) throws Exception { - Context ctx = driverContext.getCtx(); + public SparkJobRef execute(DriverQueue driverQueue, SparkWork sparkWork) throws Exception { + Context ctx = driverQueue.getCtx(); HiveConf hiveConf = (HiveConf) ctx.getConf(); refreshLocalResources(sparkWork, hiveConf); JobConf jobConf = new JobConf(hiveConf); @@ -157,7 +157,7 @@ public SparkJobRef execute(DriverContext driverContext, SparkWork sparkWork) thr new SparkPlanGenerator(sc, ctx, jobConf, emptyScratchDir, sparkReporter); SparkPlan plan = gen.generate(sparkWork); - if (driverContext.isShutdown()) { + if (driverQueue.isShutdown()) { throw new HiveException("Operation is cancelled."); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java index 49b761450c..e0103164ef 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java @@ -45,7 +45,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.conf.HiveConfUtil; import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.exec.DagUtils; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; @@ -181,7 +181,7 @@ public int getDefaultParallelism() throws Exception { } @Override - public SparkJobRef execute(final DriverContext driverContext, final SparkWork sparkWork) + public SparkJobRef execute(final DriverQueue driverQueue, final SparkWork sparkWork) throws Exception { if (SparkClientUtilities.isYarnMaster(hiveConf.get("spark.master")) && !remoteClient.isActive()) { @@ -191,14 +191,14 @@ public SparkJobRef execute(final DriverContext driverContext, final SparkWork sp } try { - return submit(driverContext, sparkWork); + return submit(driverQueue, sparkWork); } catch (Throwable cause) { throw new Exception("Failed to submit Spark work, please retry later", cause); } } - private SparkJobRef submit(final DriverContext driverContext, final SparkWork sparkWork) throws Exception { - final Context ctx = driverContext.getCtx(); + private SparkJobRef submit(final DriverQueue driverQueue, final SparkWork sparkWork) throws Exception { + final Context ctx = driverQueue.getCtx(); final HiveConf hiveConf = (HiveConf) ctx.getConf(); refreshLocalResources(sparkWork, hiveConf); final JobConf jobConf = new JobConf(hiveConf); @@ -220,7 +220,7 @@ private SparkJobRef submit(final DriverContext driverContext, final SparkWork sp byte[] sparkWorkBytes = KryoSerializer.serialize(sparkWork); JobStatusJob job = new JobStatusJob(jobConfBytes, scratchDirBytes, sparkWorkBytes); - if (driverContext.isShutdown()) { + if (driverQueue.isShutdown()) { throw new HiveException("Operation is cancelled."); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java index 92775107bc..6609d9b1a1 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java @@ -45,7 +45,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.QueryState; @@ -104,13 +104,13 @@ private transient boolean jobKilled = false; @Override - public void initialize(QueryState queryState, QueryPlan queryPlan, DriverContext driverContext, + public void initialize(QueryState queryState, QueryPlan queryPlan, DriverQueue driverQueue, CompilationOpContext opContext) { - super.initialize(queryState, queryPlan, driverContext, opContext); + super.initialize(queryState, queryPlan, driverQueue, opContext); } @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { int rc = 0; perfLogger = SessionState.getPerfLogger(); @@ -127,11 +127,11 @@ public int execute(DriverContext driverContext) { // Submit the Spark job perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_SUBMIT_JOB); submitTime = perfLogger.getStartTime(PerfLogger.SPARK_SUBMIT_JOB); - jobRef = sparkSession.submit(driverContext, sparkWork); + jobRef = sparkSession.submit(driverQueue, sparkWork); perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_SUBMIT_JOB); // If the driver context has been shutdown (due to query cancellation) kill the Spark job - if (driverContext.isShutdown()) { + if (driverQueue.isShutdown()) { LOG.warn("Killing Spark job"); killJob(); throw new HiveException(String.format("Spark task %s cancelled for query %s", getId(), sparkWork.getQueryId())); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java index b29be4897d..8f16ca875f 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java @@ -20,7 +20,7 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.SparkWork; @@ -37,11 +37,11 @@ /** * Submit given sparkWork to SparkClient. - * @param driverContext + * @param driverQueue * @param sparkWork * @return SparkJobRef */ - SparkJobRef submit(DriverContext driverContext, SparkWork sparkWork) throws Exception; + SparkJobRef submit(DriverQueue driverQueue, SparkWork sparkWork) throws Exception; /** * Get Spark shuffle memory per task, and total number of cores. This diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java index 7e643420b5..2aa7476297 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java @@ -43,7 +43,7 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.exec.spark.HiveSparkClient; import org.apache.hadoop.hive.ql.exec.spark.HiveSparkClientFactory; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; @@ -142,11 +142,11 @@ public void open(HiveConf conf) throws HiveException { } @Override - public SparkJobRef submit(DriverContext driverContext, SparkWork sparkWork) throws Exception { + public SparkJobRef submit(DriverQueue driverQueue, SparkWork sparkWork) throws Exception { closeLock.readLock().lock(); try { Preconditions.checkState(isOpen, "Hive on Spark session is not open. Can't submit jobs."); - return hiveSparkClient.execute(driverContext, sparkWork); + return hiveSparkClient.execute(driverQueue, sparkWork); } finally { closeLock.readLock().unlock(); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java index aecd1084e6..61f1e1975f 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java @@ -40,7 +40,7 @@ import org.apache.hadoop.hive.common.metrics.common.MetricsConstant; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.Task; @@ -129,7 +129,7 @@ public TezCounters getTezCounters() { @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { int rc = 1; boolean cleanContext = false; Context ctx = null; @@ -137,7 +137,7 @@ public int execute(DriverContext driverContext) { try { // Get or create Context object. If we create it we have to clean it later as well. - ctx = driverContext.getCtx(); + ctx = driverQueue.getCtx(); if (ctx == null) { ctx = new Context(conf); cleanContext = true; diff --git ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecWMEventsSummaryPrinter.java ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecWMEventsSummaryPrinter.java index 94c66c0d93..300a403071 100644 --- ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecWMEventsSummaryPrinter.java +++ ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecWMEventsSummaryPrinter.java @@ -52,7 +52,7 @@ public void run(HookContext hookContext) throws Exception { List rootTasks = Utilities.getTezTasks(plan.getRootTasks()); for (TezTask tezTask : rootTasks) { - WmContext wmContext = tezTask.getDriverContext().getCtx().getWmContext(); + WmContext wmContext = tezTask.getDriverQueue().getCtx().getWmContext(); if (wmContext != null) { wmContext.printJson(console); wmContext.shortPrint(console); diff --git ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java index acc52afdc3..bc2bb6ffc3 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.CompilationOpContext; import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.Operator; @@ -63,8 +63,8 @@ @Override public void initialize(QueryState queryState, QueryPlan queryPlan, - DriverContext driverContext, CompilationOpContext opContext) { - super.initialize(queryState, queryPlan, driverContext, opContext); + DriverQueue driverQueue, CompilationOpContext opContext) { + super.initialize(queryState, queryPlan, driverQueue, opContext); job = new JobConf(conf, MergeFileTask.class); jobExecHelper = new HadoopJobExecHelper(job, this.console, this, this); } @@ -78,9 +78,9 @@ public boolean requireLock() { * start a new map-reduce job to do the merge, almost the same as ExecDriver. */ @Override - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { - Context ctx = driverContext.getCtx(); + Context ctx = driverQueue.getCtx(); boolean ctxCreated = false; RunningJob rj = null; int returnVal = 0; diff --git ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java index 8f21f7c69e..d1da9f14b4 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.CompilationOpContext; import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.Task; @@ -61,8 +61,8 @@ @Override public void initialize(QueryState queryState, QueryPlan queryPlan, - DriverContext driverContext, CompilationOpContext opContext) { - super.initialize(queryState, queryPlan, driverContext, opContext); + DriverQueue driverQueue, CompilationOpContext opContext) { + super.initialize(queryState, queryPlan, driverQueue, opContext); job = new JobConf(conf, ColumnTruncateTask.class); jobExecHelper = new HadoopJobExecHelper(job, this.console, this, this); } @@ -78,7 +78,7 @@ public boolean requireLock() { /** * start a new map-reduce job to do the truncation, almost the same as ExecDriver. */ - public int execute(DriverContext driverContext) { + public int execute(DriverQueue driverQueue) { HiveConf.setVar(job, HiveConf.ConfVars.HIVEINPUTFORMAT, BucketizedHiveInputFormat.class.getName()); success = true; @@ -86,7 +86,7 @@ public int execute(DriverContext driverContext) { job.setOutputFormat(HiveOutputFormatImpl.class); job.setMapperClass(work.getMapperClass()); - Context ctx = driverContext.getCtx(); + Context ctx = driverQueue.getCtx(); boolean ctxCreated = false; try { if (ctx == null) { diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java index 424027077a..86c89342c7 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.ddl.DDLTask; import org.apache.hadoop.hive.ql.ddl.DDLWork; @@ -152,7 +152,7 @@ private void analyzeAcidExport(ASTNode ast) throws SemanticException { inputs.add(dbForTmpTable); //so the plan knows we are 'reading' this db - locks, security... DDLTask createTableTask = (DDLTask) TaskFactory.get(new DDLWork(new HashSet<>(), new HashSet<>(), ctlt), conf); createTableTask.setConf(conf); //above get() doesn't set it - createTableTask.execute(new DriverContext(new Context(conf))); + createTableTask.execute(new DriverQueue(new Context(conf))); newTable = db.getTable(newTableName); } catch(IOException|HiveException ex) { throw new SemanticException(ex); diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/BasicStatsWork.java ql/src/java/org/apache/hadoop/hive/ql/plan/BasicStatsWork.java index 40def601e6..9ef55e6610 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/BasicStatsWork.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/BasicStatsWork.java @@ -57,7 +57,7 @@ // sourceTask for TS is not changed (currently) but that of FS might be changed // by various optimizers (auto.convert.join, for example) - // so this is set by DriverContext in runtime + // so this is set by DriverQueue in runtime private transient Task sourceTask; private boolean isFollowedByColStats = false; diff --git ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java index a78fdfc394..9941210fda 100644 --- ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java +++ ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java @@ -34,7 +34,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.mr.ExecDriver; import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; @@ -481,10 +481,10 @@ private void populateMapRedPlan6(Table src) throws Exception { private void executePlan() throws Exception { String testName = new Exception().getStackTrace()[1].getMethodName(); MapRedTask mrtask = new MapRedTask(); - DriverContext dctx = new DriverContext(); + DriverQueue driverQueue = new DriverQueue(); mrtask.setWork(mr); - mrtask.initialize(queryState, null, dctx, null); - int exitVal = mrtask.execute(dctx); + mrtask.initialize(queryState, null, driverQueue, null); + int exitVal = mrtask.execute(driverQueue); if (exitVal != 0) { LOG.error(testName + " execution failed with exit status: " diff --git ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java index c9b68fa768..30e3196eb4 100644 --- ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java +++ ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.common.metrics.common.MetricsConstant; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.spark.SparkTask; import org.apache.hadoop.hive.ql.plan.MapWork; @@ -67,7 +67,7 @@ public void mrTaskSumbitViaChildWithImpersonation() throws IOException, LoginExc Context ctx = Mockito.mock(Context.class); when(ctx.getLocalTmpPath()).thenReturn(new Path(System.getProperty("java.io.tmpdir"))); - DriverContext dctx = new DriverContext(ctx); + DriverQueue driverQueue = new DriverQueue(ctx); QueryState queryState = new QueryState.Builder().build(); HiveConf conf= queryState.getConf(); @@ -79,12 +79,12 @@ public void mrTaskSumbitViaChildWithImpersonation() throws IOException, LoginExc MapRedTask mrTask = Mockito.spy(new MapRedTask()); mrTask.setWork(mrWork); - mrTask.initialize(queryState, null, dctx, null); + mrTask.initialize(queryState, null, driverQueue, null); mrTask.jobExecHelper = Mockito.mock(HadoopJobExecHelper.class); when(mrTask.jobExecHelper.progressLocal(Mockito.any(Process.class), Mockito.anyString())).thenReturn(0); - mrTask.execute(dctx); + mrTask.execute(driverQueue); ArgumentCaptor captor = ArgumentCaptor.forClass(String[].class); verify(mrTask).spawn(Mockito.anyString(), Mockito.anyString(), captor.capture()); diff --git ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java index 2017fc15f9..e03716456a 100644 --- ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java +++ ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hive.common.metrics.common.Metrics; import org.apache.hadoop.hive.common.metrics.common.MetricsConstant; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsBuilder; @@ -195,12 +195,12 @@ public void testHandleInterruptedException() throws Exception { SparkTask sparkTask = new SparkTask(); sparkTask.setWork(mock(SparkWork.class)); - DriverContext mockDriverContext = mock(DriverContext.class); + DriverQueue mockDriverQueue = mock(DriverQueue.class); QueryState mockQueryState = mock(QueryState.class); when(mockQueryState.getConf()).thenReturn(hiveConf); - sparkTask.initialize(mockQueryState, null, mockDriverContext, null); + sparkTask.initialize(mockQueryState, null, mockDriverQueue, null); SparkJobStatus mockSparkJobStatus = mock(SparkJobStatus.class); when(mockSparkJobStatus.getMonitorError()).thenReturn(new InterruptedException()); @@ -215,14 +215,14 @@ public void testHandleInterruptedException() throws Exception { SessionState.start(hiveConf); SessionState.get().setSparkSession(mockSparkSession); - sparkTask.execute(mockDriverContext); + sparkTask.execute(mockDriverQueue); verify(mockSparkJobRef, atLeastOnce()).cancelJob(); when(mockSparkJobStatus.getMonitorError()).thenReturn( new HiveException(new InterruptedException())); - sparkTask.execute(mockDriverContext); + sparkTask.execute(mockDriverQueue); verify(mockSparkJobRef, atLeastOnce()).cancelJob(); } diff --git ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java index 0030649aae..4786f4706a 100644 --- ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java +++ ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java @@ -22,7 +22,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.DriverQueue; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; @@ -58,7 +58,7 @@ private void analyze(ASTNode ast) throws Exception { List> rootTasks = analyzer.getRootTasks(); Assert.assertEquals(1, rootTasks.size()); for (Task task : rootTasks) { - task.setDriverContext(new DriverContext(context)); + task.setDriverQueue(new DriverQueue(context)); task.setConf(conf); Assert.assertEquals(0, task.executeTask(null)); }