Return-Path: X-Original-To: apmail-hive-commits-archive@www.apache.org Delivered-To: apmail-hive-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id D26D719DD8 for ; Mon, 25 Apr 2016 20:32:13 +0000 (UTC) Received: (qmail 82610 invoked by uid 500); 25 Apr 2016 20:32:13 -0000 Delivered-To: apmail-hive-commits-archive@hive.apache.org Received: (qmail 82568 invoked by uid 500); 25 Apr 2016 20:32:13 -0000 Mailing-List: contact commits-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hive-dev@hive.apache.org Delivered-To: mailing list commits@hive.apache.org Received: (qmail 82553 invoked by uid 99); 25 Apr 2016 20:32:13 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 25 Apr 2016 20:32:13 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 837BCDFD5B; Mon, 25 Apr 2016 20:32:13 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: aihuaxu@apache.org To: commits@hive.apache.org Date: Mon, 25 Apr 2016 20:32:13 -0000 Message-Id: <33bc8989e9634236a283f071ed5296b5@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [1/2] hive git commit: HIVE-13424: Refactoring the code to pass a QueryState object rather than HiveConf object (Reviewed by Sergey Shelukhin) Repository: hive Updated Branches: refs/heads/master 86bdcbcd3 -> caa3ec761 http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java index 93b7a66..c13a404 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java @@ -190,7 +190,7 @@ public class ProcessAnalyzeTable implements NodeProcessor { // partial scan task DriverContext driverCxt = new DriverContext(); Task partialScanTask = TaskFactory.get(scanWork, parseContext.getConf()); - partialScanTask.initialize(parseContext.getConf(), null, driverCxt, + partialScanTask.initialize(parseContext.getQueryState(), null, driverCxt, tableScan.getCompilationOpContext()); partialScanTask.setWork(scanWork); statsWork.setSourceTask(partialScanTask); http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 9af7749..11fd2c7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -72,6 +72,7 @@ import org.apache.hadoop.hive.ql.CompilationOpContext; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.QueryProperties; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator; import org.apache.hadoop.hive.ql.exec.ArchiveUtils; import org.apache.hadoop.hive.ql.exec.ColumnInfo; @@ -340,8 +341,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { int nextNum; } - public SemanticAnalyzer(HiveConf conf) throws SemanticException { - super(conf); + public SemanticAnalyzer(QueryState queryState) throws SemanticException { + super(queryState); opToPartPruner = new HashMap(); opToPartList = new HashMap(); opToSamplePruner = new HashMap(); @@ -442,7 +443,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { public ParseContext getParseContext() { // Make sure the basic query properties are initialized copyInfoToQueryProperties(queryProperties); - return new ParseContext(conf, opToPartPruner, opToPartList, topOps, + return new ParseContext(queryState, opToPartPruner, opToPartList, topOps, new HashSet(joinContext.keySet()), new HashSet(smbMapJoinContext.keySet()), loadTableWork, loadFileWork, ctx, idToTableNameMap, destTableId, uCtx, @@ -1197,18 +1198,18 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { createTable.addChild(temporary); createTable.addChild(cte.cteNode); - SemanticAnalyzer analyzer = new SemanticAnalyzer(conf); + SemanticAnalyzer analyzer = new SemanticAnalyzer(queryState); analyzer.initCtx(ctx); analyzer.init(false); // should share cte contexts analyzer.aliasToCTEs.putAll(aliasToCTEs); - HiveOperation operation = SessionState.get().getHiveOperation(); + HiveOperation operation = queryState.getHiveOperation(); try { analyzer.analyzeInternal(createTable); } finally { - SessionState.get().setCommandType(operation); + queryState.setCommandType(operation); } Table table = analyzer.tableDesc.toTable(conf); @@ -6977,7 +6978,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { if (ltd != null && SessionState.get() != null) { SessionState.get().getLineageState() .mapDirToFop(ltd.getSourcePath(), (FileSinkOperator) output); - } else if ( SessionState.get().getCommandType().equals(HiveOperation.CREATETABLE_AS_SELECT.getOperationName())) { + } else if ( queryState.getCommandType().equals(HiveOperation.CREATETABLE_AS_SELECT.getOperationName())) { Path tlocation = null; String tName = Utilities.getDbTableName(tableDesc.getTableName())[1]; @@ -9340,7 +9341,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { limit.intValue(), extraMRStep); qb.getParseInfo().setOuterQueryLimit(limit.intValue()); } - if (!SessionState.get().getHiveOperation().equals(HiveOperation.CREATEVIEW)) { + if (!queryState.getHiveOperation().equals(HiveOperation.CREATEVIEW)) { curr = genFileSinkPlan(dest, qb, curr); } } @@ -10345,7 +10346,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { return true; } } else { - SessionState.get().setCommandType(HiveOperation.QUERY); + queryState.setCommandType(HiveOperation.QUERY); } return false; @@ -10543,14 +10544,14 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { return false; } } else { - SessionState.get().setCommandType(HiveOperation.QUERY); + queryState.setCommandType(HiveOperation.QUERY); } // 3. analyze create view command if (ast.getToken().getType() == HiveParser.TOK_CREATEVIEW || (ast.getToken().getType() == HiveParser.TOK_ALTERVIEW && ast.getChild(1).getType() == HiveParser.TOK_QUERY)) { child = analyzeCreateView(ast, qb); - SessionState.get().setCommandType(HiveOperation.CREATEVIEW); + queryState.setCommandType(HiveOperation.CREATEVIEW); if (child == null) { return false; } @@ -10579,7 +10580,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { throw new IllegalStateException(SemanticAnalyzerFactory.getOperation(ast.getToken().getType()) + " is not supported yet."); } - SessionState.get().setCommandType(SemanticAnalyzerFactory.getOperation(ast.getToken().getType())); + queryState.setCommandType(SemanticAnalyzerFactory.getOperation(ast.getToken().getType())); return false; } @@ -10677,7 +10678,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { // 4. Generate Parse Context for Optimizer & Physical compiler copyInfoToQueryProperties(queryProperties); - ParseContext pCtx = new ParseContext(conf, opToPartPruner, opToPartList, topOps, + ParseContext pCtx = new ParseContext(queryState, opToPartPruner, opToPartList, topOps, new HashSet(joinContext.keySet()), new HashSet(smbMapJoinContext.keySet()), loadTableWork, loadFileWork, ctx, idToTableNameMap, destTableId, uCtx, @@ -10757,7 +10758,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { // TEZ..) if (!ctx.getExplainLogical()) { TaskCompiler compiler = TaskCompilerFactory.getCompiler(conf, pCtx); - compiler.init(conf, console, db); + compiler.init(queryState, console, db); compiler.compile(pCtx, rootTasks, inputs, outputs); fetchTask = pCtx.getFetchTask(); } @@ -11520,7 +11521,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { crtTblDesc.validate(conf); // outputs is empty, which means this create table happens in the current // database. - SessionState.get().setCommandType(HiveOperation.CREATETABLE); + queryState.setCommandType(HiveOperation.CREATETABLE); rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), crtTblDesc), conf)); break; @@ -11539,7 +11540,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { storageFormat.getInputFormat(), storageFormat.getOutputFormat(), location, storageFormat.getSerde(), storageFormat.getSerdeProps(), tblProps, ifNotExists, likeTableName, isUserStorageFormat); - SessionState.get().setCommandType(HiveOperation.CREATETABLE); + queryState.setCommandType(HiveOperation.CREATETABLE); rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), crtTblLikeDesc), conf)); break; @@ -11613,7 +11614,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { tableDesc.setNullFormat(rowFormatParams.nullFormat); qb.setTableDesc(tableDesc); - SessionState.get().setCommandType(HiveOperation.CREATETABLE_AS_SELECT); + queryState.setCommandType(HiveOperation.CREATETABLE_AS_SELECT); return selectStmt; default: http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java index 07ca409..fb8a33c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java @@ -20,6 +20,8 @@ package org.apache.hadoop.hive.ql.parse; import org.antlr.runtime.tree.Tree; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.QueryState; +import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.HiveOperation; import org.apache.hadoop.hive.ql.session.SessionState; @@ -158,24 +160,24 @@ public final class SemanticAnalyzerFactory { HiveOperation.ALTERTABLE_UPDATEPARTSTATS}); } - public static BaseSemanticAnalyzer get(HiveConf conf, ASTNode tree) + public static BaseSemanticAnalyzer get(QueryState queryState, ASTNode tree) throws SemanticException { if (tree.getToken() == null) { throw new RuntimeException("Empty Syntax Tree"); } else { - setSessionCommandType(commandType.get(tree.getType())); - + HiveOperation opType = commandType.get(tree.getType()); + queryState.setCommandType(opType); switch (tree.getType()) { case HiveParser.TOK_EXPLAIN: - return new ExplainSemanticAnalyzer(conf); + return new ExplainSemanticAnalyzer(queryState); case HiveParser.TOK_EXPLAIN_SQ_REWRITE: - return new ExplainSQRewriteSemanticAnalyzer(conf); + return new ExplainSQRewriteSemanticAnalyzer(queryState); case HiveParser.TOK_LOAD: - return new LoadSemanticAnalyzer(conf); + return new LoadSemanticAnalyzer(queryState); case HiveParser.TOK_EXPORT: - return new ExportSemanticAnalyzer(conf); + return new ExportSemanticAnalyzer(queryState); case HiveParser.TOK_IMPORT: - return new ImportSemanticAnalyzer(conf); + return new ImportSemanticAnalyzer(queryState); case HiveParser.TOK_ALTERTABLE: { Tree child = tree.getChild(1); switch (child.getType()) { @@ -193,13 +195,13 @@ public final class SemanticAnalyzerFactory { case HiveParser.TOK_ALTERTABLE_DROPPROPERTIES: case HiveParser.TOK_ALTERTABLE_EXCHANGEPARTITION: case HiveParser.TOK_ALTERTABLE_SKEWED: - setSessionCommandType(commandType.get(child.getType())); - return new DDLSemanticAnalyzer(conf); + queryState.setCommandType(commandType.get(child.getType())); + return new DDLSemanticAnalyzer(queryState); } - HiveOperation commandType = + opType = tablePartitionCommandType.get(child.getType())[tree.getChildCount() > 2 ? 1 : 0]; - setSessionCommandType(commandType); - return new DDLSemanticAnalyzer(conf); + queryState.setCommandType(opType); + return new DDLSemanticAnalyzer(queryState); } case HiveParser.TOK_ALTERVIEW: { Tree child = tree.getChild(1); @@ -209,13 +211,14 @@ public final class SemanticAnalyzerFactory { case HiveParser.TOK_ALTERVIEW_ADDPARTS: case HiveParser.TOK_ALTERVIEW_DROPPARTS: case HiveParser.TOK_ALTERVIEW_RENAME: - setSessionCommandType(commandType.get(child.getType())); - return new DDLSemanticAnalyzer(conf); + opType = commandType.get(child.getType()); + queryState.setCommandType(opType); + return new DDLSemanticAnalyzer(queryState); } // TOK_ALTERVIEW_AS assert child.getType() == HiveParser.TOK_QUERY; - setSessionCommandType(HiveOperation.ALTERVIEW_AS); - return new SemanticAnalyzer(conf); + queryState.setCommandType(HiveOperation.ALTERVIEW_AS); + return new SemanticAnalyzer(queryState); } case HiveParser.TOK_CREATEDATABASE: case HiveParser.TOK_DROPDATABASE: @@ -265,23 +268,23 @@ public final class SemanticAnalyzerFactory { case HiveParser.TOK_TRUNCATETABLE: case HiveParser.TOK_SHOW_SET_ROLE: case HiveParser.TOK_CACHE_METADATA: - return new DDLSemanticAnalyzer(conf); + return new DDLSemanticAnalyzer(queryState); case HiveParser.TOK_CREATEFUNCTION: case HiveParser.TOK_DROPFUNCTION: case HiveParser.TOK_RELOADFUNCTION: - return new FunctionSemanticAnalyzer(conf); + return new FunctionSemanticAnalyzer(queryState); case HiveParser.TOK_ANALYZE: - return new ColumnStatsSemanticAnalyzer(conf); + return new ColumnStatsSemanticAnalyzer(queryState); case HiveParser.TOK_CREATEMACRO: case HiveParser.TOK_DROPMACRO: - return new MacroSemanticAnalyzer(conf); + return new MacroSemanticAnalyzer(queryState); case HiveParser.TOK_UPDATE_TABLE: case HiveParser.TOK_DELETE_FROM: - return new UpdateDeleteSemanticAnalyzer(conf); + return new UpdateDeleteSemanticAnalyzer(queryState); case HiveParser.TOK_START_TRANSACTION: case HiveParser.TOK_COMMIT: @@ -289,20 +292,14 @@ public final class SemanticAnalyzerFactory { case HiveParser.TOK_SET_AUTOCOMMIT: default: { SemanticAnalyzer semAnalyzer = HiveConf - .getBoolVar(conf, HiveConf.ConfVars.HIVE_CBO_ENABLED) ? new CalcitePlanner(conf) - : new SemanticAnalyzer(conf); + .getBoolVar(queryState.getConf(), HiveConf.ConfVars.HIVE_CBO_ENABLED) ? + new CalcitePlanner(queryState) : new SemanticAnalyzer(queryState); return semAnalyzer; } } } } - private static void setSessionCommandType(HiveOperation commandType) { - if (SessionState.get() != null) { - SessionState.get().setCommandType(commandType); - } - } - private SemanticAnalyzerFactory() { // prevent instantiation } http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java index 75ca5f9..7efc987 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hive.metastore.Warehouse; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.ColumnStatsTask; import org.apache.hadoop.hive.ql.exec.FetchTask; import org.apache.hadoop.hive.ql.exec.StatsTask; @@ -82,10 +83,12 @@ public abstract class TaskCompiler { // Assumes one instance of this + single-threaded compilation for each query. protected Hive db; protected LogHelper console; + protected QueryState queryState; protected HiveConf conf; - public void init(HiveConf conf, LogHelper console, Hive db) { - this.conf = conf; + public void init(QueryState queryState, LogHelper console, Hive db) { + this.queryState = queryState; + this.conf = queryState.getConf(); this.db = db; this.console = console; } @@ -447,7 +450,7 @@ public abstract class TaskCompiler { * Create a clone of the parse context */ public ParseContext getParseContext(ParseContext pCtx, List> rootTasks) { - ParseContext clone = new ParseContext(conf, + ParseContext clone = new ParseContext(queryState, pCtx.getOpToPartPruner(), pCtx.getOpToPartList(), pCtx.getTopOps(), pCtx.getJoinOps(), pCtx.getSmbMapJoinOps(), pCtx.getLoadTableWork(), pCtx.getLoadFileWork(), pCtx.getContext(), http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java index 8e67ce8..cf7a875 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java @@ -35,6 +35,7 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator; import org.apache.hadoop.hive.ql.exec.CommonMergeJoinOperator; import org.apache.hadoop.hive.ql.exec.ConditionalTask; @@ -101,8 +102,8 @@ public class TezCompiler extends TaskCompiler { } @Override - public void init(HiveConf conf, LogHelper console, Hive db) { - super.init(conf, console, db); + public void init(QueryState queryState, LogHelper console, Hive db) { + super.init(queryState, console, db); // Tez requires us to use RPC for the query plan HiveConf.setBoolVar(conf, ConfVars.HIVE_RPC_QUERY_PLAN, true); http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java index 5b4365c..b8771d2 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.hooks.Entity; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; @@ -52,8 +53,8 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer { boolean useSuper = false; - public UpdateDeleteSemanticAnalyzer(HiveConf conf) throws SemanticException { - super(conf); + public UpdateDeleteSemanticAnalyzer(QueryState queryState) throws SemanticException { + super(queryState); } @Override http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java index 80ccb28..52186b4 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java @@ -187,7 +187,7 @@ public class SparkProcessAnalyzeTable implements NodeProcessor { @SuppressWarnings("unchecked") Task partialScanTask = TaskFactory.get(scanWork, parseContext.getConf()); - partialScanTask.initialize(parseContext.getConf(), null, driverCxt, + partialScanTask.initialize(parseContext.getQueryState(), null, driverCxt, tableScan.getCompilationOpContext()); partialScanTask.setWork(scanWork); statsWork.setSourceTask(partialScanTask); http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java index 344dd34..672df63 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java @@ -122,7 +122,7 @@ public class SessionState { /** * current configuration. */ - private final HiveConf conf; + private final HiveConf sessionConf; /** * silent mode. @@ -135,7 +135,7 @@ public class SessionState { protected boolean isVerbose; /** - * Is the query served from HiveServer2 + * The flag to indicate if the session serves the queries from HiveServer2 or not. */ private boolean isHiveServerQuery = false; @@ -171,11 +171,6 @@ public class SessionState { */ protected File tmpErrOutputFile; - /** - * type of the command. - */ - private HiveOperation commandType; - private String lastCommand; private HiveAuthorizationProvider authorizer; @@ -287,7 +282,7 @@ public class SessionState { } public HiveConf getConf() { - return conf; + return sessionConf; } @@ -316,8 +311,8 @@ public class SessionState { } public boolean getIsSilent() { - if(conf != null) { - return conf.getBoolVar(HiveConf.ConfVars.HIVESESSIONSILENT); + if(sessionConf != null) { + return sessionConf.getBoolVar(HiveConf.ConfVars.HIVESESSIONSILENT); } else { return isSilent; } @@ -328,8 +323,8 @@ public class SessionState { } public void setIsSilent(boolean isSilent) { - if(conf != null) { - conf.setBoolVar(HiveConf.ConfVars.HIVESESSIONSILENT, isSilent); + if(sessionConf != null) { + sessionConf.setBoolVar(HiveConf.ConfVars.HIVESESSIONSILENT, isSilent); } this.isSilent = isSilent; } @@ -355,7 +350,7 @@ public class SessionState { } public SessionState(HiveConf conf, String userName) { - this.conf = conf; + this.sessionConf = conf; this.userName = userName; if (LOG.isDebugEnabled()) { LOG.debug("SessionState user: " + userName); @@ -375,23 +370,11 @@ public class SessionState { parentLoader = SessionState.class.getClassLoader(); // Make sure that each session has its own UDFClassloader. For details see {@link UDFClassLoader} final ClassLoader currentLoader = Utilities.createUDFClassLoader((URLClassLoader) parentLoader, new String[]{}); - this.conf.setClassLoader(currentLoader); + this.sessionConf.setClassLoader(currentLoader); resourceDownloader = new ResourceDownloader(conf, HiveConf.getVar(conf, ConfVars.DOWNLOADED_RESOURCES_DIR)); } - public void setCmd(String cmdString) { - conf.setQueryString(cmdString); - } - - public String getCmd() { - return (conf.getQueryString()); - } - - public String getQueryId() { - return (conf.getVar(HiveConf.ConfVars.HIVEQUERYID)); - } - public Map getHiveVariables() { if (hiveVariables == null) { hiveVariables = new HashMap(); @@ -404,7 +387,7 @@ public class SessionState { } public String getSessionId() { - return (conf.getVar(HiveConf.ConfVars.HIVESESSIONID)); + return (sessionConf.getVar(HiveConf.ConfVars.HIVESESSIONID)); } public void updateThreadName() { @@ -457,9 +440,9 @@ public class SessionState { public HadoopShims.HdfsEncryptionShim getHdfsEncryptionShim() throws HiveException { if (hdfsEncryptionShim == null) { try { - FileSystem fs = FileSystem.get(conf); + FileSystem fs = FileSystem.get(sessionConf); if ("hdfs".equals(fs.getUri().getScheme())) { - hdfsEncryptionShim = ShimLoader.getHadoopShims().createHdfsEncryptionShim(fs, conf); + hdfsEncryptionShim = ShimLoader.getHadoopShims().createHdfsEncryptionShim(fs, sessionConf); } else { LOG.info("Could not get hdfsEncryptionShim, it is only applicable to hdfs filesystem."); } @@ -557,9 +540,9 @@ public class SessionState { // Hive object instance should be created with a copy of the conf object. If the conf is // shared with SessionState, other parts of the code might update the config, but // Hive.get(HiveConf) would not recognize the case when it needs refreshing - Hive.get(new HiveConf(startSs.conf)).getMSC(); + Hive.get(new HiveConf(startSs.sessionConf)).getMSC(); UserGroupInformation sessionUGI = Utils.getUGI(); - FileSystem.get(startSs.conf); + FileSystem.get(startSs.sessionConf); // Create scratch dirs for this session startSs.createSessionDirs(sessionUGI.getShortUserName()); @@ -611,9 +594,9 @@ public class SessionState { } // Neither open nor opening. if (!isAsync) { - startSs.tezSessionState.open(startSs.conf); // should use conf on session start-up + startSs.tezSessionState.open(startSs.sessionConf); // should use conf on session start-up } else { - startSs.tezSessionState.beginOpen(startSs.conf, null, console); + startSs.tezSessionState.beginOpen(startSs.sessionConf, null, console); } } catch (Exception e) { throw new RuntimeException(e); @@ -717,7 +700,7 @@ public class SessionState { * @return * @throws IOException */ - private void createPath(HiveConf conf, Path path, String permission, boolean isLocal, + private static void createPath(HiveConf conf, Path path, String permission, boolean isLocal, boolean isCleanUp) throws IOException { FsPermission fsPermission = new FsPermission(permission); FileSystem fs; @@ -817,18 +800,18 @@ public class SessionState { } try { - authenticator = HiveUtils.getAuthenticator(conf, + authenticator = HiveUtils.getAuthenticator(sessionConf, HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER); authenticator.setSessionState(this); - String clsStr = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER); - authorizer = HiveUtils.getAuthorizeProviderManager(conf, + String clsStr = HiveConf.getVar(sessionConf, HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER); + authorizer = HiveUtils.getAuthorizeProviderManager(sessionConf, clsStr, authenticator, true); if (authorizer == null) { // if it was null, the new (V2) authorization plugin must be specified in // config - HiveAuthorizerFactory authorizerFactory = HiveUtils.getAuthorizerFactory(conf, + HiveAuthorizerFactory authorizerFactory = HiveUtils.getAuthorizerFactory(sessionConf, HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER); HiveAuthzSessionContext.Builder authzContextBuilder = new HiveAuthzSessionContext.Builder(); @@ -837,12 +820,12 @@ public class SessionState { authzContextBuilder.setSessionString(getSessionId()); authorizerV2 = authorizerFactory.createHiveAuthorizer(new HiveMetastoreClientFactoryImpl(), - conf, authenticator, authzContextBuilder.build()); + sessionConf, authenticator, authzContextBuilder.build()); setAuthorizerV2Config(); } // create the create table grants with new config - createTableGrants = CreateTableAutomaticGrant.create(conf); + createTableGrants = CreateTableAutomaticGrant.create(sessionConf); } catch (HiveException e) { LOG.error("Error setting up authorization: " + e.getMessage(), e); @@ -858,23 +841,23 @@ public class SessionState { private void setAuthorizerV2Config() throws HiveException { // avoid processing the same config multiple times, check marker - if (conf.get(CONFIG_AUTHZ_SETTINGS_APPLIED_MARKER, "").equals(Boolean.TRUE.toString())) { + if (sessionConf.get(CONFIG_AUTHZ_SETTINGS_APPLIED_MARKER, "").equals(Boolean.TRUE.toString())) { return; } - String metastoreHook = conf.get(ConfVars.METASTORE_FILTER_HOOK.name()); + String metastoreHook = sessionConf.get(ConfVars.METASTORE_FILTER_HOOK.name()); if (!ConfVars.METASTORE_FILTER_HOOK.getDefaultValue().equals(metastoreHook) && !AuthorizationMetaStoreFilterHook.class.getName().equals(metastoreHook)) { LOG.warn(ConfVars.METASTORE_FILTER_HOOK.name() + " will be ignored, since hive.security.authorization.manager" + " is set to instance of HiveAuthorizerFactory."); } - conf.setVar(ConfVars.METASTORE_FILTER_HOOK, + sessionConf.setVar(ConfVars.METASTORE_FILTER_HOOK, AuthorizationMetaStoreFilterHook.class.getName()); - authorizerV2.applyAuthorizationConfigPolicy(conf); + authorizerV2.applyAuthorizationConfigPolicy(sessionConf); // update config in Hive thread local as well and init the metastore client try { - Hive.get(conf).getMSC(); + Hive.get(sessionConf).getMSC(); } catch (Exception e) { // catch-all due to some exec time dependencies on session state // that would cause ClassNoFoundException otherwise @@ -882,7 +865,7 @@ public class SessionState { } // set a marker that this conf has been processed. - conf.set(CONFIG_AUTHZ_SETTINGS_APPLIED_MARKER, Boolean.TRUE.toString()); + sessionConf.set(CONFIG_AUTHZ_SETTINGS_APPLIED_MARKER, Boolean.TRUE.toString()); } public Object getActiveAuthorizer() { @@ -1124,7 +1107,7 @@ public class SessionState { public void reloadAuxJars() throws IOException { final Set reloadedAuxJars = new HashSet(); - final String renewableJarPath = conf.getVar(ConfVars.HIVERELOADABLEJARS); + final String renewableJarPath = sessionConf.getVar(ConfVars.HIVERELOADABLEJARS); // do nothing if this property is not specified or empty if (renewableJarPath == null || renewableJarPath.isEmpty()) { return; @@ -1154,7 +1137,7 @@ public class SessionState { currentCLoader = (URLClassLoader) Utilities.addToClassPath(currentCLoader, reloadedAuxJars.toArray(new String[0])); - conf.setClassLoader(currentCLoader); + sessionConf.setClassLoader(currentCLoader); Thread.currentThread().setContextClassLoader(currentCLoader); } preReloadableAuxJars.clear(); @@ -1394,21 +1377,6 @@ public class SessionState { } } - public String getCommandType() { - if (commandType == null) { - return null; - } - return commandType.getOperationName(); - } - - public HiveOperation getHiveOperation() { - return commandType; - } - - public void setCommandType(HiveOperation commandType) { - this.commandType = commandType; - } - public HiveAuthorizationProvider getAuthorizer() { setupAuth(); return authorizer; @@ -1499,7 +1467,7 @@ public class SessionState { public void close() throws IOException { registry.clear(); if (txnMgr != null) txnMgr.closeTxnManager(); - JavaUtils.closeClassLoadersTo(conf.getClassLoader(), parentLoader); + JavaUtils.closeClassLoadersTo(sessionConf.getClassLoader(), parentLoader); File resourceDir = new File(getConf().getVar(HiveConf.ConfVars.DOWNLOADED_RESOURCES_DIR)); LOG.debug("Removing resource dir " + resourceDir); @@ -1526,7 +1494,7 @@ public class SessionState { try { closeSparkSession(); registry.closeCUDFLoaders(); - dropSessionPaths(conf); + dropSessionPaths(sessionConf); unCacheDataNucleusClassLoaders(); } finally { // removes the threadlocal variables, closes underlying HMS connection @@ -1536,10 +1504,10 @@ public class SessionState { private void unCacheDataNucleusClassLoaders() { try { - Hive threadLocalHive = Hive.get(conf); + Hive threadLocalHive = Hive.get(sessionConf); if ((threadLocalHive != null) && (threadLocalHive.getMSC() != null) && (threadLocalHive.getMSC().isLocalMetaStore())) { - if (conf.getVar(ConfVars.METASTORE_RAW_STORE_IMPL).equals(ObjectStore.class.getName())) { + if (sessionConf.getVar(ConfVars.METASTORE_RAW_STORE_IMPL).equals(ObjectStore.class.getName())) { ObjectStore.unCacheDataNucleusClassLoaders(); } } @@ -1667,9 +1635,9 @@ public class SessionState { queryCurrentTimestamp = new Timestamp(System.currentTimeMillis()); // Provide a facility to set current timestamp during tests - if (conf.getBoolVar(ConfVars.HIVE_IN_TEST)) { + if (sessionConf.getBoolVar(ConfVars.HIVE_IN_TEST)) { String overrideTimestampString = - HiveConf.getVar(conf, HiveConf.ConfVars.HIVETESTCURRENTTIMESTAMP, (String)null); + HiveConf.getVar(sessionConf, HiveConf.ConfVars.HIVETESTCURRENTTIMESTAMP, (String)null); if (overrideTimestampString != null && overrideTimestampString.length() > 0) { queryCurrentTimestamp = Timestamp.valueOf(overrideTimestampString); } http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java ---------------------------------------------------------------------- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java index eaeb66b..667d5c2 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.MetaStoreUtils; import org.apache.hadoop.hive.ql.CompilationOpContext; import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.WindowsPathUtil; import org.apache.hadoop.hive.ql.exec.mr.ExecDriver; import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; @@ -71,6 +72,7 @@ import org.apache.hadoop.util.Shell; */ public class TestExecDriver extends TestCase { + static QueryState queryState; static HiveConf conf; private static final String tmpdir; @@ -82,7 +84,8 @@ public class TestExecDriver extends TestCase { static { try { - conf = new HiveConf(ExecDriver.class); + queryState = new QueryState(new HiveConf(ExecDriver.class)); + conf = queryState.getConf(); conf.setBoolVar(HiveConf.ConfVars.SUBMITVIACHILD, true); conf.setBoolVar(HiveConf.ConfVars.SUBMITLOCALTASKVIACHILD, true); @@ -480,7 +483,7 @@ public class TestExecDriver extends TestCase { MapRedTask mrtask = new MapRedTask(); DriverContext dctx = new DriverContext (); mrtask.setWork(mr); - mrtask.initialize(conf, null, dctx, null); + mrtask.initialize(queryState, null, dctx, null); int exitVal = mrtask.execute(dctx); if (exitVal != 0) { http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java ---------------------------------------------------------------------- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java index 9d7166c..c659806 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java @@ -25,6 +25,7 @@ import junit.framework.Assert; 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.QueryState; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.session.SessionState; @@ -36,16 +37,18 @@ public class TestMacroSemanticAnalyzer { private ParseDriver parseDriver; private MacroSemanticAnalyzer analyzer; + private QueryState queryState; private HiveConf conf; private Context context; @Before public void setup() throws Exception { - conf = new HiveConf(); + queryState = new QueryState(null); + conf = queryState.getConf(); SessionState.start(conf); context = new Context(conf); parseDriver = new ParseDriver(); - analyzer = new MacroSemanticAnalyzer(conf); + analyzer = new MacroSemanticAnalyzer(queryState); } private ASTNode parse(String command) throws Exception { http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java ---------------------------------------------------------------------- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java index ccdf272..8fe4d02 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java @@ -18,9 +18,11 @@ package org.apache.hadoop.hive.ql.parse; import junit.framework.Assert; + import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat; @@ -42,11 +44,13 @@ import java.util.Map; * Tests for parsing and semantic analysis of ALTER TABLE ... compact. */ public class TestQBCompact { + static QueryState queryState; static HiveConf conf; @BeforeClass public static void init() throws Exception { - conf = new HiveConf(); + queryState = new QueryState(null); + conf = queryState.getConf(); SessionState.start(conf); // Create a table so we can work against it @@ -65,7 +69,7 @@ public class TestQBCompact { private AlterTableSimpleDesc parseAndAnalyze(String query) throws Exception { ParseDriver hd = new ParseDriver(); ASTNode head = (ASTNode)hd.parse(query).getChild(0); - BaseSemanticAnalyzer a = SemanticAnalyzerFactory.get(conf, head); + BaseSemanticAnalyzer a = SemanticAnalyzerFactory.get(queryState, head); a.analyze(head, new Context(conf)); List> roots = a.getRootTasks(); Assert.assertEquals(1, roots.size()); http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBJoinTreeApplyPredicate.java ---------------------------------------------------------------------- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBJoinTreeApplyPredicate.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBJoinTreeApplyPredicate.java index 70d86c1..e607f10 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBJoinTreeApplyPredicate.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBJoinTreeApplyPredicate.java @@ -23,6 +23,7 @@ import java.util.List; import junit.framework.Assert; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.session.SessionState; import org.junit.Before; import org.junit.BeforeClass; @@ -30,19 +31,21 @@ import org.junit.Test; public class TestQBJoinTreeApplyPredicate { + static QueryState queryState; static HiveConf conf; SemanticAnalyzer sA; @BeforeClass public static void initialize() { - conf = new HiveConf(SemanticAnalyzer.class); + queryState = new QueryState(new HiveConf(SemanticAnalyzer.class)); + conf = queryState.getConf(); SessionState.start(conf); } @Before public void setup() throws SemanticException { - sA = new CalcitePlanner(conf); + sA = new CalcitePlanner(queryState); } static ASTNode constructIdentifier(String nm) { http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBSubQuery.java ---------------------------------------------------------------------- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBSubQuery.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBSubQuery.java index f6f0abb..f9db2c8 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBSubQuery.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBSubQuery.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.List; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.session.SessionState; import org.junit.Assert; import org.junit.Before; @@ -29,6 +30,7 @@ import org.junit.BeforeClass; import org.junit.Test; public class TestQBSubQuery { + static QueryState queryState; static HiveConf conf; private static String IN_QUERY = " select * " + @@ -46,14 +48,15 @@ public class TestQBSubQuery { @BeforeClass public static void initialize() { - conf = new HiveConf(SemanticAnalyzer.class); + queryState = new QueryState(new HiveConf(SemanticAnalyzer.class)); + conf = queryState.getConf(); SessionState.start(conf); } @Before public void setup() throws SemanticException { pd = new ParseDriver(); - sA = new CalcitePlanner(conf); + sA = new CalcitePlanner(queryState); } ASTNode parse(String query) throws ParseException { http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSemanticAnalyzerFactory.java ---------------------------------------------------------------------- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSemanticAnalyzerFactory.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSemanticAnalyzerFactory.java index 4a52efb..5849950 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSemanticAnalyzerFactory.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestSemanticAnalyzerFactory.java @@ -21,27 +21,30 @@ import junit.framework.Assert; import org.antlr.runtime.CommonToken; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.QueryState; import org.junit.Before; import org.junit.Test; public class TestSemanticAnalyzerFactory { + private QueryState queryState; private HiveConf conf; @Before public void setup() throws Exception { - conf = new HiveConf(); + queryState = new QueryState(null); + conf = queryState.getConf(); } @Test public void testCreate() throws Exception { BaseSemanticAnalyzer analyzer = SemanticAnalyzerFactory. - get(conf, new ASTNode(new CommonToken(HiveParser.TOK_CREATEMACRO))); + get(queryState, new ASTNode(new CommonToken(HiveParser.TOK_CREATEMACRO))); Assert.assertTrue(analyzer.getClass().getSimpleName(), analyzer instanceof MacroSemanticAnalyzer); } @Test public void testDrop() throws Exception { BaseSemanticAnalyzer analyzer = SemanticAnalyzerFactory. - get(conf, new ASTNode(new CommonToken(HiveParser.TOK_DROPMACRO))); + get(queryState, new ASTNode(new CommonToken(HiveParser.TOK_DROPMACRO))); Assert.assertTrue(analyzer.getClass().getSimpleName(), analyzer instanceof MacroSemanticAnalyzer); } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java ---------------------------------------------------------------------- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java index 8614d40..78f5ea9 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.ExplainTask; import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat; @@ -47,6 +48,7 @@ public class TestUpdateDeleteSemanticAnalyzer { static final private Logger LOG = LoggerFactory.getLogger(TestUpdateDeleteSemanticAnalyzer.class.getName()); + private QueryState queryState; private HiveConf conf; private Hive db; @@ -221,7 +223,8 @@ public class TestUpdateDeleteSemanticAnalyzer { @Before public void setup() { - conf = new HiveConf(); + queryState = new QueryState(null); + conf = queryState.getConf(); conf.setVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict"); conf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict"); conf.setVar(HiveConf.ConfVars.HIVE_TXN_MANAGER, "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager"); @@ -256,7 +259,7 @@ public class TestUpdateDeleteSemanticAnalyzer { ASTNode tree = pd.parse(query, ctx); tree = ParseUtils.findRootNonNullToken(tree); - BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(conf, tree); + BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(queryState, tree); SessionState.get().initTxnMgr(conf); db = sem.getDb(); @@ -295,7 +298,7 @@ public class TestUpdateDeleteSemanticAnalyzer { sem.getFetchTask(), sem, true, false, false, false, false, false, null); ExplainTask task = new ExplainTask(); task.setWork(work); - task.initialize(conf, plan, null, null); + task.initialize(queryState, plan, null, null); task.execute(null); FSDataInputStream in = fs.open(tmp); StringBuilder builder = new StringBuilder(); http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/AuthorizationTestUtil.java ---------------------------------------------------------------------- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/AuthorizationTestUtil.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/AuthorizationTestUtil.java index 2e85a2a..e8e29ee 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/AuthorizationTestUtil.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/AuthorizationTestUtil.java @@ -22,8 +22,8 @@ import java.util.List; import junit.framework.Assert; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.parse.ASTNode; @@ -46,10 +46,10 @@ public class AuthorizationTestUtil { * @return * @throws Exception */ - public static DDLWork analyze(ASTNode ast, HiveConf conf, Hive db) throws Exception { - DDLSemanticAnalyzer analyzer = new DDLSemanticAnalyzer(conf, db); - SessionState.start(conf); - analyzer.analyze(ast, new Context(conf)); + public static DDLWork analyze(ASTNode ast, QueryState queryState, Hive db) throws Exception { + DDLSemanticAnalyzer analyzer = new DDLSemanticAnalyzer(queryState, db); + SessionState.start(queryState.getConf()); + analyzer.analyze(ast, new Context(queryState.getConf())); List> rootTasks = analyzer.getRootTasks(); return (DDLWork) inList(rootTasks).ofSize(1).get(0).getWork(); } @@ -62,8 +62,8 @@ public class AuthorizationTestUtil { * @return * @throws Exception */ - public static DDLWork analyze(String command, HiveConf conf, Hive db) throws Exception { - return analyze(parse(command), conf, db); + public static DDLWork analyze(String command, QueryState queryState, Hive db) throws Exception { + return analyze(parse(command), queryState, db); } private static ASTNode parse(String command) throws Exception { http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/PrivilegesTestBase.java ---------------------------------------------------------------------- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/PrivilegesTestBase.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/PrivilegesTestBase.java index e3b82f3..b324dfd 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/PrivilegesTestBase.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/PrivilegesTestBase.java @@ -21,6 +21,7 @@ import junit.framework.Assert; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.PrincipalType; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.plan.DDLWork; import org.apache.hadoop.hive.ql.plan.GrantDesc; @@ -34,9 +35,9 @@ public class PrivilegesTestBase { protected static final String TABLE_QNAME = DB + "." + TABLE; protected static final String USER = "user1"; - public static void grantUserTable(String privStr, PrivilegeType privType, HiveConf conf, Hive db) + public static void grantUserTable(String privStr, PrivilegeType privType, QueryState queryState, Hive db) throws Exception { - DDLWork work = AuthorizationTestUtil.analyze("GRANT " + privStr + " ON TABLE " + TABLE + " TO USER " + USER, conf, db); + DDLWork work = AuthorizationTestUtil.analyze("GRANT " + privStr + " ON TABLE " + TABLE + " TO USER " + USER, queryState, db); GrantDesc grantDesc = work.getGrantDesc(); Assert.assertNotNull("Grant should not be null", grantDesc); http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestHiveAuthorizationTaskFactory.java ---------------------------------------------------------------------- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestHiveAuthorizationTaskFactory.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestHiveAuthorizationTaskFactory.java index 038e5fd..bfacbd0 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestHiveAuthorizationTaskFactory.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestHiveAuthorizationTaskFactory.java @@ -19,12 +19,13 @@ package org.apache.hadoop.hive.ql.parse.authorization; import java.util.HashMap; -import junit.framework.Assert; +import org.junit.Assert; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.api.PrincipalType; import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Table; @@ -89,7 +90,7 @@ public class TestHiveAuthorizationTaskFactory { private ParseDriver parseDriver; private DDLSemanticAnalyzer analyzer; - private HiveConf conf; + private QueryState queryState; private Context context; private String currentUser; private Hive db; @@ -98,7 +99,8 @@ public class TestHiveAuthorizationTaskFactory { @Before public void setup() throws Exception { - conf = new HiveConf(); + queryState = new QueryState(null); + HiveConf conf = queryState.getConf(); conf.setVar(ConfVars.HIVE_AUTHORIZATION_TASK_FACTORY, TestHiveAuthorizationTaskFactory.DummyHiveAuthorizationTaskFactoryImpl.class.getName()); db = Mockito.mock(Hive.class); @@ -107,7 +109,7 @@ public class TestHiveAuthorizationTaskFactory { SessionState.start(conf); context = new Context(conf); parseDriver = new ParseDriver(); - analyzer = new DDLSemanticAnalyzer(conf, db); + analyzer = new DDLSemanticAnalyzer(queryState, db); Mockito.when(db.getTable(DB, TABLE, false)).thenReturn(table); Mockito.when(db.getTable(TABLE_QNAME, false)).thenReturn(table); Mockito.when(db.getPartition(table, new HashMap(), false)) @@ -478,7 +480,7 @@ public class TestHiveAuthorizationTaskFactory { } private DDLWork analyze(String command) throws Exception { - return AuthorizationTestUtil.analyze(command, conf, db); + return AuthorizationTestUtil.analyze(command, queryState, db); } http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestPrivilegesV1.java ---------------------------------------------------------------------- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestPrivilegesV1.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestPrivilegesV1.java index c614630..41584af 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestPrivilegesV1.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestPrivilegesV1.java @@ -22,6 +22,7 @@ import java.util.HashMap; import junit.framework.Assert; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Table; @@ -33,18 +34,18 @@ import org.mockito.Mockito; public class TestPrivilegesV1 extends PrivilegesTestBase{ - private HiveConf conf; + private QueryState queryState; private Hive db; private Table table; private Partition partition; @Before public void setup() throws Exception { - conf = new HiveConf(); + queryState = new QueryState(null); db = Mockito.mock(Hive.class); table = new Table(DB, TABLE); partition = new Partition(table); - SessionState.start(conf); + SessionState.start(queryState.getConf()); Mockito.when(db.getTable(DB, TABLE, false)).thenReturn(table); Mockito.when(db.getTable(TABLE_QNAME, false)).thenReturn(table); Mockito.when(db.getPartition(table, new HashMap(), false)) @@ -81,6 +82,6 @@ public class TestPrivilegesV1 extends PrivilegesTestBase{ } private void grantUserTable(String privName, PrivilegeType privType) throws Exception { - grantUserTable(privName, privType, conf, db); + grantUserTable(privName, privType, queryState, db); } } http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestPrivilegesV2.java ---------------------------------------------------------------------- diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestPrivilegesV2.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestPrivilegesV2.java index 7b28375..c552ba7 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestPrivilegesV2.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestPrivilegesV2.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.parse.authorization; import java.util.HashMap; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Table; @@ -32,15 +33,16 @@ import org.mockito.Mockito; public class TestPrivilegesV2 extends PrivilegesTestBase{ - private HiveConf conf; + private QueryState queryState; private Hive db; private Table table; private Partition partition; @Before public void setup() throws Exception { - conf = new HiveConf(); + queryState = new QueryState(null); //set authorization mode to V2 + HiveConf conf = queryState.getConf(); conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, SQLStdHiveAuthorizerFactory.class.getName()); db = Mockito.mock(Hive.class); @@ -67,7 +69,7 @@ public class TestPrivilegesV2 extends PrivilegesTestBase{ } private void grantUserTable(String privName, PrivilegeType privType) throws Exception { - grantUserTable(privName, privType, conf, db); + grantUserTable(privName, privType, queryState, db); } } http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/service/src/java/org/apache/hive/service/cli/operation/Operation.java ---------------------------------------------------------------------- diff --git a/service/src/java/org/apache/hive/service/cli/operation/Operation.java b/service/src/java/org/apache/hive/service/cli/operation/Operation.java index 4f3e9c2..63b1a48 100644 --- a/service/src/java/org/apache/hive/service/cli/operation/Operation.java +++ b/service/src/java/org/apache/hive/service/cli/operation/Operation.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.common.metrics.common.MetricsConstant; import org.apache.hadoop.hive.common.metrics.common.MetricsFactory; import org.apache.hadoop.hive.common.metrics.common.MetricsScope; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; import org.apache.hadoop.hive.ql.session.OperationLog; import org.apache.hadoop.hive.ql.session.SessionState; @@ -62,7 +62,6 @@ public abstract class Operation { private volatile OperationState state = OperationState.INITIALIZED; private volatile MetricsScope currentStateScope; private final OperationHandle opHandle; - private HiveConf configuration; public static final FetchOrientation DEFAULT_FETCH_ORIENTATION = FetchOrientation.FETCH_NEXT; public static final Logger LOG = LoggerFactory.getLogger(Operation.class.getName()); public static final long DEFAULT_FETCH_MAX_ROWS = 100; @@ -81,13 +80,13 @@ public abstract class Operation { protected long operationStart; protected long operationComplete; + protected final QueryState queryState; + protected static final EnumSet DEFAULT_FETCH_ORIENTATION_SET = EnumSet.of(FetchOrientation.FETCH_NEXT,FetchOrientation.FETCH_FIRST); protected Operation(HiveSession parentSession, OperationType opType, boolean runInBackground) { this(parentSession, null, opType, runInBackground); - // Generate a queryId for the operation if no queryId is provided - confOverlay.put(HiveConf.ConfVars.HIVEQUERYID.varname, QueryPlan.makeQueryId()); } protected Operation(HiveSession parentSession, Map confOverlay, OperationType opType, boolean runInBackground) { @@ -102,6 +101,11 @@ public abstract class Operation { operationTimeout = HiveConf.getTimeVar(parentSession.getHiveConf(), HiveConf.ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT, TimeUnit.MILLISECONDS); setMetrics(state); + queryState = new QueryState(parentSession.getHiveConf(), confOverlay, runAsync); + } + + public QueryState getQueryState() { + return queryState; } public Future getBackgroundHandle() { @@ -116,13 +120,6 @@ public abstract class Operation { return runAsync; } - public void setConfiguration(HiveConf configuration) { - this.configuration = new HiveConf(configuration); - } - - public HiveConf getConfiguration() { - return new HiveConf(configuration); - } public HiveSession getParentSession() { return parentSession; http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java ---------------------------------------------------------------------- diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java index b921e6e..01dd48c 100644 --- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java +++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hive.metastore.api.Schema; import org.apache.hadoop.hive.ql.CommandNeedRetryException; import org.apache.hadoop.hive.ql.Driver; import org.apache.hadoop.hive.ql.QueryDisplay; +import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.ExplainTask; import org.apache.hadoop.hive.ql.exec.FetchTask; import org.apache.hadoop.hive.ql.exec.Task; @@ -125,10 +126,10 @@ public class SQLOperation extends ExecuteStatementOperation { * @param sqlOperationConf * @throws HiveSQLException */ - public void prepare(HiveConf sqlOperationConf) throws HiveSQLException { + public void prepare(QueryState queryState) throws HiveSQLException { setState(OperationState.RUNNING); try { - driver = new Driver(sqlOperationConf, getParentSession().getUserName()); + driver = new Driver(queryState, getParentSession().getUserName()); sqlOpDisplay.setQueryDisplay(driver.getQueryDisplay()); // set the operation handle information in Driver, so that thrift API users @@ -181,7 +182,7 @@ public class SQLOperation extends ExecuteStatementOperation { } } - private void runQuery(HiveConf sqlOperationConf) throws HiveSQLException { + private void runQuery() throws HiveSQLException { try { // In Hive server mode, we are not able to retry in the FetchTask // case, when calling fetch queries since execute() has returned. @@ -213,11 +214,10 @@ public class SQLOperation extends ExecuteStatementOperation { @Override public void runInternal() throws HiveSQLException { setState(OperationState.PENDING); - final HiveConf opConfig = getConfigForOperation(); - prepare(opConfig); + prepare(queryState); if (!shouldRunAsync()) { - runQuery(opConfig); + runQuery(); } else { // We'll pass ThreadLocals in the background thread from the foreground (handler) thread final SessionState parentSessionState = SessionState.get(); @@ -226,7 +226,7 @@ public class SQLOperation extends ExecuteStatementOperation { final Hive parentHive = parentSession.getSessionHive(); // Current UGI will get used by metastore when metsatore is in embedded mode // So this needs to get passed to the new background thread - final UserGroupInformation currentUGI = getCurrentUGI(opConfig); + final UserGroupInformation currentUGI = getCurrentUGI(); // Runnable impl to call runInternal asynchronously, // from a different thread Runnable backgroundOperation = new Runnable() { @@ -241,7 +241,7 @@ public class SQLOperation extends ExecuteStatementOperation { registerCurrentOperationLog(); registerLoggingContext(); try { - runQuery(opConfig); + runQuery(); } catch (HiveSQLException e) { setOperationException(e); LOG.error("Error running hive query: ", e); @@ -292,7 +292,7 @@ public class SQLOperation extends ExecuteStatementOperation { * @return UserGroupInformation * @throws HiveSQLException */ - private UserGroupInformation getCurrentUGI(HiveConf opConfig) throws HiveSQLException { + private UserGroupInformation getCurrentUGI() throws HiveSQLException { try { return Utils.getUGI(); } catch (Exception e) { @@ -361,14 +361,13 @@ public class SQLOperation extends ExecuteStatementOperation { public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - HiveConf hiveConf = getConfigForOperation(); validateDefaultFetchOrientation(orientation); assertState(new ArrayList(Arrays.asList(OperationState.FINISHED))); FetchTask fetchTask = driver.getFetchTask(); boolean isBlobBased = false; - if (fetchTask != null && fetchTask.getWork().isHiveServerQuery() && HiveConf.getBoolVar(hiveConf, + if (fetchTask != null && fetchTask.getWork().isHiveServerQuery() && HiveConf.getBoolVar(queryState.getConf(), HiveConf.ConfVars.HIVE_SERVER2_THRIFT_RESULTSET_SERIALIZE_IN_TASKS) && (fetchTask.getTblDesc().getSerdeClassName().equalsIgnoreCase(ThriftJDBCBinarySerDe.class .getName()))) { @@ -519,34 +518,6 @@ public class SQLOperation extends ExecuteStatementOperation { } /** - * If there are query specific settings to overlay, then create a copy of config - * There are two cases we need to clone the session config that's being passed to hive driver - * 1. Async query - - * If the client changes a config setting, that shouldn't reflect in the execution already underway - * 2. confOverlay - - * The query specific settings should only be applied to the query config and not session - * @return new configuration - * @throws HiveSQLException - */ - public HiveConf getConfigForOperation() throws HiveSQLException { - HiveConf sqlOperationConf = getParentSession().getHiveConf(); - if (!confOverlay.isEmpty() || shouldRunAsync()) { - // clone the partent session config for this query - sqlOperationConf = new HiveConf(sqlOperationConf); - - // apply overlay query specific settings, if any - for (Map.Entry confEntry : confOverlay.entrySet()) { - try { - sqlOperationConf.verifyAndSet(confEntry.getKey(), confEntry.getValue()); - } catch (IllegalArgumentException e) { - throw new HiveSQLException("Error applying statement specific settings", e); - } - } - } - return sqlOperationConf; - } - - /** * Get summary information of this SQLOperation for display in WebUI. */ public SQLOperationDisplay getSQLOperationDisplay() { http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/service/src/java/org/apache/hive/service/cli/operation/SQLOperationDisplay.java ---------------------------------------------------------------------- diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperationDisplay.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperationDisplay.java index fe93426..c521706 100644 --- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperationDisplay.java +++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperationDisplay.java @@ -41,7 +41,7 @@ public class SQLOperationDisplay { public SQLOperationDisplay(SQLOperation sqlOperation) throws HiveSQLException { this.state = sqlOperation.getState(); this.userName = sqlOperation.getParentSession().getUserName(); - this.executionEngine = sqlOperation.getConfigForOperation().getVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE); + this.executionEngine = sqlOperation.getQueryState().getConf().getVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE); this.beginTime = System.currentTimeMillis(); this.operationId = sqlOperation.getHandle().getHandleIdentifier().toString(); } http://git-wip-us.apache.org/repos/asf/hive/blob/caa3ec76/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java ---------------------------------------------------------------------- diff --git a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java index c24f162..0cfec7a 100644 --- a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java +++ b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java @@ -87,7 +87,7 @@ public class HiveSessionImpl implements HiveSession { private final SessionHandle sessionHandle; private String username; private final String password; - private final HiveConf hiveConf; + private final HiveConf sessionConf; private final long creationTime; // TODO: some SessionState internals are not thread safe. The compile-time internals are synced // via session-scope or global compile lock. The run-time internals work by magic! @@ -121,24 +121,24 @@ public class HiveSessionImpl implements HiveSession { this.password = password; creationTime = System.currentTimeMillis(); this.sessionHandle = sessionHandle != null ? sessionHandle : new SessionHandle(protocol); - this.hiveConf = new HiveConf(serverhiveConf); + this.sessionConf = new HiveConf(serverhiveConf); this.ipAddress = ipAddress; try { // In non-impersonation mode, map scheduler queue to current user // if fair scheduler is configured. - if (! hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && - hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_MAP_FAIR_SCHEDULER_QUEUE)) { - ShimLoader.getHadoopShims().refreshDefaultQueue(hiveConf, username); + if (! sessionConf.getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && + sessionConf.getBoolVar(ConfVars.HIVE_SERVER2_MAP_FAIR_SCHEDULER_QUEUE)) { + ShimLoader.getHadoopShims().refreshDefaultQueue(sessionConf, username); } } catch (IOException e) { LOG.warn("Error setting scheduler queue: " + e, e); } // Set an explicit session name to control the download directory name - hiveConf.set(ConfVars.HIVESESSIONID.varname, + sessionConf.set(ConfVars.HIVESESSIONID.varname, this.sessionHandle.getHandleIdentifier().toString()); // Use thrift transportable formatter - hiveConf.set(SerDeUtils.LIST_SINK_OUTPUT_FORMATTER, ThriftFormatter.class.getName()); - hiveConf.setInt(SerDeUtils.LIST_SINK_OUTPUT_PROTOCOL, protocol.getValue()); + sessionConf.set(SerDeUtils.LIST_SINK_OUTPUT_FORMATTER, ThriftFormatter.class.getName()); + sessionConf.setInt(SerDeUtils.LIST_SINK_OUTPUT_PROTOCOL, protocol.getValue()); } public HiveSessionImpl(TProtocolVersion protocol, String username, String password, @@ -158,7 +158,7 @@ public class HiveSessionImpl implements HiveSession { * That's why it is important to create SessionState here rather than in the constructor. */ public void open(Map sessionConfMap) throws HiveSQLException { - sessionState = new SessionState(hiveConf, username); + sessionState = new SessionState(sessionConf, username); sessionState.setUserIpAddress(ipAddress); sessionState.setIsHiveServerQuery(true); sessionState.setForwardedAddresses(SessionManager.getForwardedAddresses()); @@ -224,7 +224,7 @@ public class HiveSessionImpl implements HiveSession { IHiveFileProcessor processor = new GlobalHivercFileProcessor(); try { - String hiverc = hiveConf.getVar(ConfVars.HIVE_SERVER2_GLOBAL_INIT_FILE_LOCATION); + String hiverc = sessionConf.getVar(ConfVars.HIVE_SERVER2_GLOBAL_INIT_FILE_LOCATION); if (hiverc != null) { File hivercFile = new File(hiverc); if (hivercFile.isDirectory()) { @@ -258,7 +258,7 @@ public class HiveSessionImpl implements HiveSession { } else if (key.startsWith("use:")) { SessionState.get().setCurrentDatabase(entry.getValue()); } else { - hiveConf.verifyAndSet(key, entry.getValue()); + sessionConf.verifyAndSet(key, entry.getValue()); } } } @@ -380,8 +380,8 @@ public class HiveSessionImpl implements HiveSession { @Override public HiveConf getHiveConf() { - hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHOUTPUTSERDE, FETCH_WORK_SERDE_CLASS); - return hiveConf; + sessionConf.setVar(HiveConf.ConfVars.HIVEFETCHOUTPUTSERDE, FETCH_WORK_SERDE_CLASS); + return sessionConf; } @Override @@ -442,16 +442,6 @@ public class HiveSessionImpl implements HiveSession { throws HiveSQLException { acquire(true); - // Make a copy of confOverlay - if (confOverlay == null) { - confOverlay = new HashMap(); - } else { - Map conf = new HashMap(); - conf.putAll(confOverlay); - confOverlay = conf; - } - confOverlay.put(HiveConf.ConfVars.HIVEQUERYID.varname, QueryPlan.makeQueryId()); - OperationManager operationManager = getOperationManager(); ExecuteStatementOperation operation = operationManager .newExecuteStatementOperation(getSession(), statement, confOverlay, runAsync); @@ -578,7 +568,7 @@ public class HiveSessionImpl implements HiveSession { public OperationHandle getColumns(String catalogName, String schemaName, String tableName, String columnName) throws HiveSQLException { acquire(true); - String addedJars = Utilities.getResourceFiles(hiveConf, SessionState.ResourceType.JAR); + String addedJars = Utilities.getResourceFiles(sessionConf, SessionState.ResourceType.JAR); if (StringUtils.isNotBlank(addedJars)) { IMetaStoreClient metastoreClient = getSession().getMetaStoreClient(); metastoreClient.setHiveAddedJars(addedJars); @@ -788,7 +778,7 @@ public class HiveSessionImpl implements HiveSession { if (fetchType == FetchType.QUERY_OUTPUT) { return operationManager.getOperationNextRowSet(opHandle, orientation, maxRows); } - return operationManager.getOperationLogRowSet(opHandle, orientation, maxRows, hiveConf); + return operationManager.getOperationLogRowSet(opHandle, orientation, maxRows, sessionConf); } finally { release(true); }