diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 4ecafada27..c673e815d4 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -2148,8 +2148,6 @@ public static enum ConfVars { new TimeValidator(TimeUnit.SECONDS), "Number of seconds a request will wait to acquire the compile lock before giving up. " + "Setting it to 0s disables the timeout."), - HIVE_SERVER2_PARALLEL_OPS_IN_SESSION("hive.server2.parallel.ops.in.session", true, - "Whether to allow several parallel operations (such as SQL statements) in one session."), // HiveServer2 WebUI HIVE_SERVER2_WEBUI_BIND_HOST("hive.server2.webui.host", "0.0.0.0", "The host address the HiveServer2 WebUI will listen on"), diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java index 21028f4532..7243648645 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java @@ -135,7 +135,6 @@ public static void setUpBeforeClass() throws SQLException, ClassNotFoundExceptio System.setProperty(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL.varname, "verbose"); System.setProperty(ConfVars.HIVEMAPREDMODE.varname, "nonstrict"); System.setProperty(ConfVars.HIVE_AUTHORIZATION_MANAGER.varname, "org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider"); - System.setProperty(ConfVars.HIVE_SERVER2_PARALLEL_OPS_IN_SESSION.varname, "false"); Statement stmt1 = con1.createStatement(); assertNotNull("Statement is null", stmt1); @@ -328,23 +327,6 @@ private void checkBadUrl(String url) throws SQLException { } } - @Test - public void testSerializedExecution() throws Exception { - // Test running parallel queries (with parallel queries disabled). - // Should be serialized in the order of execution. - HiveStatement stmt1 = (HiveStatement) con.createStatement(); - HiveStatement stmt2 = (HiveStatement) con.createStatement(); - stmt1.execute("create temporary function sleepMsUDF as '" + SleepMsUDF.class.getName() + "'"); - stmt1.execute("create table test_ser_1(i int)"); - stmt1.executeAsync("insert into test_ser_1 select sleepMsUDF(under_col, 500) from " - + tableName + " limit 1"); - boolean isResultSet = stmt2.executeAsync("select * from test_ser_1"); - assertTrue(isResultSet); - ResultSet rs = stmt2.getResultSet(); - assertTrue(rs.next()); - assertFalse(rs.next()); - } - @Test public void testParentReferences() throws Exception { /* Test parent references from Statement */ @@ -2552,19 +2534,6 @@ public Integer evaluate(final Integer value) { } } - - // A udf which sleeps for some number of ms to simulate a long running query - public static class SleepMsUDF extends UDF { - public Integer evaluate(final Integer value, final Integer ms) { - try { - Thread.sleep(ms); - } catch (InterruptedException e) { - // No-op - } - return value; - } - } - /** * Loads data from a table containing non-ascii value column * Runs a query and compares the return value diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestHiveSessionImpl.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestHiveSessionImpl.java index c7fa5da8ea..d58a9133bf 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestHiveSessionImpl.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestHiveSessionImpl.java @@ -51,11 +51,11 @@ public void testLeakOperationHandle() throws HiveSQLException { HiveSessionImpl session = new HiveSessionImpl(protocol, username, password, serverhiveConf, ipAddress) { @Override - protected synchronized void acquire(boolean userAccess, boolean isOperation) { + protected synchronized void acquire(boolean userAccess) { } @Override - protected synchronized void release(boolean userAccess, boolean isOperation) { + protected synchronized void release(boolean userAccess) { } }; diff --git a/service/src/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/service/src/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java index 2dd90b69b3..ff46ed8802 100644 --- a/service/src/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java +++ b/service/src/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java @@ -31,7 +31,7 @@ public abstract class ExecuteStatementOperation extends Operation { public ExecuteStatementOperation(HiveSession parentSession, String statement, Map confOverlay, boolean runInBackground) { - super(parentSession, confOverlay, OperationType.EXECUTE_STATEMENT); + super(parentSession, confOverlay, OperationType.EXECUTE_STATEMENT, runInBackground); this.statement = statement; } 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 023f6b29ca..c3be295677 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 @@ -67,6 +67,7 @@ public abstract class Operation { public static final long DEFAULT_FETCH_MAX_ROWS = 100; protected boolean hasResultSet; protected volatile HiveSQLException operationException; + protected final boolean runAsync; protected volatile Future backgroundHandle; protected OperationLog operationLog; protected boolean isOperationLogEnabled; @@ -84,29 +85,23 @@ public abstract class Operation { protected static final EnumSet DEFAULT_FETCH_ORIENTATION_SET = EnumSet.of(FetchOrientation.FETCH_NEXT,FetchOrientation.FETCH_FIRST); - protected Operation(HiveSession parentSession, OperationType opType) { - this(parentSession, null, opType); - } - - protected Operation(HiveSession parentSession, Map confOverlay, - OperationType opType) { - this(parentSession, confOverlay, opType, false); - } + this(parentSession, null, opType, false); + } - protected Operation(HiveSession parentSession, - Map confOverlay, OperationType opType, boolean isAsyncQueryState) { + protected Operation(HiveSession parentSession, Map confOverlay, OperationType opType, boolean runInBackground) { this.parentSession = parentSession; if (confOverlay != null) { this.confOverlay = confOverlay; } + this.runAsync = runInBackground; this.opHandle = new OperationHandle(opType, parentSession.getProtocolVersion()); beginTime = System.currentTimeMillis(); lastAccessTime = beginTime; operationTimeout = HiveConf.getTimeVar(parentSession.getHiveConf(), HiveConf.ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT, TimeUnit.MILLISECONDS); setMetrics(state); - queryState = new QueryState(parentSession.getHiveConf(), confOverlay, isAsyncQueryState); + queryState = new QueryState(parentSession.getHiveConf(), confOverlay, runAsync); } public Future getBackgroundHandle() { @@ -118,9 +113,10 @@ protected void setBackgroundHandle(Future backgroundHandle) { } public boolean shouldRunAsync() { - return false; // Most operations cannot run asynchronously. + return runAsync; } + public HiveSession getParentSession() { return parentSession; } 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 3df831c211..57c954a170 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 @@ -89,6 +89,7 @@ */ @SuppressWarnings("deprecation") public class SQLOperation extends ExecuteStatementOperation { + private Driver driver = null; private CommandProcessorResponse response; private TableSchema resultSchema = null; @@ -100,7 +101,6 @@ public class SQLOperation extends ExecuteStatementOperation { private SQLOperationDisplay sqlOpDisplay; private long queryTimeout; private ScheduledExecutorService timeoutExecutor; - private final boolean runAsync; /** * A map to track query count running by each user @@ -112,7 +112,6 @@ public SQLOperation(HiveSession parentSession, String statement, Map doAsAction = new PrivilegedExceptionAction() { + @Override + public Object run() throws HiveSQLException { + Hive.set(parentHive); + // TODO: can this result in cross-thread reuse of session state? + SessionState.setCurrentSessionState(parentSessionState); + PerfLogger.setPerfLogger(parentPerfLogger); + // Set current OperationLog in this async thread for keeping on saving query log. + registerCurrentOperationLog(); + registerLoggingContext(); + try { + if (asyncPrepare) { + prepare(queryState); + } + runQuery(); + } catch (HiveSQLException e) { + setOperationException(e); + LOG.error("Error running hive query: ", e); + } finally { + unregisterLoggingContext(); + unregisterOperationLog(); + } + return null; + } + }; + try { + currentUGI.doAs(doAsAction); + } catch (Exception e) { + setOperationException(new HiveSQLException(e)); + LOG.error("Error running hive query as user : " + currentUGI.getShortUserName(), e); + } + finally { + /** + * We'll cache the ThreadLocal RawStore object for this background thread for an orderly cleanup + * when this thread is garbage collected later. + * @see org.apache.hive.service.server.ThreadWithGarbageCleanup#finalize() + */ + if (ThreadWithGarbageCleanup.currentThread() instanceof ThreadWithGarbageCleanup) { + ThreadWithGarbageCleanup currentThread = + (ThreadWithGarbageCleanup) ThreadWithGarbageCleanup.currentThread(); + currentThread.cacheThreadLocalRawStore(); + } + } + } + }; try { // This submit blocks if no background threads are available to run this operation - Future backgroundHandle = getParentSession().submitBackgroundOperation(work); + Future backgroundHandle = + getParentSession().getSessionManager().submitBackgroundOperation(backgroundOperation); setBackgroundHandle(backgroundHandle); } catch (RejectedExecutionException rejected) { setState(OperationState.ERROR); @@ -297,74 +345,6 @@ public void runInternal() throws HiveSQLException { } } - - private final class BackgroundWork implements Runnable { - private final UserGroupInformation currentUGI; - private final Hive parentHive; - private final PerfLogger parentPerfLogger; - private final SessionState parentSessionState; - private final boolean asyncPrepare; - - private BackgroundWork(UserGroupInformation currentUGI, - Hive parentHive, PerfLogger parentPerfLogger, - SessionState parentSessionState, boolean asyncPrepare) { - this.currentUGI = currentUGI; - this.parentHive = parentHive; - this.parentPerfLogger = parentPerfLogger; - this.parentSessionState = parentSessionState; - this.asyncPrepare = asyncPrepare; - } - - @Override - public void run() { - PrivilegedExceptionAction doAsAction = new PrivilegedExceptionAction() { - @Override - public Object run() throws HiveSQLException { - Hive.set(parentHive); - // TODO: can this result in cross-thread reuse of session state? - SessionState.setCurrentSessionState(parentSessionState); - PerfLogger.setPerfLogger(parentPerfLogger); - // Set current OperationLog in this async thread for keeping on saving query log. - registerCurrentOperationLog(); - registerLoggingContext(); - try { - if (asyncPrepare) { - prepare(queryState); - } - runQuery(); - } catch (HiveSQLException e) { - setOperationException(e); - LOG.error("Error running hive query: ", e); - } finally { - unregisterLoggingContext(); - unregisterOperationLog(); - } - return null; - } - }; - - try { - currentUGI.doAs(doAsAction); - } catch (Exception e) { - setOperationException(new HiveSQLException(e)); - LOG.error("Error running hive query as user : " + currentUGI.getShortUserName(), e); - } - finally { - /** - * We'll cache the ThreadLocal RawStore object for this background thread for an orderly cleanup - * when this thread is garbage collected later. - * @see org.apache.hive.service.server.ThreadWithGarbageCleanup#finalize() - */ - if (ThreadWithGarbageCleanup.currentThread() instanceof ThreadWithGarbageCleanup) { - ThreadWithGarbageCleanup currentThread = - (ThreadWithGarbageCleanup) ThreadWithGarbageCleanup.currentThread(); - currentThread.cacheThreadLocalRawStore(); - } - } - } - } - - /** * Returns the current UGI on the stack * @param opConfig @@ -683,5 +663,4 @@ private void decrementUserQueries(Metrics metrics) throws IOException { public String getExecutionEngine() { return queryState.getConf().getVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE); } - } diff --git a/service/src/java/org/apache/hive/service/cli/session/HiveSession.java b/service/src/java/org/apache/hive/service/cli/session/HiveSession.java index e5d865b5d9..78ff388be2 100644 --- a/service/src/java/org/apache/hive/service/cli/session/HiveSession.java +++ b/service/src/java/org/apache/hive/service/cli/session/HiveSession.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.Map; -import java.util.concurrent.Future; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.ql.metadata.Hive; @@ -212,6 +211,4 @@ void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) void closeExpiredOperations(); long getNoOperationTime(); - - Future submitBackgroundOperation(Runnable work); } 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 ae6cb1ad3c..efc834d972 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 @@ -24,13 +24,11 @@ import java.io.IOException; import java.io.InputStreamReader; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.Future; -import java.util.concurrent.FutureTask; -import java.util.concurrent.Semaphore; import org.apache.commons.io.FileUtils; import org.apache.commons.lang.StringUtils; @@ -40,6 +38,8 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hadoop.hive.ql.exec.ListSinkOperator; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.history.HiveHistory; import org.apache.hadoop.hive.ql.metadata.Hive; @@ -113,18 +113,16 @@ public class HiveSessionImpl implements HiveSession { private volatile long lastAccessTime; private volatile long lastIdleTime; - private final Semaphore operationLock; - public HiveSessionImpl(SessionHandle sessionHandle, TProtocolVersion protocol, - String username, String password, HiveConf serverConf, String ipAddress) { + + public HiveSessionImpl(SessionHandle sessionHandle, TProtocolVersion protocol, String username, String password, + HiveConf serverhiveConf, String ipAddress) { this.username = username; this.password = password; creationTime = System.currentTimeMillis(); this.sessionHandle = sessionHandle != null ? sessionHandle : new SessionHandle(protocol); - this.sessionConf = new HiveConf(serverConf); + this.sessionConf = new HiveConf(serverhiveConf); this.ipAddress = ipAddress; - this.operationLock = serverConf.getBoolVar( - ConfVars.HIVE_SERVER2_PARALLEL_OPS_IN_SESSION) ? null : new Semaphore(1); try { // In non-impersonation mode, map scheduler queue to current user // if fair scheduler is configured. @@ -327,27 +325,7 @@ public void setOperationManager(OperationManager operationManager) { this.operationManager = operationManager; } - protected void acquire(boolean userAccess, boolean isOperation) { - if (isOperation && operationLock != null) { - try { - operationLock.acquire(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } - boolean success = false; - try { - acquireAfterOpLock(userAccess); - success = true; - } finally { - if (!success && isOperation && operationLock != null) { - operationLock.release(); - } - } - } - - private synchronized void acquireAfterOpLock(boolean userAccess) { + protected synchronized void acquire(boolean userAccess) { // Need to make sure that the this HiveServer2's session's SessionState is // stored in the thread local for the handler thread. SessionState.setCurrentSessionState(sessionState); @@ -367,17 +345,7 @@ private synchronized void acquireAfterOpLock(boolean userAccess) { * when this thread is garbage collected later. * @see org.apache.hive.service.server.ThreadWithGarbageCleanup#finalize() */ - protected void release(boolean userAccess, boolean isOperation) { - try { - releaseBeforeOpLock(userAccess); - } finally { - if (isOperation && operationLock != null) { - operationLock.release(); - } - } - } - - private synchronized void releaseBeforeOpLock(boolean userAccess) { + protected synchronized void release(boolean userAccess) { if (sessionState != null) { // can be null in-case of junit tests. skip reset. // reset thread name at release time. @@ -433,7 +401,7 @@ public IMetaStoreClient getMetaStoreClient() throws HiveSQLException { @Override public GetInfoValue getInfo(GetInfoType getInfoType) throws HiveSQLException { - acquire(true, true); + acquire(true); try { switch (getInfoType) { case CLI_SERVER_NAME: @@ -453,7 +421,7 @@ public GetInfoValue getInfo(GetInfoType getInfoType) throw new HiveSQLException("Unrecognized GetInfoType value: " + getInfoType.toString()); } } finally { - release(true, true); + release(true); } } @@ -481,11 +449,12 @@ public OperationHandle executeStatementAsync(String statement, Map confOverlay, boolean runAsync, long queryTimeout) throws HiveSQLException { - acquire(true, true); + acquire(true); OperationManager operationManager = getOperationManager(); - ExecuteStatementOperation operation = operationManager.newExecuteStatementOperation( - getSession(), statement, confOverlay, runAsync, queryTimeout); + ExecuteStatementOperation operation = + operationManager.newExecuteStatementOperation(getSession(), statement, confOverlay, + runAsync, queryTimeout); OperationHandle opHandle = operation.getHandle(); try { operation.run(); @@ -498,29 +467,14 @@ private OperationHandle executeStatementInternal(String statement, operationManager.closeOperation(opHandle); throw e; } finally { - if (operation.getBackgroundHandle() == null) { - release(true, true); // Not async, or wasn't submitted for some reason (failure, etc.) - } else { - releaseBeforeOpLock(true); // Release, but keep the lock (if present). - } + release(true); } } - @Override - public Future submitBackgroundOperation(Runnable work) { - return getSessionManager().submitBackgroundOperation( - operationLock == null ? work : new FutureTask(work, null) { - protected void done() { - // We assume this always comes from a user operation that took the lock. - operationLock.release(); - }; - }); - } - @Override public OperationHandle getTypeInfo() throws HiveSQLException { - acquire(true, true); + acquire(true); OperationManager operationManager = getOperationManager(); GetTypeInfoOperation operation = operationManager.newGetTypeInfoOperation(getSession()); @@ -533,14 +487,14 @@ public OperationHandle getTypeInfo() operationManager.closeOperation(opHandle); throw e; } finally { - release(true, true); + release(true); } } @Override public OperationHandle getCatalogs() throws HiveSQLException { - acquire(true, true); + acquire(true); OperationManager operationManager = getOperationManager(); GetCatalogsOperation operation = operationManager.newGetCatalogsOperation(getSession()); @@ -553,14 +507,14 @@ public OperationHandle getCatalogs() operationManager.closeOperation(opHandle); throw e; } finally { - release(true, true); + release(true); } } @Override public OperationHandle getSchemas(String catalogName, String schemaName) throws HiveSQLException { - acquire(true, true); + acquire(true); OperationManager operationManager = getOperationManager(); GetSchemasOperation operation = @@ -574,7 +528,7 @@ public OperationHandle getSchemas(String catalogName, String schemaName) operationManager.closeOperation(opHandle); throw e; } finally { - release(true, true); + release(true); } } @@ -582,7 +536,7 @@ public OperationHandle getSchemas(String catalogName, String schemaName) public OperationHandle getTables(String catalogName, String schemaName, String tableName, List tableTypes) throws HiveSQLException { - acquire(true, true); + acquire(true); OperationManager operationManager = getOperationManager(); MetadataOperation operation = @@ -596,14 +550,14 @@ public OperationHandle getTables(String catalogName, String schemaName, String t operationManager.closeOperation(opHandle); throw e; } finally { - release(true, true); + release(true); } } @Override public OperationHandle getTableTypes() throws HiveSQLException { - acquire(true, true); + acquire(true); OperationManager operationManager = getOperationManager(); GetTableTypesOperation operation = operationManager.newGetTableTypesOperation(getSession()); @@ -616,14 +570,14 @@ public OperationHandle getTableTypes() operationManager.closeOperation(opHandle); throw e; } finally { - release(true, true); + release(true); } } @Override public OperationHandle getColumns(String catalogName, String schemaName, String tableName, String columnName) throws HiveSQLException { - acquire(true, true); + acquire(true); String addedJars = Utilities.getResourceFiles(sessionConf, SessionState.ResourceType.JAR); if (StringUtils.isNotBlank(addedJars)) { IMetaStoreClient metastoreClient = getSession().getMetaStoreClient(); @@ -641,7 +595,7 @@ public OperationHandle getColumns(String catalogName, String schemaName, operationManager.closeOperation(opHandle); throw e; } finally { - release(true, true); + release(true); } } @@ -654,7 +608,7 @@ private void addOpHandle(OperationHandle opHandle) { @Override public OperationHandle getFunctions(String catalogName, String schemaName, String functionName) throws HiveSQLException { - acquire(true, true); + acquire(true); OperationManager operationManager = getOperationManager(); GetFunctionsOperation operation = operationManager @@ -668,14 +622,14 @@ public OperationHandle getFunctions(String catalogName, String schemaName, Strin operationManager.closeOperation(opHandle); throw e; } finally { - release(true, true); + release(true); } } @Override public void close() throws HiveSQLException { try { - acquire(true, false); + acquire(true); // Iterate through the opHandles and close their operations List ops = null; synchronized (opHandleSet) { @@ -717,7 +671,7 @@ public void close() throws HiveSQLException { } sessionHive = null; } - release(true, false); + release(true); } } @@ -778,7 +732,7 @@ public long getNoOperationTime() { } private void closeTimedOutOperations(List operations) { - acquire(false, false); + acquire(false); try { for (Operation operation : operations) { synchronized (opHandleSet) { @@ -791,54 +745,54 @@ private void closeTimedOutOperations(List operations) { } } } finally { - release(false, false); + release(false); } } @Override public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { - acquire(true, false); + acquire(true); try { sessionManager.getOperationManager().cancelOperation(opHandle); } finally { - release(true, false); + release(true); } } @Override public void closeOperation(OperationHandle opHandle) throws HiveSQLException { - acquire(true, false); + acquire(true); try { operationManager.closeOperation(opHandle); synchronized (opHandleSet) { opHandleSet.remove(opHandle); } } finally { - release(true, false); + release(true); } } @Override public TableSchema getResultSetMetadata(OperationHandle opHandle) throws HiveSQLException { - acquire(true, true); + acquire(true); try { return sessionManager.getOperationManager().getOperationResultSetSchema(opHandle); } finally { - release(true, true); + release(true); } } @Override public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, long maxRows, FetchType fetchType) throws HiveSQLException { - acquire(true, false); + acquire(true); try { if (fetchType == FetchType.QUERY_OUTPUT) { return operationManager.getOperationNextRowSet(opHandle, orientation, maxRows); } return operationManager.getOperationLogRowSet(opHandle, orientation, maxRows, sessionConf); } finally { - release(true, false); + release(true); } } @@ -892,7 +846,7 @@ private String getUserFromToken(HiveAuthFactory authFactory, String tokenStr) th @Override public OperationHandle getPrimaryKeys(String catalog, String schema, String table) throws HiveSQLException { - acquire(true, true); + acquire(true); OperationManager operationManager = getOperationManager(); GetPrimaryKeysOperation operation = operationManager @@ -906,7 +860,7 @@ public OperationHandle getPrimaryKeys(String catalog, String schema, operationManager.closeOperation(opHandle); throw e; } finally { - release(true, true); + release(true); } } @@ -914,7 +868,7 @@ public OperationHandle getPrimaryKeys(String catalog, String schema, public OperationHandle getCrossReference(String primaryCatalog, String primarySchema, String primaryTable, String foreignCatalog, String foreignSchema, String foreignTable) throws HiveSQLException { - acquire(true, true); + acquire(true); OperationManager operationManager = getOperationManager(); GetCrossReferenceOperation operation = operationManager @@ -930,7 +884,7 @@ public OperationHandle getCrossReference(String primaryCatalog, operationManager.closeOperation(opHandle); throw e; } finally { - release(true, true); + release(true); } } } diff --git a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java index afed9e2e99..f7b3412108 100644 --- a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java +++ b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java @@ -89,10 +89,9 @@ public String getDelegationToken () { @Override public void close() throws HiveSQLException { try { - acquire(true, false); + acquire(true); cancelDelegationToken(); } finally { - release(true, false); try { super.close(); } finally {