From 7feeb82cb7f462e44f7e698c7c3b6ac3a77aade4 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 29 Apr 2016 09:32:42 -0700 Subject: [SPARK-14987][SQL] inline hive-service (cli) into sql/hive-thriftserver ## What changes were proposed in this pull request? This PR copy the thrift-server from hive-service-1.2 (including TCLIService.thrift and generated Java source code) into sql/hive-thriftserver, so we can do further cleanup and improvements. ## How was this patch tested? Existing tests. Author: Davies Liu Closes #12764 from davies/thrift_server. --- .../org/apache/hive/service/AbstractService.java | 184 +++++ .../org/apache/hive/service/BreakableService.java | 121 ++++ .../org/apache/hive/service/CompositeService.java | 133 ++++ .../java/org/apache/hive/service/CookieSigner.java | 108 +++ .../org/apache/hive/service/FilterService.java | 83 +++ .../main/java/org/apache/hive/service/Service.java | 122 ++++ .../org/apache/hive/service/ServiceException.java | 38 ++ .../org/apache/hive/service/ServiceOperations.java | 141 ++++ .../hive/service/ServiceStateChangeListener.java | 46 ++ .../java/org/apache/hive/service/ServiceUtils.java | 44 ++ .../auth/AnonymousAuthenticationProviderImpl.java | 33 + .../auth/AuthenticationProviderFactory.java | 71 ++ .../auth/CustomAuthenticationProviderImpl.java | 50 ++ .../apache/hive/service/auth/HiveAuthFactory.java | 364 ++++++++++ .../apache/hive/service/auth/HttpAuthUtils.java | 189 ++++++ .../service/auth/HttpAuthenticationException.java | 43 ++ .../hive/service/auth/KerberosSaslHelper.java | 111 ++++ .../auth/LdapAuthenticationProviderImpl.java | 84 +++ .../auth/PamAuthenticationProviderImpl.java | 51 ++ .../service/auth/PasswdAuthenticationProvider.java | 39 ++ .../apache/hive/service/auth/PlainSaslHelper.java | 154 +++++ .../apache/hive/service/auth/PlainSaslServer.java | 177 +++++ .../java/org/apache/hive/service/auth/SaslQOP.java | 62 ++ .../hive/service/auth/TSetIpAddressProcessor.java | 117 ++++ .../service/auth/TSubjectAssumingTransport.java | 70 ++ .../org/apache/hive/service/cli/CLIService.java | 508 ++++++++++++++ .../apache/hive/service/cli/CLIServiceClient.java | 56 ++ .../apache/hive/service/cli/CLIServiceUtils.java | 76 +++ .../java/org/apache/hive/service/cli/Column.java | 423 ++++++++++++ .../apache/hive/service/cli/ColumnBasedSet.java | 149 +++++ .../apache/hive/service/cli/ColumnDescriptor.java | 99 +++ .../org/apache/hive/service/cli/ColumnValue.java | 307 +++++++++ .../hive/service/cli/EmbeddedCLIServiceClient.java | 208 ++++++ .../apache/hive/service/cli/FetchOrientation.java | 54 ++ .../org/apache/hive/service/cli/FetchType.java | 47 ++ .../org/apache/hive/service/cli/GetInfoType.java | 96 +++ .../org/apache/hive/service/cli/GetInfoValue.java | 82 +++ .../java/org/apache/hive/service/cli/Handle.java | 78 +++ .../apache/hive/service/cli/HandleIdentifier.java | 113 ++++ .../apache/hive/service/cli/HiveSQLException.java | 248 +++++++ .../org/apache/hive/service/cli/ICLIService.java | 105 +++ .../apache/hive/service/cli/OperationHandle.java | 102 +++ .../apache/hive/service/cli/OperationState.java | 107 +++ .../apache/hive/service/cli/OperationStatus.java | 43 ++ .../org/apache/hive/service/cli/OperationType.java | 58 ++ .../hive/service/cli/PatternOrIdentifier.java | 47 ++ .../org/apache/hive/service/cli/RowBasedSet.java | 140 ++++ .../java/org/apache/hive/service/cli/RowSet.java | 38 ++ .../org/apache/hive/service/cli/RowSetFactory.java | 41 ++ .../org/apache/hive/service/cli/SessionHandle.java | 67 ++ .../org/apache/hive/service/cli/TableSchema.java | 102 +++ .../java/org/apache/hive/service/cli/Type.java | 348 ++++++++++ .../apache/hive/service/cli/TypeDescriptor.java | 159 +++++ .../apache/hive/service/cli/TypeQualifiers.java | 133 ++++ .../cli/operation/ClassicTableTypeMapping.java | 86 +++ .../cli/operation/ExecuteStatementOperation.java | 70 ++ .../cli/operation/GetCatalogsOperation.java | 81 +++ .../service/cli/operation/GetColumnsOperation.java | 236 +++++++ .../cli/operation/GetFunctionsOperation.java | 148 +++++ .../service/cli/operation/GetSchemasOperation.java | 104 +++ .../cli/operation/GetTableTypesOperation.java | 93 +++ .../service/cli/operation/GetTablesOperation.java | 135 ++++ .../cli/operation/GetTypeInfoOperation.java | 142 ++++ .../cli/operation/HiveCommandOperation.java | 213 ++++++ .../cli/operation/HiveTableTypeMapping.java | 51 ++ .../service/cli/operation/LogDivertAppender.java | 209 ++++++ .../service/cli/operation/MetadataOperation.java | 135 ++++ .../hive/service/cli/operation/Operation.java | 322 +++++++++ .../service/cli/operation/OperationManager.java | 284 ++++++++ .../hive/service/cli/operation/SQLOperation.java | 473 +++++++++++++ .../service/cli/operation/TableTypeMapping.java | 44 ++ .../cli/operation/TableTypeMappingFactory.java | 37 ++ .../hive/service/cli/session/HiveSession.java | 156 +++++ .../hive/service/cli/session/HiveSessionBase.java | 93 +++ .../hive/service/cli/session/HiveSessionHook.java | 37 ++ .../cli/session/HiveSessionHookContext.java | 46 ++ .../cli/session/HiveSessionHookContextImpl.java | 52 ++ .../hive/service/cli/session/HiveSessionImpl.java | 734 ++++++++++++++++++++ .../cli/session/HiveSessionImplwithUGI.java | 183 +++++ .../hive/service/cli/session/HiveSessionProxy.java | 91 +++ .../hive/service/cli/session/SessionManager.java | 402 +++++++++++ .../cli/thrift/EmbeddedThriftBinaryCLIService.java | 48 ++ .../service/cli/thrift/ThriftBinaryCLIService.java | 108 +++ .../hive/service/cli/thrift/ThriftCLIService.java | 740 +++++++++++++++++++++ .../service/cli/thrift/ThriftCLIServiceClient.java | 440 ++++++++++++ .../service/cli/thrift/ThriftHttpCLIService.java | 167 +++++ .../hive/service/cli/thrift/ThriftHttpServlet.java | 546 +++++++++++++++ .../apache/hive/service/server/HiveServer2.java | 621 +++++++++++++++++ .../server/ThreadFactoryWithGarbageCleanup.java | 64 ++ .../service/server/ThreadWithGarbageCleanup.java | 77 +++ 90 files changed, 14137 insertions(+) create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/BreakableService.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/FilterService.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceException.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceStateChangeListener.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthenticationException.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PamAuthenticationProviderImpl.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslServer.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSubjectAssumingTransport.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceClient.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceUtils.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Column.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnValue.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchOrientation.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchType.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoType.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoValue.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Handle.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ICLIService.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationHandle.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationStatus.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationType.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSet.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSetFactory.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/SessionHandle.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TableSchema.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Type.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMappingFactory.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSession.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/EmbeddedThriftBinaryCLIService.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadFactoryWithGarbageCleanup.java create mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java (limited to 'sql/hive-thriftserver/src/main') diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java new file mode 100644 index 0000000000..c2a2b2d478 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java @@ -0,0 +1,184 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; + +/** + * AbstractService. + * + */ +public abstract class AbstractService implements Service { + + private static final Log LOG = LogFactory.getLog(AbstractService.class); + + /** + * Service state: initially {@link STATE#NOTINITED}. + */ + private STATE state = STATE.NOTINITED; + + /** + * Service name. + */ + private final String name; + /** + * Service start time. Will be zero until the service is started. + */ + private long startTime; + + /** + * The configuration. Will be null until the service is initialized. + */ + private HiveConf hiveConf; + + /** + * List of state change listeners; it is final to ensure + * that it will never be null. + */ + private final List listeners = + new ArrayList(); + + /** + * Construct the service. + * + * @param name + * service name + */ + public AbstractService(String name) { + this.name = name; + } + + @Override + public synchronized STATE getServiceState() { + return state; + } + + /** + * {@inheritDoc} + * + * @throws IllegalStateException + * if the current service state does not permit + * this action + */ + @Override + public synchronized void init(HiveConf hiveConf) { + ensureCurrentState(STATE.NOTINITED); + this.hiveConf = hiveConf; + changeState(STATE.INITED); + LOG.info("Service:" + getName() + " is inited."); + } + + /** + * {@inheritDoc} + * + * @throws IllegalStateException + * if the current service state does not permit + * this action + */ + @Override + public synchronized void start() { + startTime = System.currentTimeMillis(); + ensureCurrentState(STATE.INITED); + changeState(STATE.STARTED); + LOG.info("Service:" + getName() + " is started."); + } + + /** + * {@inheritDoc} + * + * @throws IllegalStateException + * if the current service state does not permit + * this action + */ + @Override + public synchronized void stop() { + if (state == STATE.STOPPED || + state == STATE.INITED || + state == STATE.NOTINITED) { + // already stopped, or else it was never + // started (eg another service failing canceled startup) + return; + } + ensureCurrentState(STATE.STARTED); + changeState(STATE.STOPPED); + LOG.info("Service:" + getName() + " is stopped."); + } + + @Override + public synchronized void register(ServiceStateChangeListener l) { + listeners.add(l); + } + + @Override + public synchronized void unregister(ServiceStateChangeListener l) { + listeners.remove(l); + } + + @Override + public String getName() { + return name; + } + + @Override + public synchronized HiveConf getHiveConf() { + return hiveConf; + } + + @Override + public long getStartTime() { + return startTime; + } + + /** + * Verify that that a service is in a given state. + * + * @param currentState + * the desired state + * @throws IllegalStateException + * if the service state is different from + * the desired state + */ + private void ensureCurrentState(STATE currentState) { + ServiceOperations.ensureCurrentState(state, currentState); + } + + /** + * Change to a new state and notify all listeners. + * This is a private method that is only invoked from synchronized methods, + * which avoid having to clone the listener list. It does imply that + * the state change listener methods should be short lived, as they + * will delay the state transition. + * + * @param newState + * new service state + */ + private void changeState(STATE newState) { + state = newState; + // notify listeners + for (ServiceStateChangeListener l : listeners) { + l.stateChanged(this); + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/BreakableService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/BreakableService.java new file mode 100644 index 0000000000..9c44beb2fb --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/BreakableService.java @@ -0,0 +1,121 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hive.service.Service.STATE; + +/** + * This is a service that can be configured to break on any of the lifecycle + * events, so test the failure handling of other parts of the service + * infrastructure. + * + * It retains a counter to the number of times each entry point is called - + * these counters are incremented before the exceptions are raised and + * before the superclass state methods are invoked. + * + */ +public class BreakableService extends AbstractService { + private boolean failOnInit; + private boolean failOnStart; + private boolean failOnStop; + private final int[] counts = new int[4]; + + public BreakableService() { + this(false, false, false); + } + + public BreakableService(boolean failOnInit, + boolean failOnStart, + boolean failOnStop) { + super("BreakableService"); + this.failOnInit = failOnInit; + this.failOnStart = failOnStart; + this.failOnStop = failOnStop; + inc(STATE.NOTINITED); + } + + private int convert(STATE state) { + switch (state) { + case NOTINITED: return 0; + case INITED: return 1; + case STARTED: return 2; + case STOPPED: return 3; + default: return 0; + } + } + + private void inc(STATE state) { + int index = convert(state); + counts[index] ++; + } + + public int getCount(STATE state) { + return counts[convert(state)]; + } + + private void maybeFail(boolean fail, String action) { + if (fail) { + throw new BrokenLifecycleEvent(action); + } + } + + @Override + public void init(HiveConf conf) { + inc(STATE.INITED); + maybeFail(failOnInit, "init"); + super.init(conf); + } + + @Override + public void start() { + inc(STATE.STARTED); + maybeFail(failOnStart, "start"); + super.start(); + } + + @Override + public void stop() { + inc(STATE.STOPPED); + maybeFail(failOnStop, "stop"); + super.stop(); + } + + public void setFailOnInit(boolean failOnInit) { + this.failOnInit = failOnInit; + } + + public void setFailOnStart(boolean failOnStart) { + this.failOnStart = failOnStart; + } + + public void setFailOnStop(boolean failOnStop) { + this.failOnStop = failOnStop; + } + + /** + * The exception explicitly raised on a failure + */ + public static class BrokenLifecycleEvent extends RuntimeException { + BrokenLifecycleEvent(String action) { + super("Lifecycle Failure during " + action); + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java new file mode 100644 index 0000000000..897911872b --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java @@ -0,0 +1,133 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; + +/** + * CompositeService. + * + */ +public class CompositeService extends AbstractService { + + private static final Log LOG = LogFactory.getLog(CompositeService.class); + + private final List serviceList = new ArrayList(); + + public CompositeService(String name) { + super(name); + } + + public Collection getServices() { + return Collections.unmodifiableList(serviceList); + } + + protected synchronized void addService(Service service) { + serviceList.add(service); + } + + protected synchronized boolean removeService(Service service) { + return serviceList.remove(service); + } + + @Override + public synchronized void init(HiveConf hiveConf) { + for (Service service : serviceList) { + service.init(hiveConf); + } + super.init(hiveConf); + } + + @Override + public synchronized void start() { + int i = 0; + try { + for (int n = serviceList.size(); i < n; i++) { + Service service = serviceList.get(i); + service.start(); + } + super.start(); + } catch (Throwable e) { + LOG.error("Error starting services " + getName(), e); + // Note that the state of the failed service is still INITED and not + // STARTED. Even though the last service is not started completely, still + // call stop() on all services including failed service to make sure cleanup + // happens. + stop(i); + throw new ServiceException("Failed to Start " + getName(), e); + } + + } + + @Override + public synchronized void stop() { + if (this.getServiceState() == STATE.STOPPED) { + // The base composite-service is already stopped, don't do anything again. + return; + } + if (serviceList.size() > 0) { + stop(serviceList.size() - 1); + } + super.stop(); + } + + private synchronized void stop(int numOfServicesStarted) { + // stop in reserve order of start + for (int i = numOfServicesStarted; i >= 0; i--) { + Service service = serviceList.get(i); + try { + service.stop(); + } catch (Throwable t) { + LOG.info("Error stopping " + service.getName(), t); + } + } + } + + /** + * JVM Shutdown hook for CompositeService which will stop the given + * CompositeService gracefully in case of JVM shutdown. + */ + public static class CompositeServiceShutdownHook implements Runnable { + + private final CompositeService compositeService; + + public CompositeServiceShutdownHook(CompositeService compositeService) { + this.compositeService = compositeService; + } + + @Override + public void run() { + try { + // Stop the Composite Service + compositeService.stop(); + } catch (Throwable t) { + LOG.info("Error stopping " + compositeService.getName(), t); + } + } + } + + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java new file mode 100644 index 0000000000..ee51c24351 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.logging.LogFactory; +import org.apache.commons.logging.Log; + +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; + +/** + * The cookie signer generates a signature based on SHA digest + * and appends it to the cookie value generated at the + * server side. It uses SHA digest algorithm to sign and verify signatures. + */ +public class CookieSigner { + private static final String SIGNATURE = "&s="; + private static final String SHA_STRING = "SHA"; + private byte[] secretBytes; + private static final Log LOG = LogFactory.getLog(CookieSigner.class); + + /** + * Constructor + * @param secret Secret Bytes + */ + public CookieSigner(byte[] secret) { + if (secret == null) { + throw new IllegalArgumentException(" NULL Secret Bytes"); + } + this.secretBytes = secret.clone(); + } + + /** + * Sign the cookie given the string token as input. + * @param str Input token + * @return Signed token that can be used to create a cookie + */ + public String signCookie(String str) { + if (str == null || str.isEmpty()) { + throw new IllegalArgumentException("NULL or empty string to sign"); + } + String signature = getSignature(str); + + if (LOG.isDebugEnabled()) { + LOG.debug("Signature generated for " + str + " is " + signature); + } + return str + SIGNATURE + signature; + } + + /** + * Verify a signed string and extracts the original string. + * @param signedStr The already signed string + * @return Raw Value of the string without the signature + */ + public String verifyAndExtract(String signedStr) { + int index = signedStr.lastIndexOf(SIGNATURE); + if (index == -1) { + throw new IllegalArgumentException("Invalid input sign: " + signedStr); + } + String originalSignature = signedStr.substring(index + SIGNATURE.length()); + String rawValue = signedStr.substring(0, index); + String currentSignature = getSignature(rawValue); + + if (LOG.isDebugEnabled()) { + LOG.debug("Signature generated for " + rawValue + " inside verify is " + currentSignature); + } + if (!originalSignature.equals(currentSignature)) { + throw new IllegalArgumentException("Invalid sign, original = " + originalSignature + + " current = " + currentSignature); + } + return rawValue; + } + + /** + * Get the signature of the input string based on SHA digest algorithm. + * @param str Input token + * @return Signed String + */ + private String getSignature(String str) { + try { + MessageDigest md = MessageDigest.getInstance(SHA_STRING); + md.update(str.getBytes()); + md.update(secretBytes); + byte[] digest = md.digest(); + return new Base64(0).encodeToString(digest); + } catch (NoSuchAlgorithmException ex) { + throw new RuntimeException("Invalid SHA digest String: " + SHA_STRING + + " " + ex.getMessage(), ex); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/FilterService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/FilterService.java new file mode 100644 index 0000000000..5a50874541 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/FilterService.java @@ -0,0 +1,83 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +import org.apache.hadoop.hive.conf.HiveConf; + +/** + * FilterService. + * + */ +public class FilterService implements Service { + + + private final Service service; + private final long startTime = System.currentTimeMillis(); + + public FilterService(Service service) { + this.service = service; + } + + @Override + public void init(HiveConf config) { + service.init(config); + } + + @Override + public void start() { + service.start(); + } + + @Override + public void stop() { + service.stop(); + } + + + @Override + public void register(ServiceStateChangeListener listener) { + service.register(listener); + } + + @Override + public void unregister(ServiceStateChangeListener listener) { + service.unregister(listener); + } + + @Override + public String getName() { + return service.getName(); + } + + @Override + public HiveConf getHiveConf() { + return service.getHiveConf(); + } + + @Override + public STATE getServiceState() { + return service.getServiceState(); + } + + @Override + public long getStartTime() { + return startTime; + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java new file mode 100644 index 0000000000..2111837cac --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java @@ -0,0 +1,122 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +import org.apache.hadoop.hive.conf.HiveConf; + +/** + * Service. + * + */ +public interface Service { + + /** + * Service states + */ + public enum STATE { + /** Constructed but not initialized */ + NOTINITED, + + /** Initialized but not started or stopped */ + INITED, + + /** started and not stopped */ + STARTED, + + /** stopped. No further state transitions are permitted */ + STOPPED + } + + /** + * Initialize the service. + * + * The transition must be from {@link STATE#NOTINITED} to {@link STATE#INITED} unless the + * operation failed and an exception was raised. + * + * @param config + * the configuration of the service + */ + void init(HiveConf conf); + + + /** + * Start the service. + * + * The transition should be from {@link STATE#INITED} to {@link STATE#STARTED} unless the + * operation failed and an exception was raised. + */ + void start(); + + /** + * Stop the service. + * + * This operation must be designed to complete regardless of the initial state + * of the service, including the state of all its internal fields. + */ + void stop(); + + /** + * Register an instance of the service state change events. + * + * @param listener + * a new listener + */ + void register(ServiceStateChangeListener listener); + + /** + * Unregister a previously instance of the service state change events. + * + * @param listener + * the listener to unregister. + */ + void unregister(ServiceStateChangeListener listener); + + /** + * Get the name of this service. + * + * @return the service name + */ + String getName(); + + /** + * Get the configuration of this service. + * This is normally not a clone and may be manipulated, though there are no + * guarantees as to what the consequences of such actions may be + * + * @return the current configuration, unless a specific implementation chooses + * otherwise. + */ + HiveConf getHiveConf(); + + /** + * Get the current service state + * + * @return the state of the service + */ + STATE getServiceState(); + + /** + * Get the service start time + * + * @return the start time of the service. This will be zero if the service + * has not yet been started. + */ + long getStartTime(); + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceException.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceException.java new file mode 100644 index 0000000000..3622cf8920 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceException.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +/** + * ServiceException. + * + */ +public class ServiceException extends RuntimeException { + + public ServiceException(Throwable cause) { + super(cause); + } + + public ServiceException(String message) { + super(message); + } + + public ServiceException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java new file mode 100644 index 0000000000..8946219d85 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java @@ -0,0 +1,141 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; + +/** + * ServiceOperations. + * + */ +public final class ServiceOperations { + private static final Log LOG = LogFactory.getLog(AbstractService.class); + + private ServiceOperations() { + } + + /** + * Verify that that a service is in a given state. + * @param state the actual state a service is in + * @param expectedState the desired state + * @throws IllegalStateException if the service state is different from + * the desired state + */ + public static void ensureCurrentState(Service.STATE state, + Service.STATE expectedState) { + if (state != expectedState) { + throw new IllegalStateException("For this operation, the " + + "current service state must be " + + expectedState + + " instead of " + state); + } + } + + /** + * Initialize a service. + *

+ * The service state is checked before the operation begins. + * This process is not thread safe. + * @param service a service that must be in the state + * {@link Service.STATE#NOTINITED} + * @param configuration the configuration to initialize the service with + * @throws RuntimeException on a state change failure + * @throws IllegalStateException if the service is in the wrong state + */ + + public static void init(Service service, HiveConf configuration) { + Service.STATE state = service.getServiceState(); + ensureCurrentState(state, Service.STATE.NOTINITED); + service.init(configuration); + } + + /** + * Start a service. + *

+ * The service state is checked before the operation begins. + * This process is not thread safe. + * @param service a service that must be in the state + * {@link Service.STATE#INITED} + * @throws RuntimeException on a state change failure + * @throws IllegalStateException if the service is in the wrong state + */ + + public static void start(Service service) { + Service.STATE state = service.getServiceState(); + ensureCurrentState(state, Service.STATE.INITED); + service.start(); + } + + /** + * Initialize then start a service. + *

+ * The service state is checked before the operation begins. + * This process is not thread safe. + * @param service a service that must be in the state + * {@link Service.STATE#NOTINITED} + * @param configuration the configuration to initialize the service with + * @throws RuntimeException on a state change failure + * @throws IllegalStateException if the service is in the wrong state + */ + public static void deploy(Service service, HiveConf configuration) { + init(service, configuration); + start(service); + } + + /** + * Stop a service. + *

Do nothing if the service is null or not + * in a state in which it can be/needs to be stopped. + *

+ * The service state is checked before the operation begins. + * This process is not thread safe. + * @param service a service or null + */ + public static void stop(Service service) { + if (service != null) { + Service.STATE state = service.getServiceState(); + if (state == Service.STATE.STARTED) { + service.stop(); + } + } + } + + /** + * Stop a service; if it is null do nothing. Exceptions are caught and + * logged at warn level. (but not Throwables). This operation is intended to + * be used in cleanup operations + * + * @param service a service; may be null + * @return any exception that was caught; null if none was. + */ + public static Exception stopQuietly(Service service) { + try { + stop(service); + } catch (Exception e) { + LOG.warn("When stopping the service " + service.getName() + + " : " + e, + e); + return e; + } + return null; + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceStateChangeListener.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceStateChangeListener.java new file mode 100644 index 0000000000..16ad9a991e --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceStateChangeListener.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +/** + * ServiceStateChangeListener. + * + */ +public interface ServiceStateChangeListener { + + /** + * Callback to notify of a state change. The service will already + * have changed state before this callback is invoked. + * + * This operation is invoked on the thread that initiated the state change, + * while the service itself in in a sychronized section. + *

    + *
  1. Any long-lived operation here will prevent the service state + * change from completing in a timely manner.
  2. + *
  3. If another thread is somehow invoked from the listener, and + * that thread invokes the methods of the service (including + * subclass-specific methods), there is a risk of a deadlock.
  4. + *
+ * + * + * @param service the service that has changed. + */ + void stateChanged(Service service); + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java new file mode 100644 index 0000000000..e712aaf234 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service; + +public class ServiceUtils { + + /* + * Get the index separating the user name from domain name (the user's name up + * to the first '/' or '@'). + * + * @param userName full user name. + * @return index of domain match or -1 if not found + */ + public static int indexOfDomainMatch(String userName) { + if (userName == null) { + return -1; + } + + int idx = userName.indexOf('/'); + int idx2 = userName.indexOf('@'); + int endIdx = Math.min(idx, idx2); // Use the earlier match. + // Unless at least one of '/' or '@' was not found, in + // which case, user the latter match. + if (endIdx == -1) { + endIdx = Math.max(idx, idx2); + } + return endIdx; + } +} \ No newline at end of file diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java new file mode 100644 index 0000000000..c8f93ff6a5 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java @@ -0,0 +1,33 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.auth; + +import javax.security.sasl.AuthenticationException; + +/** + * This authentication provider allows any combination of username and password. + */ +public class AnonymousAuthenticationProviderImpl implements PasswdAuthenticationProvider { + + @Override + public void Authenticate(String user, String password) throws AuthenticationException { + // no-op authentication + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java new file mode 100644 index 0000000000..4b95503eb1 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java @@ -0,0 +1,71 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import javax.security.sasl.AuthenticationException; + +/** + * This class helps select a {@link PasswdAuthenticationProvider} for a given {@code AuthMethod}. + */ +public final class AuthenticationProviderFactory { + + public enum AuthMethods { + LDAP("LDAP"), + PAM("PAM"), + CUSTOM("CUSTOM"), + NONE("NONE"); + + private final String authMethod; + + AuthMethods(String authMethod) { + this.authMethod = authMethod; + } + + public String getAuthMethod() { + return authMethod; + } + + public static AuthMethods getValidAuthMethod(String authMethodStr) + throws AuthenticationException { + for (AuthMethods auth : AuthMethods.values()) { + if (authMethodStr.equals(auth.getAuthMethod())) { + return auth; + } + } + throw new AuthenticationException("Not a valid authentication method"); + } + } + + private AuthenticationProviderFactory() { + } + + public static PasswdAuthenticationProvider getAuthenticationProvider(AuthMethods authMethod) + throws AuthenticationException { + if (authMethod == AuthMethods.LDAP) { + return new LdapAuthenticationProviderImpl(); + } else if (authMethod == AuthMethods.PAM) { + return new PamAuthenticationProviderImpl(); + } else if (authMethod == AuthMethods.CUSTOM) { + return new CustomAuthenticationProviderImpl(); + } else if (authMethod == AuthMethods.NONE) { + return new AnonymousAuthenticationProviderImpl(); + } else { + throw new AuthenticationException("Unsupported authentication method"); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java new file mode 100644 index 0000000000..3dc0aa86e2 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java @@ -0,0 +1,50 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import javax.security.sasl.AuthenticationException; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.util.ReflectionUtils; + +/** + * This authentication provider implements the {@code CUSTOM} authentication. It allows a {@link + * PasswdAuthenticationProvider} to be specified at configuration time which may additionally + * implement {@link org.apache.hadoop.conf.Configurable Configurable} to grab Hive's {@link + * org.apache.hadoop.conf.Configuration Configuration}. + */ +public class CustomAuthenticationProviderImpl implements PasswdAuthenticationProvider { + + private final PasswdAuthenticationProvider customProvider; + + @SuppressWarnings("unchecked") + CustomAuthenticationProviderImpl() { + HiveConf conf = new HiveConf(); + Class customHandlerClass = + (Class) conf.getClass( + HiveConf.ConfVars.HIVE_SERVER2_CUSTOM_AUTHENTICATION_CLASS.varname, + PasswdAuthenticationProvider.class); + customProvider = ReflectionUtils.newInstance(customHandlerClass, conf); + } + + @Override + public void Authenticate(String user, String password) throws AuthenticationException { + customProvider.Authenticate(user, password); + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java new file mode 100644 index 0000000000..1e6ac4f3df --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java @@ -0,0 +1,364 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import javax.net.ssl.SSLServerSocket; +import javax.security.auth.login.LoginException; +import javax.security.sasl.Sasl; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.HiveMetaStore; +import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.shims.HadoopShims.KerberosNameShim; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.hive.thrift.DBTokenStore; +import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge; +import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server.ServerMode; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authorize.ProxyUsers; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.thrift.ThriftCLIService; +import org.apache.thrift.TProcessorFactory; +import org.apache.thrift.transport.TSSLTransportFactory; +import org.apache.thrift.transport.TServerSocket; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; +import org.apache.thrift.transport.TTransportFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class helps in some aspects of authentication. It creates the proper Thrift classes for the + * given configuration as well as helps with authenticating requests. + */ +public class HiveAuthFactory { + private static final Logger LOG = LoggerFactory.getLogger(HiveAuthFactory.class); + + + public enum AuthTypes { + NOSASL("NOSASL"), + NONE("NONE"), + LDAP("LDAP"), + KERBEROS("KERBEROS"), + CUSTOM("CUSTOM"), + PAM("PAM"); + + private final String authType; + + AuthTypes(String authType) { + this.authType = authType; + } + + public String getAuthName() { + return authType; + } + + } + + private HadoopThriftAuthBridge.Server saslServer; + private String authTypeStr; + private final String transportMode; + private final HiveConf conf; + + public static final String HS2_PROXY_USER = "hive.server2.proxy.user"; + public static final String HS2_CLIENT_TOKEN = "hiveserver2ClientToken"; + + public HiveAuthFactory(HiveConf conf) throws TTransportException { + this.conf = conf; + transportMode = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE); + authTypeStr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_AUTHENTICATION); + + // In http mode we use NOSASL as the default auth type + if ("http".equalsIgnoreCase(transportMode)) { + if (authTypeStr == null) { + authTypeStr = AuthTypes.NOSASL.getAuthName(); + } + } else { + if (authTypeStr == null) { + authTypeStr = AuthTypes.NONE.getAuthName(); + } + if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { + saslServer = ShimLoader.getHadoopThriftAuthBridge() + .createServer(conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB), + conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL)); + // start delegation token manager + try { + // rawStore is only necessary for DBTokenStore + Object rawStore = null; + String tokenStoreClass = conf.getVar(HiveConf.ConfVars.METASTORE_CLUSTER_DELEGATION_TOKEN_STORE_CLS); + + if (tokenStoreClass.equals(DBTokenStore.class.getName())) { + HMSHandler baseHandler = new HiveMetaStore.HMSHandler( + "new db based metaserver", conf, true); + rawStore = baseHandler.getMS(); + } + + saslServer.startDelegationTokenSecretManager(conf, rawStore, ServerMode.HIVESERVER2); + } + catch (MetaException|IOException e) { + throw new TTransportException("Failed to start token manager", e); + } + } + } + } + + public Map getSaslProperties() { + Map saslProps = new HashMap(); + SaslQOP saslQOP = SaslQOP.fromString(conf.getVar(ConfVars.HIVE_SERVER2_THRIFT_SASL_QOP)); + saslProps.put(Sasl.QOP, saslQOP.toString()); + saslProps.put(Sasl.SERVER_AUTH, "true"); + return saslProps; + } + + public TTransportFactory getAuthTransFactory() throws LoginException { + TTransportFactory transportFactory; + if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { + try { + transportFactory = saslServer.createTransportFactory(getSaslProperties()); + } catch (TTransportException e) { + throw new LoginException(e.getMessage()); + } + } else if (authTypeStr.equalsIgnoreCase(AuthTypes.NONE.getAuthName())) { + transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); + } else if (authTypeStr.equalsIgnoreCase(AuthTypes.LDAP.getAuthName())) { + transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); + } else if (authTypeStr.equalsIgnoreCase(AuthTypes.PAM.getAuthName())) { + transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); + } else if (authTypeStr.equalsIgnoreCase(AuthTypes.NOSASL.getAuthName())) { + transportFactory = new TTransportFactory(); + } else if (authTypeStr.equalsIgnoreCase(AuthTypes.CUSTOM.getAuthName())) { + transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); + } else { + throw new LoginException("Unsupported authentication type " + authTypeStr); + } + return transportFactory; + } + + /** + * Returns the thrift processor factory for HiveServer2 running in binary mode + * @param service + * @return + * @throws LoginException + */ + public TProcessorFactory getAuthProcFactory(ThriftCLIService service) throws LoginException { + if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { + return KerberosSaslHelper.getKerberosProcessorFactory(saslServer, service); + } else { + return PlainSaslHelper.getPlainProcessorFactory(service); + } + } + + public String getRemoteUser() { + return saslServer == null ? null : saslServer.getRemoteUser(); + } + + public String getIpAddress() { + if (saslServer == null || saslServer.getRemoteAddress() == null) { + return null; + } else { + return saslServer.getRemoteAddress().getHostAddress(); + } + } + + // Perform kerberos login using the hadoop shim API if the configuration is available + public static void loginFromKeytab(HiveConf hiveConf) throws IOException { + String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL); + String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB); + if (principal.isEmpty() || keyTabFile.isEmpty()) { + throw new IOException("HiveServer2 Kerberos principal or keytab is not correctly configured"); + } else { + UserGroupInformation.loginUserFromKeytab(SecurityUtil.getServerPrincipal(principal, "0.0.0.0"), keyTabFile); + } + } + + // Perform SPNEGO login using the hadoop shim API if the configuration is available + public static UserGroupInformation loginFromSpnegoKeytabAndReturnUGI(HiveConf hiveConf) + throws IOException { + String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_PRINCIPAL); + String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_KEYTAB); + if (principal.isEmpty() || keyTabFile.isEmpty()) { + throw new IOException("HiveServer2 SPNEGO principal or keytab is not correctly configured"); + } else { + return UserGroupInformation.loginUserFromKeytabAndReturnUGI(SecurityUtil.getServerPrincipal(principal, "0.0.0.0"), keyTabFile); + } + } + + public static TTransport getSocketTransport(String host, int port, int loginTimeout) { + return new TSocket(host, port, loginTimeout); + } + + public static TTransport getSSLSocket(String host, int port, int loginTimeout) + throws TTransportException { + return TSSLTransportFactory.getClientSocket(host, port, loginTimeout); + } + + public static TTransport getSSLSocket(String host, int port, int loginTimeout, + String trustStorePath, String trustStorePassWord) throws TTransportException { + TSSLTransportFactory.TSSLTransportParameters params = + new TSSLTransportFactory.TSSLTransportParameters(); + params.setTrustStore(trustStorePath, trustStorePassWord); + params.requireClientAuth(true); + return TSSLTransportFactory.getClientSocket(host, port, loginTimeout, params); + } + + public static TServerSocket getServerSocket(String hiveHost, int portNum) + throws TTransportException { + InetSocketAddress serverAddress; + if (hiveHost == null || hiveHost.isEmpty()) { + // Wildcard bind + serverAddress = new InetSocketAddress(portNum); + } else { + serverAddress = new InetSocketAddress(hiveHost, portNum); + } + return new TServerSocket(serverAddress); + } + + public static TServerSocket getServerSSLSocket(String hiveHost, int portNum, String keyStorePath, + String keyStorePassWord, List sslVersionBlacklist) throws TTransportException, + UnknownHostException { + TSSLTransportFactory.TSSLTransportParameters params = + new TSSLTransportFactory.TSSLTransportParameters(); + params.setKeyStore(keyStorePath, keyStorePassWord); + InetSocketAddress serverAddress; + if (hiveHost == null || hiveHost.isEmpty()) { + // Wildcard bind + serverAddress = new InetSocketAddress(portNum); + } else { + serverAddress = new InetSocketAddress(hiveHost, portNum); + } + TServerSocket thriftServerSocket = + TSSLTransportFactory.getServerSocket(portNum, 0, serverAddress.getAddress(), params); + if (thriftServerSocket.getServerSocket() instanceof SSLServerSocket) { + List sslVersionBlacklistLocal = new ArrayList(); + for (String sslVersion : sslVersionBlacklist) { + sslVersionBlacklistLocal.add(sslVersion.trim().toLowerCase()); + } + SSLServerSocket sslServerSocket = (SSLServerSocket) thriftServerSocket.getServerSocket(); + List enabledProtocols = new ArrayList(); + for (String protocol : sslServerSocket.getEnabledProtocols()) { + if (sslVersionBlacklistLocal.contains(protocol.toLowerCase())) { + LOG.debug("Disabling SSL Protocol: " + protocol); + } else { + enabledProtocols.add(protocol); + } + } + sslServerSocket.setEnabledProtocols(enabledProtocols.toArray(new String[0])); + LOG.info("SSL Server Socket Enabled Protocols: " + + Arrays.toString(sslServerSocket.getEnabledProtocols())); + } + return thriftServerSocket; + } + + // retrieve delegation token for the given user + public String getDelegationToken(String owner, String renewer) throws HiveSQLException { + if (saslServer == null) { + throw new HiveSQLException( + "Delegation token only supported over kerberos authentication", "08S01"); + } + + try { + String tokenStr = saslServer.getDelegationTokenWithService(owner, renewer, HS2_CLIENT_TOKEN); + if (tokenStr == null || tokenStr.isEmpty()) { + throw new HiveSQLException( + "Received empty retrieving delegation token for user " + owner, "08S01"); + } + return tokenStr; + } catch (IOException e) { + throw new HiveSQLException( + "Error retrieving delegation token for user " + owner, "08S01", e); + } catch (InterruptedException e) { + throw new HiveSQLException("delegation token retrieval interrupted", "08S01", e); + } + } + + // cancel given delegation token + public void cancelDelegationToken(String delegationToken) throws HiveSQLException { + if (saslServer == null) { + throw new HiveSQLException( + "Delegation token only supported over kerberos authentication", "08S01"); + } + try { + saslServer.cancelDelegationToken(delegationToken); + } catch (IOException e) { + throw new HiveSQLException( + "Error canceling delegation token " + delegationToken, "08S01", e); + } + } + + public void renewDelegationToken(String delegationToken) throws HiveSQLException { + if (saslServer == null) { + throw new HiveSQLException( + "Delegation token only supported over kerberos authentication", "08S01"); + } + try { + saslServer.renewDelegationToken(delegationToken); + } catch (IOException e) { + throw new HiveSQLException( + "Error renewing delegation token " + delegationToken, "08S01", e); + } + } + + public String getUserFromToken(String delegationToken) throws HiveSQLException { + if (saslServer == null) { + throw new HiveSQLException( + "Delegation token only supported over kerberos authentication", "08S01"); + } + try { + return saslServer.getUserFromToken(delegationToken); + } catch (IOException e) { + throw new HiveSQLException( + "Error extracting user from delegation token " + delegationToken, "08S01", e); + } + } + + public static void verifyProxyAccess(String realUser, String proxyUser, String ipAddress, + HiveConf hiveConf) throws HiveSQLException { + try { + UserGroupInformation sessionUgi; + if (UserGroupInformation.isSecurityEnabled()) { + KerberosNameShim kerbName = ShimLoader.getHadoopShims().getKerberosNameShim(realUser); + sessionUgi = UserGroupInformation.createProxyUser( + kerbName.getServiceName(), UserGroupInformation.getLoginUser()); + } else { + sessionUgi = UserGroupInformation.createRemoteUser(realUser); + } + if (!proxyUser.equalsIgnoreCase(realUser)) { + ProxyUsers.refreshSuperUserGroupsConfiguration(hiveConf); + ProxyUsers.authorize(UserGroupInformation.createProxyUser(proxyUser, sessionUgi), + ipAddress, hiveConf); + } + } catch (IOException e) { + throw new HiveSQLException( + "Failed to validate proxy privilege of " + realUser + " for " + proxyUser, "08S01", e); + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java new file mode 100644 index 0000000000..3ef55779a6 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java @@ -0,0 +1,189 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.auth; + +import java.security.AccessControlContext; +import java.security.AccessController; +import java.security.PrivilegedExceptionAction; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.StringTokenizer; + +import javax.security.auth.Subject; + +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.http.protocol.BasicHttpContext; +import org.apache.http.protocol.HttpContext; +import org.ietf.jgss.GSSContext; +import org.ietf.jgss.GSSManager; +import org.ietf.jgss.GSSName; +import org.ietf.jgss.Oid; + +/** + * Utility functions for HTTP mode authentication. + */ +public final class HttpAuthUtils { + public static final String WWW_AUTHENTICATE = "WWW-Authenticate"; + public static final String AUTHORIZATION = "Authorization"; + public static final String BASIC = "Basic"; + public static final String NEGOTIATE = "Negotiate"; + private static final Log LOG = LogFactory.getLog(HttpAuthUtils.class); + private static final String COOKIE_ATTR_SEPARATOR = "&"; + private static final String COOKIE_CLIENT_USER_NAME = "cu"; + private static final String COOKIE_CLIENT_RAND_NUMBER = "rn"; + private static final String COOKIE_KEY_VALUE_SEPARATOR = "="; + private final static Set COOKIE_ATTRIBUTES = + new HashSet(Arrays.asList(COOKIE_CLIENT_USER_NAME, COOKIE_CLIENT_RAND_NUMBER)); + + /** + * @return Stringified Base64 encoded kerberosAuthHeader on success + * @throws Exception + */ + public static String getKerberosServiceTicket(String principal, String host, + String serverHttpUrl, boolean assumeSubject) throws Exception { + String serverPrincipal = + ShimLoader.getHadoopThriftAuthBridge().getServerPrincipal(principal, host); + if (assumeSubject) { + // With this option, we're assuming that the external application, + // using the JDBC driver has done a JAAS kerberos login already + AccessControlContext context = AccessController.getContext(); + Subject subject = Subject.getSubject(context); + if (subject == null) { + throw new Exception("The Subject is not set"); + } + return Subject.doAs(subject, new HttpKerberosClientAction(serverPrincipal, serverHttpUrl)); + } else { + // JAAS login from ticket cache to setup the client UserGroupInformation + UserGroupInformation clientUGI = + ShimLoader.getHadoopThriftAuthBridge().getCurrentUGIWithConf("kerberos"); + return clientUGI.doAs(new HttpKerberosClientAction(serverPrincipal, serverHttpUrl)); + } + } + + /** + * Creates and returns a HS2 cookie token. + * @param clientUserName Client User name. + * @return An unsigned cookie token generated from input parameters. + * The final cookie generated is of the following format : + * cu=&rn=&s= + */ + public static String createCookieToken(String clientUserName) { + StringBuffer sb = new StringBuffer(); + sb.append(COOKIE_CLIENT_USER_NAME).append(COOKIE_KEY_VALUE_SEPARATOR).append(clientUserName). + append(COOKIE_ATTR_SEPARATOR); + sb.append(COOKIE_CLIENT_RAND_NUMBER).append(COOKIE_KEY_VALUE_SEPARATOR). + append((new Random(System.currentTimeMillis())).nextLong()); + return sb.toString(); + } + + /** + * Parses a cookie token to retrieve client user name. + * @param tokenStr Token String. + * @return A valid user name if input is of valid format, else returns null. + */ + public static String getUserNameFromCookieToken(String tokenStr) { + Map map = splitCookieToken(tokenStr); + + if (!map.keySet().equals(COOKIE_ATTRIBUTES)) { + LOG.error("Invalid token with missing attributes " + tokenStr); + return null; + } + return map.get(COOKIE_CLIENT_USER_NAME); + } + + /** + * Splits the cookie token into attributes pairs. + * @param str input token. + * @return a map with the attribute pairs of the token if the input is valid. + * Else, returns null. + */ + private static Map splitCookieToken(String tokenStr) { + Map map = new HashMap(); + StringTokenizer st = new StringTokenizer(tokenStr, COOKIE_ATTR_SEPARATOR); + + while (st.hasMoreTokens()) { + String part = st.nextToken(); + int separator = part.indexOf(COOKIE_KEY_VALUE_SEPARATOR); + if (separator == -1) { + LOG.error("Invalid token string " + tokenStr); + return null; + } + String key = part.substring(0, separator); + String value = part.substring(separator + 1); + map.put(key, value); + } + return map; + } + + + private HttpAuthUtils() { + throw new UnsupportedOperationException("Can't initialize class"); + } + + /** + * We'll create an instance of this class within a doAs block so that the client's TGT credentials + * can be read from the Subject + */ + public static class HttpKerberosClientAction implements PrivilegedExceptionAction { + public static final String HTTP_RESPONSE = "HTTP_RESPONSE"; + public static final String SERVER_HTTP_URL = "SERVER_HTTP_URL"; + private final String serverPrincipal; + private final String serverHttpUrl; + private final Base64 base64codec; + private final HttpContext httpContext; + + public HttpKerberosClientAction(String serverPrincipal, String serverHttpUrl) { + this.serverPrincipal = serverPrincipal; + this.serverHttpUrl = serverHttpUrl; + base64codec = new Base64(0); + httpContext = new BasicHttpContext(); + httpContext.setAttribute(SERVER_HTTP_URL, serverHttpUrl); + } + + @Override + public String run() throws Exception { + // This Oid for Kerberos GSS-API mechanism. + Oid mechOid = new Oid("1.2.840.113554.1.2.2"); + // Oid for kerberos principal name + Oid krb5PrincipalOid = new Oid("1.2.840.113554.1.2.2.1"); + GSSManager manager = GSSManager.getInstance(); + // GSS name for server + GSSName serverName = manager.createName(serverPrincipal, krb5PrincipalOid); + // Create a GSSContext for authentication with the service. + // We're passing client credentials as null since we want them to be read from the Subject. + GSSContext gssContext = + manager.createContext(serverName, mechOid, null, GSSContext.DEFAULT_LIFETIME); + gssContext.requestMutualAuth(false); + // Establish context + byte[] inToken = new byte[0]; + byte[] outToken = gssContext.initSecContext(inToken, 0, inToken.length); + gssContext.dispose(); + // Base64 encoded and stringified token for server + return new String(base64codec.encode(outToken)); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthenticationException.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthenticationException.java new file mode 100644 index 0000000000..5764325602 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthenticationException.java @@ -0,0 +1,43 @@ +/** + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. See accompanying LICENSE file. + */ + +package org.apache.hive.service.auth; + +public class HttpAuthenticationException extends Exception { + + private static final long serialVersionUID = 0; + + /** + * @param cause original exception + */ + public HttpAuthenticationException(Throwable cause) { + super(cause); + } + + /** + * @param msg exception message + */ + public HttpAuthenticationException(String msg) { + super(msg); + } + + /** + * @param msg exception message + * @param cause original exception + */ + public HttpAuthenticationException(String msg, Throwable cause) { + super(msg, cause); + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java new file mode 100644 index 0000000000..11d26699fe --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java @@ -0,0 +1,111 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import java.io.IOException; +import java.util.Map; +import javax.security.sasl.SaslException; + +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge; +import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server; +import org.apache.hive.service.cli.thrift.TCLIService; +import org.apache.hive.service.cli.thrift.TCLIService.Iface; +import org.apache.hive.service.cli.thrift.ThriftCLIService; +import org.apache.thrift.TProcessor; +import org.apache.thrift.TProcessorFactory; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TTransport; + +public final class KerberosSaslHelper { + + public static TProcessorFactory getKerberosProcessorFactory(Server saslServer, + ThriftCLIService service) { + return new CLIServiceProcessorFactory(saslServer, service); + } + + public static TTransport getKerberosTransport(String principal, String host, + TTransport underlyingTransport, Map saslProps, boolean assumeSubject) + throws SaslException { + try { + String[] names = principal.split("[/@]"); + if (names.length != 3) { + throw new IllegalArgumentException("Kerberos principal should have 3 parts: " + principal); + } + + if (assumeSubject) { + return createSubjectAssumedTransport(principal, underlyingTransport, saslProps); + } else { + HadoopThriftAuthBridge.Client authBridge = + ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos"); + return authBridge.createClientTransport(principal, host, "KERBEROS", null, + underlyingTransport, saslProps); + } + } catch (IOException e) { + throw new SaslException("Failed to open client transport", e); + } + } + + public static TTransport createSubjectAssumedTransport(String principal, + TTransport underlyingTransport, Map saslProps) throws IOException { + String[] names = principal.split("[/@]"); + try { + TTransport saslTransport = + new TSaslClientTransport("GSSAPI", null, names[0], names[1], saslProps, null, + underlyingTransport); + return new TSubjectAssumingTransport(saslTransport); + } catch (SaslException se) { + throw new IOException("Could not instantiate SASL transport", se); + } + } + + public static TTransport getTokenTransport(String tokenStr, String host, + TTransport underlyingTransport, Map saslProps) throws SaslException { + HadoopThriftAuthBridge.Client authBridge = + ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos"); + + try { + return authBridge.createClientTransport(null, host, "DIGEST", tokenStr, underlyingTransport, + saslProps); + } catch (IOException e) { + throw new SaslException("Failed to open client transport", e); + } + } + + private KerberosSaslHelper() { + throw new UnsupportedOperationException("Can't initialize class"); + } + + private static class CLIServiceProcessorFactory extends TProcessorFactory { + + private final ThriftCLIService service; + private final Server saslServer; + + public CLIServiceProcessorFactory(Server saslServer, ThriftCLIService service) { + super(null); + this.service = service; + this.saslServer = saslServer; + } + + @Override + public TProcessor getProcessor(TTransport trans) { + TProcessor sqlProcessor = new TCLIService.Processor(service); + return saslServer.wrapNonAssumingProcessor(sqlProcessor); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java new file mode 100644 index 0000000000..4e2ef90a1e --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java @@ -0,0 +1,84 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import java.util.Hashtable; +import javax.naming.Context; +import javax.naming.NamingException; +import javax.naming.directory.InitialDirContext; +import javax.security.sasl.AuthenticationException; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hive.service.ServiceUtils; + +public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvider { + + private final String ldapURL; + private final String baseDN; + private final String ldapDomain; + + LdapAuthenticationProviderImpl() { + HiveConf conf = new HiveConf(); + ldapURL = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_URL); + baseDN = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN); + ldapDomain = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_DOMAIN); + } + + @Override + public void Authenticate(String user, String password) throws AuthenticationException { + + Hashtable env = new Hashtable(); + env.put(Context.INITIAL_CONTEXT_FACTORY, "com.sun.jndi.ldap.LdapCtxFactory"); + env.put(Context.PROVIDER_URL, ldapURL); + + // If the domain is available in the config, then append it unless domain is + // already part of the username. LDAP providers like Active Directory use a + // fully qualified user name like foo@bar.com. + if (!hasDomain(user) && ldapDomain != null) { + user = user + "@" + ldapDomain; + } + + if (password == null || password.isEmpty() || password.getBytes()[0] == 0) { + throw new AuthenticationException("Error validating LDAP user:" + + " a null or blank password has been provided"); + } + + // setup the security principal + String bindDN; + if (baseDN == null) { + bindDN = user; + } else { + bindDN = "uid=" + user + "," + baseDN; + } + env.put(Context.SECURITY_AUTHENTICATION, "simple"); + env.put(Context.SECURITY_PRINCIPAL, bindDN); + env.put(Context.SECURITY_CREDENTIALS, password); + + try { + // Create initial context + Context ctx = new InitialDirContext(env); + ctx.close(); + } catch (NamingException e) { + throw new AuthenticationException("Error validating LDAP user", e); + } + } + + private boolean hasDomain(String userName) { + return (ServiceUtils.indexOfDomainMatch(userName) > 0); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PamAuthenticationProviderImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PamAuthenticationProviderImpl.java new file mode 100644 index 0000000000..68f62c4617 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PamAuthenticationProviderImpl.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import javax.security.sasl.AuthenticationException; + +import net.sf.jpam.Pam; +import org.apache.hadoop.hive.conf.HiveConf; + +public class PamAuthenticationProviderImpl implements PasswdAuthenticationProvider { + + private final String pamServiceNames; + + PamAuthenticationProviderImpl() { + HiveConf conf = new HiveConf(); + pamServiceNames = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PAM_SERVICES); + } + + @Override + public void Authenticate(String user, String password) throws AuthenticationException { + + if (pamServiceNames == null || pamServiceNames.trim().isEmpty()) { + throw new AuthenticationException("No PAM services are set."); + } + + String[] pamServices = pamServiceNames.split(","); + for (String pamService : pamServices) { + Pam pam = new Pam(pamService); + boolean isAuthenticated = pam.authenticateSuccessful(user, password); + if (!isAuthenticated) { + throw new AuthenticationException( + "Error authenticating with the PAM service: " + pamService); + } + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java new file mode 100644 index 0000000000..e2a6de165a --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import javax.security.sasl.AuthenticationException; + +public interface PasswdAuthenticationProvider { + + /** + * The Authenticate method is called by the HiveServer2 authentication layer + * to authenticate users for their requests. + * If a user is to be granted, return nothing/throw nothing. + * When a user is to be disallowed, throw an appropriate {@link AuthenticationException}. + *

+ * For an example implementation, see {@link LdapAuthenticationProviderImpl}. + * + * @param user The username received over the connection request + * @param password The password received over the connection request + * + * @throws AuthenticationException When a user is found to be + * invalid by the implementation + */ + void Authenticate(String user, String password) throws AuthenticationException; +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java new file mode 100644 index 0000000000..afc144199f --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java @@ -0,0 +1,154 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import java.io.IOException; +import java.security.Security; +import java.util.HashMap; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.auth.login.LoginException; +import javax.security.sasl.AuthenticationException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.SaslException; + +import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods; +import org.apache.hive.service.auth.PlainSaslServer.SaslPlainProvider; +import org.apache.hive.service.cli.thrift.TCLIService.Iface; +import org.apache.hive.service.cli.thrift.ThriftCLIService; +import org.apache.thrift.TProcessor; +import org.apache.thrift.TProcessorFactory; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TSaslServerTransport; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportFactory; + +public final class PlainSaslHelper { + + public static TProcessorFactory getPlainProcessorFactory(ThriftCLIService service) { + return new SQLPlainProcessorFactory(service); + } + + // Register Plain SASL server provider + static { + Security.addProvider(new SaslPlainProvider()); + } + + public static TTransportFactory getPlainTransportFactory(String authTypeStr) + throws LoginException { + TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory(); + try { + saslFactory.addServerDefinition("PLAIN", authTypeStr, null, new HashMap(), + new PlainServerCallbackHandler(authTypeStr)); + } catch (AuthenticationException e) { + throw new LoginException("Error setting callback handler" + e); + } + return saslFactory; + } + + public static TTransport getPlainTransport(String username, String password, + TTransport underlyingTransport) throws SaslException { + return new TSaslClientTransport("PLAIN", null, null, null, new HashMap(), + new PlainCallbackHandler(username, password), underlyingTransport); + } + + private PlainSaslHelper() { + throw new UnsupportedOperationException("Can't initialize class"); + } + + private static final class PlainServerCallbackHandler implements CallbackHandler { + + private final AuthMethods authMethod; + + PlainServerCallbackHandler(String authMethodStr) throws AuthenticationException { + authMethod = AuthMethods.getValidAuthMethod(authMethodStr); + } + + @Override + public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { + String username = null; + String password = null; + AuthorizeCallback ac = null; + + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + NameCallback nc = (NameCallback) callback; + username = nc.getName(); + } else if (callback instanceof PasswordCallback) { + PasswordCallback pc = (PasswordCallback) callback; + password = new String(pc.getPassword()); + } else if (callback instanceof AuthorizeCallback) { + ac = (AuthorizeCallback) callback; + } else { + throw new UnsupportedCallbackException(callback); + } + } + PasswdAuthenticationProvider provider = + AuthenticationProviderFactory.getAuthenticationProvider(authMethod); + provider.Authenticate(username, password); + if (ac != null) { + ac.setAuthorized(true); + } + } + } + + public static class PlainCallbackHandler implements CallbackHandler { + + private final String username; + private final String password; + + public PlainCallbackHandler(String username, String password) { + this.username = username; + this.password = password; + } + + @Override + public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + NameCallback nameCallback = (NameCallback) callback; + nameCallback.setName(username); + } else if (callback instanceof PasswordCallback) { + PasswordCallback passCallback = (PasswordCallback) callback; + passCallback.setPassword(password.toCharArray()); + } else { + throw new UnsupportedCallbackException(callback); + } + } + } + } + + private static final class SQLPlainProcessorFactory extends TProcessorFactory { + + private final ThriftCLIService service; + + SQLPlainProcessorFactory(ThriftCLIService service) { + super(null); + this.service = service; + } + + @Override + public TProcessor getProcessor(TTransport trans) { + return new TSetIpAddressProcessor(service); + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslServer.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslServer.java new file mode 100644 index 0000000000..cd675da29a --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslServer.java @@ -0,0 +1,177 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import java.io.IOException; +import java.security.Provider; +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.Map; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.SaslException; +import javax.security.sasl.SaslServer; +import javax.security.sasl.SaslServerFactory; + +import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods; + +/** + * Sun JDK only provides a PLAIN client and no server. This class implements the Plain SASL server + * conforming to RFC #4616 (http://www.ietf.org/rfc/rfc4616.txt). + */ +public class PlainSaslServer implements SaslServer { + + public static final String PLAIN_METHOD = "PLAIN"; + private String user; + private final CallbackHandler handler; + + PlainSaslServer(CallbackHandler handler, String authMethodStr) throws SaslException { + this.handler = handler; + AuthMethods.getValidAuthMethod(authMethodStr); + } + + @Override + public String getMechanismName() { + return PLAIN_METHOD; + } + + @Override + public byte[] evaluateResponse(byte[] response) throws SaslException { + try { + // parse the response + // message = [authzid] UTF8NUL authcid UTF8NUL passwd' + + Deque tokenList = new ArrayDeque(); + StringBuilder messageToken = new StringBuilder(); + for (byte b : response) { + if (b == 0) { + tokenList.addLast(messageToken.toString()); + messageToken = new StringBuilder(); + } else { + messageToken.append((char) b); + } + } + tokenList.addLast(messageToken.toString()); + + // validate response + if (tokenList.size() < 2 || tokenList.size() > 3) { + throw new SaslException("Invalid message format"); + } + String passwd = tokenList.removeLast(); + user = tokenList.removeLast(); + // optional authzid + String authzId; + if (tokenList.isEmpty()) { + authzId = user; + } else { + authzId = tokenList.removeLast(); + } + if (user == null || user.isEmpty()) { + throw new SaslException("No user name provided"); + } + if (passwd == null || passwd.isEmpty()) { + throw new SaslException("No password name provided"); + } + + NameCallback nameCallback = new NameCallback("User"); + nameCallback.setName(user); + PasswordCallback pcCallback = new PasswordCallback("Password", false); + pcCallback.setPassword(passwd.toCharArray()); + AuthorizeCallback acCallback = new AuthorizeCallback(user, authzId); + + Callback[] cbList = {nameCallback, pcCallback, acCallback}; + handler.handle(cbList); + if (!acCallback.isAuthorized()) { + throw new SaslException("Authentication failed"); + } + } catch (IllegalStateException eL) { + throw new SaslException("Invalid message format", eL); + } catch (IOException eI) { + throw new SaslException("Error validating the login", eI); + } catch (UnsupportedCallbackException eU) { + throw new SaslException("Error validating the login", eU); + } + return null; + } + + @Override + public boolean isComplete() { + return user != null; + } + + @Override + public String getAuthorizationID() { + return user; + } + + @Override + public byte[] unwrap(byte[] incoming, int offset, int len) { + throw new UnsupportedOperationException(); + } + + @Override + public byte[] wrap(byte[] outgoing, int offset, int len) { + throw new UnsupportedOperationException(); + } + + @Override + public Object getNegotiatedProperty(String propName) { + return null; + } + + @Override + public void dispose() {} + + public static class SaslPlainServerFactory implements SaslServerFactory { + + @Override + public SaslServer createSaslServer(String mechanism, String protocol, String serverName, + Map props, CallbackHandler cbh) { + if (PLAIN_METHOD.equals(mechanism)) { + try { + return new PlainSaslServer(cbh, protocol); + } catch (SaslException e) { + /* This is to fulfill the contract of the interface which states that an exception shall + be thrown when a SaslServer cannot be created due to an error but null should be + returned when a Server can't be created due to the parameters supplied. And the only + thing PlainSaslServer can fail on is a non-supported authentication mechanism. + That's why we return null instead of throwing the Exception */ + return null; + } + } + return null; + } + + @Override + public String[] getMechanismNames(Map props) { + return new String[] {PLAIN_METHOD}; + } + } + + public static class SaslPlainProvider extends Provider { + + public SaslPlainProvider() { + super("HiveSaslPlain", 1.0, "Hive Plain SASL provider"); + put("SaslServerFactory.PLAIN", SaslPlainServerFactory.class.getName()); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java new file mode 100644 index 0000000000..479ebf32ce --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.auth; + +import java.util.HashMap; +import java.util.Map; + +/** + * Possible values of SASL quality-of-protection value. + */ +public enum SaslQOP { + AUTH("auth"), // Authentication only. + AUTH_INT("auth-int"), // Authentication and integrity checking by using signatures. + AUTH_CONF("auth-conf"); // Authentication, integrity and confidentiality checking + // by using signatures and encryption. + + public final String saslQop; + + private static final Map STR_TO_ENUM = new HashMap(); + + static { + for (SaslQOP saslQop : values()) { + STR_TO_ENUM.put(saslQop.toString(), saslQop); + } + } + + SaslQOP(String saslQop) { + this.saslQop = saslQop; + } + + public String toString() { + return saslQop; + } + + public static SaslQOP fromString(String str) { + if (str != null) { + str = str.toLowerCase(); + } + SaslQOP saslQOP = STR_TO_ENUM.get(str); + if (saslQOP == null) { + throw new IllegalArgumentException( + "Unknown auth type: " + str + " Allowed values are: " + STR_TO_ENUM.keySet()); + } + return saslQOP; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java new file mode 100644 index 0000000000..645e3e2bbd --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java @@ -0,0 +1,117 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.auth; + +import org.apache.hive.service.cli.thrift.TCLIService; +import org.apache.hive.service.cli.thrift.TCLIService.Iface; +import org.apache.thrift.TException; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TSaslServerTransport; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class is responsible for setting the ipAddress for operations executed via HiveServer2. + *

+ *

    + *
  • IP address is only set for operations that calls listeners with hookContext
  • + *
  • IP address is only set if the underlying transport mechanism is socket
  • + *
+ *

+ * + * @see org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext + */ +public class TSetIpAddressProcessor extends TCLIService.Processor { + + private static final Logger LOGGER = LoggerFactory.getLogger(TSetIpAddressProcessor.class.getName()); + + public TSetIpAddressProcessor(Iface iface) { + super(iface); + } + + @Override + public boolean process(final TProtocol in, final TProtocol out) throws TException { + setIpAddress(in); + setUserName(in); + try { + return super.process(in, out); + } finally { + THREAD_LOCAL_USER_NAME.remove(); + THREAD_LOCAL_IP_ADDRESS.remove(); + } + } + + private void setUserName(final TProtocol in) { + TTransport transport = in.getTransport(); + if (transport instanceof TSaslServerTransport) { + String userName = ((TSaslServerTransport) transport).getSaslServer().getAuthorizationID(); + THREAD_LOCAL_USER_NAME.set(userName); + } + } + + protected void setIpAddress(final TProtocol in) { + TTransport transport = in.getTransport(); + TSocket tSocket = getUnderlyingSocketFromTransport(transport); + if (tSocket == null) { + LOGGER.warn("Unknown Transport, cannot determine ipAddress"); + } else { + THREAD_LOCAL_IP_ADDRESS.set(tSocket.getSocket().getInetAddress().getHostAddress()); + } + } + + private TSocket getUnderlyingSocketFromTransport(TTransport transport) { + while (transport != null) { + if (transport instanceof TSaslServerTransport) { + transport = ((TSaslServerTransport) transport).getUnderlyingTransport(); + } + if (transport instanceof TSaslClientTransport) { + transport = ((TSaslClientTransport) transport).getUnderlyingTransport(); + } + if (transport instanceof TSocket) { + return (TSocket) transport; + } + } + return null; + } + + private static final ThreadLocal THREAD_LOCAL_IP_ADDRESS = new ThreadLocal() { + @Override + protected synchronized String initialValue() { + return null; + } + }; + + private static final ThreadLocal THREAD_LOCAL_USER_NAME = new ThreadLocal() { + @Override + protected synchronized String initialValue() { + return null; + } + }; + + public static String getUserIpAddress() { + return THREAD_LOCAL_IP_ADDRESS.get(); + } + + public static String getUserName() { + return THREAD_LOCAL_USER_NAME.get(); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSubjectAssumingTransport.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSubjectAssumingTransport.java new file mode 100644 index 0000000000..2422e86c6b --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSubjectAssumingTransport.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.auth; + +import java.security.AccessControlContext; +import java.security.AccessController; +import java.security.PrivilegedActionException; +import java.security.PrivilegedExceptionAction; +import javax.security.auth.Subject; + +import org.apache.hadoop.hive.thrift.TFilterTransport; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; + +/** + * This is used on the client side, where the API explicitly opens a transport to + * the server using the Subject.doAs(). + */ +public class TSubjectAssumingTransport extends TFilterTransport { + + public TSubjectAssumingTransport(TTransport wrapped) { + super(wrapped); + } + + @Override + public void open() throws TTransportException { + try { + AccessControlContext context = AccessController.getContext(); + Subject subject = Subject.getSubject(context); + Subject.doAs(subject, new PrivilegedExceptionAction() { + public Void run() { + try { + wrapped.open(); + } catch (TTransportException tte) { + // Wrap the transport exception in an RTE, since Subject.doAs() then goes + // and unwraps this for us out of the doAs block. We then unwrap one + // more time in our catch clause to get back the TTE. (ugh) + throw new RuntimeException(tte); + } + return null; + } + }); + } catch (PrivilegedActionException ioe) { + throw new RuntimeException("Received an ioe we never threw!", ioe); + } catch (RuntimeException rte) { + if (rte.getCause() instanceof TTransportException) { + throw (TTransportException) rte.getCause(); + } else { + throw rte; + } + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java new file mode 100644 index 0000000000..a3af7b2d66 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java @@ -0,0 +1,508 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import javax.security.auth.login.LoginException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.shims.Utils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.service.CompositeService; +import org.apache.hive.service.ServiceException; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.operation.Operation; +import org.apache.hive.service.cli.session.SessionManager; +import org.apache.hive.service.cli.thrift.TProtocolVersion; +import org.apache.hive.service.server.HiveServer2; + +/** + * CLIService. + * + */ +public class CLIService extends CompositeService implements ICLIService { + + public static final TProtocolVersion SERVER_VERSION; + + static { + TProtocolVersion[] protocols = TProtocolVersion.values(); + SERVER_VERSION = protocols[protocols.length - 1]; + } + + private final Log LOG = LogFactory.getLog(CLIService.class.getName()); + + private HiveConf hiveConf; + private SessionManager sessionManager; + private UserGroupInformation serviceUGI; + private UserGroupInformation httpUGI; + // The HiveServer2 instance running this service + private final HiveServer2 hiveServer2; + + public CLIService(HiveServer2 hiveServer2) { + super(CLIService.class.getSimpleName()); + this.hiveServer2 = hiveServer2; + } + + @Override + public synchronized void init(HiveConf hiveConf) { + this.hiveConf = hiveConf; + sessionManager = new SessionManager(hiveServer2); + addService(sessionManager); + // If the hadoop cluster is secure, do a kerberos login for the service from the keytab + if (UserGroupInformation.isSecurityEnabled()) { + try { + HiveAuthFactory.loginFromKeytab(hiveConf); + this.serviceUGI = Utils.getUGI(); + } catch (IOException e) { + throw new ServiceException("Unable to login to kerberos with given principal/keytab", e); + } catch (LoginException e) { + throw new ServiceException("Unable to login to kerberos with given principal/keytab", e); + } + + // Also try creating a UGI object for the SPNego principal + String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_PRINCIPAL); + String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_KEYTAB); + if (principal.isEmpty() || keyTabFile.isEmpty()) { + LOG.info("SPNego httpUGI not created, spNegoPrincipal: " + principal + + ", ketabFile: " + keyTabFile); + } else { + try { + this.httpUGI = HiveAuthFactory.loginFromSpnegoKeytabAndReturnUGI(hiveConf); + LOG.info("SPNego httpUGI successfully created."); + } catch (IOException e) { + LOG.warn("SPNego httpUGI creation failed: ", e); + } + } + } + // creates connection to HMS and thus *must* occur after kerberos login above + try { + applyAuthorizationConfigPolicy(hiveConf); + } catch (Exception e) { + throw new RuntimeException("Error applying authorization policy on hive configuration: " + + e.getMessage(), e); + } + setupBlockedUdfs(); + super.init(hiveConf); + } + + private void applyAuthorizationConfigPolicy(HiveConf newHiveConf) throws HiveException, + MetaException { + // authorization setup using SessionState should be revisited eventually, as + // authorization and authentication are not session specific settings + SessionState ss = new SessionState(newHiveConf); + ss.setIsHiveServerQuery(true); + SessionState.start(ss); + ss.applyAuthorizationPolicy(); + } + + private void setupBlockedUdfs() { + FunctionRegistry.setupPermissionsForBuiltinUDFs( + hiveConf.getVar(ConfVars.HIVE_SERVER2_BUILTIN_UDF_WHITELIST), + hiveConf.getVar(ConfVars.HIVE_SERVER2_BUILTIN_UDF_BLACKLIST)); + } + + public UserGroupInformation getServiceUGI() { + return this.serviceUGI; + } + + public UserGroupInformation getHttpUGI() { + return this.httpUGI; + } + + @Override + public synchronized void start() { + super.start(); + // Initialize and test a connection to the metastore + IMetaStoreClient metastoreClient = null; + try { + metastoreClient = new HiveMetaStoreClient(hiveConf); + metastoreClient.getDatabases("default"); + } catch (Exception e) { + throw new ServiceException("Unable to connect to MetaStore!", e); + } + finally { + if (metastoreClient != null) { + metastoreClient.close(); + } + } + } + + @Override + public synchronized void stop() { + super.stop(); + } + + /** + * @deprecated Use {@link #openSession(TProtocolVersion, String, String, String, Map)} + */ + @Deprecated + public SessionHandle openSession(TProtocolVersion protocol, String username, String password, + Map configuration) throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, null, configuration, false, null); + LOG.debug(sessionHandle + ": openSession()"); + return sessionHandle; + } + + /** + * @deprecated Use {@link #openSessionWithImpersonation(TProtocolVersion, String, String, String, Map, String)} + */ + @Deprecated + public SessionHandle openSessionWithImpersonation(TProtocolVersion protocol, String username, + String password, Map configuration, String delegationToken) + throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, null, configuration, + true, delegationToken); + LOG.debug(sessionHandle + ": openSessionWithImpersonation()"); + return sessionHandle; + } + + public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress, + Map configuration) throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, ipAddress, configuration, false, null); + LOG.debug(sessionHandle + ": openSession()"); + return sessionHandle; + } + + public SessionHandle openSessionWithImpersonation(TProtocolVersion protocol, String username, + String password, String ipAddress, Map configuration, String delegationToken) + throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, ipAddress, configuration, + true, delegationToken); + LOG.debug(sessionHandle + ": openSession()"); + return sessionHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map) + */ + @Override + public SessionHandle openSession(String username, String password, Map configuration) + throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(SERVER_VERSION, username, password, null, configuration, false, null); + LOG.debug(sessionHandle + ": openSession()"); + return sessionHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map) + */ + @Override + public SessionHandle openSessionWithImpersonation(String username, String password, Map configuration, + String delegationToken) throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(SERVER_VERSION, username, password, null, configuration, + true, delegationToken); + LOG.debug(sessionHandle + ": openSession()"); + return sessionHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public void closeSession(SessionHandle sessionHandle) + throws HiveSQLException { + sessionManager.closeSession(sessionHandle); + LOG.debug(sessionHandle + ": closeSession()"); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List) + */ + @Override + public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType getInfoType) + throws HiveSQLException { + GetInfoValue infoValue = sessionManager.getSession(sessionHandle) + .getInfo(getInfoType); + LOG.debug(sessionHandle + ": getInfo()"); + return infoValue; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#executeStatement(org.apache.hive.service.cli.SessionHandle, + * java.lang.String, java.util.Map) + */ + @Override + public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, + Map confOverlay) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .executeStatement(statement, confOverlay); + LOG.debug(sessionHandle + ": executeStatement()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#executeStatementAsync(org.apache.hive.service.cli.SessionHandle, + * java.lang.String, java.util.Map) + */ + @Override + public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, + Map confOverlay) throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .executeStatementAsync(statement, confOverlay); + LOG.debug(sessionHandle + ": executeStatementAsync()"); + return opHandle; + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTypeInfo(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTypeInfo(SessionHandle sessionHandle) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getTypeInfo(); + LOG.debug(sessionHandle + ": getTypeInfo()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getCatalogs(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getCatalogs(SessionHandle sessionHandle) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getCatalogs(); + LOG.debug(sessionHandle + ": getCatalogs()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String) + */ + @Override + public OperationHandle getSchemas(SessionHandle sessionHandle, + String catalogName, String schemaName) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getSchemas(catalogName, schemaName); + LOG.debug(sessionHandle + ": getSchemas()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List) + */ + @Override + public OperationHandle getTables(SessionHandle sessionHandle, + String catalogName, String schemaName, String tableName, List tableTypes) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getTables(catalogName, schemaName, tableName, tableTypes); + LOG.debug(sessionHandle + ": getTables()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTableTypes(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTableTypes(SessionHandle sessionHandle) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getTableTypes(); + LOG.debug(sessionHandle + ": getTableTypes()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getColumns(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getColumns(SessionHandle sessionHandle, + String catalogName, String schemaName, String tableName, String columnName) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getColumns(catalogName, schemaName, tableName, columnName); + LOG.debug(sessionHandle + ": getColumns()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getFunctions(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getFunctions(SessionHandle sessionHandle, + String catalogName, String schemaName, String functionName) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getFunctions(catalogName, schemaName, functionName); + LOG.debug(sessionHandle + ": getFunctions()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getOperationStatus(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public OperationStatus getOperationStatus(OperationHandle opHandle) + throws HiveSQLException { + Operation operation = sessionManager.getOperationManager().getOperation(opHandle); + /** + * If this is a background operation run asynchronously, + * we block for a configured duration, before we return + * (duration: HIVE_SERVER2_LONG_POLLING_TIMEOUT). + * However, if the background operation is complete, we return immediately. + */ + if (operation.shouldRunAsync()) { + HiveConf conf = operation.getParentSession().getHiveConf(); + long timeout = HiveConf.getTimeVar(conf, + HiveConf.ConfVars.HIVE_SERVER2_LONG_POLLING_TIMEOUT, TimeUnit.MILLISECONDS); + try { + operation.getBackgroundHandle().get(timeout, TimeUnit.MILLISECONDS); + } catch (TimeoutException e) { + // No Op, return to the caller since long polling timeout has expired + LOG.trace(opHandle + ": Long polling timed out"); + } catch (CancellationException e) { + // The background operation thread was cancelled + LOG.trace(opHandle + ": The background operation was cancelled", e); + } catch (ExecutionException e) { + // The background operation thread was aborted + LOG.warn(opHandle + ": The background operation was aborted", e); + } catch (InterruptedException e) { + // No op, this thread was interrupted + // In this case, the call might return sooner than long polling timeout + } + } + OperationStatus opStatus = operation.getStatus(); + LOG.debug(opHandle + ": getOperationStatus()"); + return opStatus; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#cancelOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void cancelOperation(OperationHandle opHandle) + throws HiveSQLException { + sessionManager.getOperationManager().getOperation(opHandle) + .getParentSession().cancelOperation(opHandle); + LOG.debug(opHandle + ": cancelOperation()"); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#closeOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void closeOperation(OperationHandle opHandle) + throws HiveSQLException { + sessionManager.getOperationManager().getOperation(opHandle) + .getParentSession().closeOperation(opHandle); + LOG.debug(opHandle + ": closeOperation"); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public TableSchema getResultSetMetadata(OperationHandle opHandle) + throws HiveSQLException { + TableSchema tableSchema = sessionManager.getOperationManager() + .getOperation(opHandle).getParentSession().getResultSetMetadata(opHandle); + LOG.debug(opHandle + ": getResultSetMetadata()"); + return tableSchema; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#fetchResults(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public RowSet fetchResults(OperationHandle opHandle) + throws HiveSQLException { + return fetchResults(opHandle, Operation.DEFAULT_FETCH_ORIENTATION, + Operation.DEFAULT_FETCH_MAX_ROWS, FetchType.QUERY_OUTPUT); + } + + @Override + public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, + long maxRows, FetchType fetchType) throws HiveSQLException { + RowSet rowSet = sessionManager.getOperationManager().getOperation(opHandle) + .getParentSession().fetchResults(opHandle, orientation, maxRows, fetchType); + LOG.debug(opHandle + ": fetchResults()"); + return rowSet; + } + + // obtain delegation token for the give user from metastore + public synchronized String getDelegationTokenFromMetaStore(String owner) + throws HiveSQLException, UnsupportedOperationException, LoginException, IOException { + if (!hiveConf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL) || + !hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS)) { + throw new UnsupportedOperationException( + "delegation token is can only be obtained for a secure remote metastore"); + } + + try { + Hive.closeCurrent(); + return Hive.get(hiveConf).getDelegationToken(owner, owner); + } catch (HiveException e) { + if (e.getCause() instanceof UnsupportedOperationException) { + throw (UnsupportedOperationException)e.getCause(); + } else { + throw new HiveSQLException("Error connect metastore to setup impersonation", e); + } + } + } + + @Override + public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String owner, String renewer) throws HiveSQLException { + String delegationToken = sessionManager.getSession(sessionHandle). + getDelegationToken(authFactory, owner, renewer); + LOG.info(sessionHandle + ": getDelegationToken()"); + return delegationToken; + } + + @Override + public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + sessionManager.getSession(sessionHandle). + cancelDelegationToken(authFactory, tokenStr); + LOG.info(sessionHandle + ": cancelDelegationToken()"); + } + + @Override + public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + sessionManager.getSession(sessionHandle).renewDelegationToken(authFactory, tokenStr); + LOG.info(sessionHandle + ": renewDelegationToken()"); + } + + public SessionManager getSessionManager() { + return sessionManager; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceClient.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceClient.java new file mode 100644 index 0000000000..3155c238ff --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceClient.java @@ -0,0 +1,56 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.Collections; + +import org.apache.hive.service.auth.HiveAuthFactory; + + +/** + * CLIServiceClient. + * + */ +public abstract class CLIServiceClient implements ICLIService { + private static final long DEFAULT_MAX_ROWS = 1000; + + public SessionHandle openSession(String username, String password) + throws HiveSQLException { + return openSession(username, password, Collections.emptyMap()); + } + + @Override + public RowSet fetchResults(OperationHandle opHandle) throws HiveSQLException { + // TODO: provide STATIC default value + return fetchResults(opHandle, FetchOrientation.FETCH_NEXT, DEFAULT_MAX_ROWS, FetchType.QUERY_OUTPUT); + } + + @Override + public abstract String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String owner, String renewer) throws HiveSQLException; + + @Override + public abstract void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException; + + @Override + public abstract void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException; + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceUtils.java new file mode 100644 index 0000000000..9d64b102e0 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceUtils.java @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.log4j.Layout; +import org.apache.log4j.PatternLayout; + +/** + * CLIServiceUtils. + * + */ +public class CLIServiceUtils { + + + private static final char SEARCH_STRING_ESCAPE = '\\'; + public static final Layout verboseLayout = new PatternLayout( + "%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n"); + public static final Layout nonVerboseLayout = new PatternLayout( + "%-5p : %m%n"); + + /** + * Convert a SQL search pattern into an equivalent Java Regex. + * + * @param pattern input which may contain '%' or '_' wildcard characters, or + * these characters escaped using {@link #getSearchStringEscape()}. + * @return replace %/_ with regex search characters, also handle escaped + * characters. + */ + public static String patternToRegex(String pattern) { + if (pattern == null) { + return ".*"; + } else { + StringBuilder result = new StringBuilder(pattern.length()); + + boolean escaped = false; + for (int i = 0, len = pattern.length(); i < len; i++) { + char c = pattern.charAt(i); + if (escaped) { + if (c != SEARCH_STRING_ESCAPE) { + escaped = false; + } + result.append(c); + } else { + if (c == SEARCH_STRING_ESCAPE) { + escaped = true; + continue; + } else if (c == '%') { + result.append(".*"); + } else if (c == '_') { + result.append('.'); + } else { + result.append(Character.toLowerCase(c)); + } + } + } + return result.toString(); + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Column.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Column.java new file mode 100644 index 0000000000..2e21f18d61 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Column.java @@ -0,0 +1,423 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.nio.ByteBuffer; +import java.util.AbstractList; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.BitSet; +import java.util.List; + +import com.google.common.primitives.Booleans; +import com.google.common.primitives.Bytes; +import com.google.common.primitives.Doubles; +import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; +import com.google.common.primitives.Shorts; +import org.apache.hive.service.cli.thrift.TBinaryColumn; +import org.apache.hive.service.cli.thrift.TBoolColumn; +import org.apache.hive.service.cli.thrift.TByteColumn; +import org.apache.hive.service.cli.thrift.TColumn; +import org.apache.hive.service.cli.thrift.TDoubleColumn; +import org.apache.hive.service.cli.thrift.TI16Column; +import org.apache.hive.service.cli.thrift.TI32Column; +import org.apache.hive.service.cli.thrift.TI64Column; +import org.apache.hive.service.cli.thrift.TStringColumn; + +/** + * Column. + */ +public class Column extends AbstractList { + + private static final int DEFAULT_SIZE = 100; + + private final Type type; + + private BitSet nulls; + + private int size; + private boolean[] boolVars; + private byte[] byteVars; + private short[] shortVars; + private int[] intVars; + private long[] longVars; + private double[] doubleVars; + private List stringVars; + private List binaryVars; + + public Column(Type type, BitSet nulls, Object values) { + this.type = type; + this.nulls = nulls; + if (type == Type.BOOLEAN_TYPE) { + boolVars = (boolean[]) values; + size = boolVars.length; + } else if (type == Type.TINYINT_TYPE) { + byteVars = (byte[]) values; + size = byteVars.length; + } else if (type == Type.SMALLINT_TYPE) { + shortVars = (short[]) values; + size = shortVars.length; + } else if (type == Type.INT_TYPE) { + intVars = (int[]) values; + size = intVars.length; + } else if (type == Type.BIGINT_TYPE) { + longVars = (long[]) values; + size = longVars.length; + } else if (type == Type.DOUBLE_TYPE) { + doubleVars = (double[]) values; + size = doubleVars.length; + } else if (type == Type.BINARY_TYPE) { + binaryVars = (List) values; + size = binaryVars.size(); + } else if (type == Type.STRING_TYPE) { + stringVars = (List) values; + size = stringVars.size(); + } else { + throw new IllegalStateException("invalid union object"); + } + } + + public Column(Type type) { + nulls = new BitSet(); + switch (type) { + case BOOLEAN_TYPE: + boolVars = new boolean[DEFAULT_SIZE]; + break; + case TINYINT_TYPE: + byteVars = new byte[DEFAULT_SIZE]; + break; + case SMALLINT_TYPE: + shortVars = new short[DEFAULT_SIZE]; + break; + case INT_TYPE: + intVars = new int[DEFAULT_SIZE]; + break; + case BIGINT_TYPE: + longVars = new long[DEFAULT_SIZE]; + break; + case FLOAT_TYPE: + case DOUBLE_TYPE: + type = Type.DOUBLE_TYPE; + doubleVars = new double[DEFAULT_SIZE]; + break; + case BINARY_TYPE: + binaryVars = new ArrayList(); + break; + default: + type = Type.STRING_TYPE; + stringVars = new ArrayList(); + } + this.type = type; + } + + public Column(TColumn colValues) { + if (colValues.isSetBoolVal()) { + type = Type.BOOLEAN_TYPE; + nulls = toBitset(colValues.getBoolVal().getNulls()); + boolVars = Booleans.toArray(colValues.getBoolVal().getValues()); + size = boolVars.length; + } else if (colValues.isSetByteVal()) { + type = Type.TINYINT_TYPE; + nulls = toBitset(colValues.getByteVal().getNulls()); + byteVars = Bytes.toArray(colValues.getByteVal().getValues()); + size = byteVars.length; + } else if (colValues.isSetI16Val()) { + type = Type.SMALLINT_TYPE; + nulls = toBitset(colValues.getI16Val().getNulls()); + shortVars = Shorts.toArray(colValues.getI16Val().getValues()); + size = shortVars.length; + } else if (colValues.isSetI32Val()) { + type = Type.INT_TYPE; + nulls = toBitset(colValues.getI32Val().getNulls()); + intVars = Ints.toArray(colValues.getI32Val().getValues()); + size = intVars.length; + } else if (colValues.isSetI64Val()) { + type = Type.BIGINT_TYPE; + nulls = toBitset(colValues.getI64Val().getNulls()); + longVars = Longs.toArray(colValues.getI64Val().getValues()); + size = longVars.length; + } else if (colValues.isSetDoubleVal()) { + type = Type.DOUBLE_TYPE; + nulls = toBitset(colValues.getDoubleVal().getNulls()); + doubleVars = Doubles.toArray(colValues.getDoubleVal().getValues()); + size = doubleVars.length; + } else if (colValues.isSetBinaryVal()) { + type = Type.BINARY_TYPE; + nulls = toBitset(colValues.getBinaryVal().getNulls()); + binaryVars = colValues.getBinaryVal().getValues(); + size = binaryVars.size(); + } else if (colValues.isSetStringVal()) { + type = Type.STRING_TYPE; + nulls = toBitset(colValues.getStringVal().getNulls()); + stringVars = colValues.getStringVal().getValues(); + size = stringVars.size(); + } else { + throw new IllegalStateException("invalid union object"); + } + } + + public Column extractSubset(int start, int end) { + BitSet subNulls = nulls.get(start, end); + if (type == Type.BOOLEAN_TYPE) { + Column subset = new Column(type, subNulls, Arrays.copyOfRange(boolVars, start, end)); + boolVars = Arrays.copyOfRange(boolVars, end, size); + nulls = nulls.get(start, size); + size = boolVars.length; + return subset; + } + if (type == Type.TINYINT_TYPE) { + Column subset = new Column(type, subNulls, Arrays.copyOfRange(byteVars, start, end)); + byteVars = Arrays.copyOfRange(byteVars, end, size); + nulls = nulls.get(start, size); + size = byteVars.length; + return subset; + } + if (type == Type.SMALLINT_TYPE) { + Column subset = new Column(type, subNulls, Arrays.copyOfRange(shortVars, start, end)); + shortVars = Arrays.copyOfRange(shortVars, end, size); + nulls = nulls.get(start, size); + size = shortVars.length; + return subset; + } + if (type == Type.INT_TYPE) { + Column subset = new Column(type, subNulls, Arrays.copyOfRange(intVars, start, end)); + intVars = Arrays.copyOfRange(intVars, end, size); + nulls = nulls.get(start, size); + size = intVars.length; + return subset; + } + if (type == Type.BIGINT_TYPE) { + Column subset = new Column(type, subNulls, Arrays.copyOfRange(longVars, start, end)); + longVars = Arrays.copyOfRange(longVars, end, size); + nulls = nulls.get(start, size); + size = longVars.length; + return subset; + } + if (type == Type.DOUBLE_TYPE) { + Column subset = new Column(type, subNulls, Arrays.copyOfRange(doubleVars, start, end)); + doubleVars = Arrays.copyOfRange(doubleVars, end, size); + nulls = nulls.get(start, size); + size = doubleVars.length; + return subset; + } + if (type == Type.BINARY_TYPE) { + Column subset = new Column(type, subNulls, binaryVars.subList(start, end)); + binaryVars = binaryVars.subList(end, binaryVars.size()); + nulls = nulls.get(start, size); + size = binaryVars.size(); + return subset; + } + if (type == Type.STRING_TYPE) { + Column subset = new Column(type, subNulls, stringVars.subList(start, end)); + stringVars = stringVars.subList(end, stringVars.size()); + nulls = nulls.get(start, size); + size = stringVars.size(); + return subset; + } + throw new IllegalStateException("invalid union object"); + } + + private static final byte[] MASKS = new byte[] { + 0x01, 0x02, 0x04, 0x08, 0x10, 0x20, 0x40, (byte)0x80 + }; + + private static BitSet toBitset(byte[] nulls) { + BitSet bitset = new BitSet(); + int bits = nulls.length * 8; + for (int i = 0; i < bits; i++) { + bitset.set(i, (nulls[i / 8] & MASKS[i % 8]) != 0); + } + return bitset; + } + + private static byte[] toBinary(BitSet bitset) { + byte[] nulls = new byte[1 + (bitset.length() / 8)]; + for (int i = 0; i < bitset.length(); i++) { + nulls[i / 8] |= bitset.get(i) ? MASKS[i % 8] : 0; + } + return nulls; + } + + public Type getType() { + return type; + } + + @Override + public Object get(int index) { + if (nulls.get(index)) { + return null; + } + switch (type) { + case BOOLEAN_TYPE: + return boolVars[index]; + case TINYINT_TYPE: + return byteVars[index]; + case SMALLINT_TYPE: + return shortVars[index]; + case INT_TYPE: + return intVars[index]; + case BIGINT_TYPE: + return longVars[index]; + case DOUBLE_TYPE: + return doubleVars[index]; + case STRING_TYPE: + return stringVars.get(index); + case BINARY_TYPE: + return binaryVars.get(index).array(); + } + return null; + } + + @Override + public int size() { + return size; + } + + public TColumn toTColumn() { + TColumn value = new TColumn(); + ByteBuffer nullMasks = ByteBuffer.wrap(toBinary(nulls)); + switch (type) { + case BOOLEAN_TYPE: + value.setBoolVal(new TBoolColumn(Booleans.asList(Arrays.copyOfRange(boolVars, 0, size)), nullMasks)); + break; + case TINYINT_TYPE: + value.setByteVal(new TByteColumn(Bytes.asList(Arrays.copyOfRange(byteVars, 0, size)), nullMasks)); + break; + case SMALLINT_TYPE: + value.setI16Val(new TI16Column(Shorts.asList(Arrays.copyOfRange(shortVars, 0, size)), nullMasks)); + break; + case INT_TYPE: + value.setI32Val(new TI32Column(Ints.asList(Arrays.copyOfRange(intVars, 0, size)), nullMasks)); + break; + case BIGINT_TYPE: + value.setI64Val(new TI64Column(Longs.asList(Arrays.copyOfRange(longVars, 0, size)), nullMasks)); + break; + case DOUBLE_TYPE: + value.setDoubleVal(new TDoubleColumn(Doubles.asList(Arrays.copyOfRange(doubleVars, 0, size)), nullMasks)); + break; + case STRING_TYPE: + value.setStringVal(new TStringColumn(stringVars, nullMasks)); + break; + case BINARY_TYPE: + value.setBinaryVal(new TBinaryColumn(binaryVars, nullMasks)); + break; + } + return value; + } + + private static final ByteBuffer EMPTY_BINARY = ByteBuffer.allocate(0); + private static final String EMPTY_STRING = ""; + + public void addValue(Type type, Object field) { + switch (type) { + case BOOLEAN_TYPE: + nulls.set(size, field == null); + boolVars()[size] = field == null ? true : (Boolean)field; + break; + case TINYINT_TYPE: + nulls.set(size, field == null); + byteVars()[size] = field == null ? 0 : (Byte) field; + break; + case SMALLINT_TYPE: + nulls.set(size, field == null); + shortVars()[size] = field == null ? 0 : (Short)field; + break; + case INT_TYPE: + nulls.set(size, field == null); + intVars()[size] = field == null ? 0 : (Integer)field; + break; + case BIGINT_TYPE: + nulls.set(size, field == null); + longVars()[size] = field == null ? 0 : (Long)field; + break; + case FLOAT_TYPE: + nulls.set(size, field == null); + doubleVars()[size] = field == null ? 0 : ((Float)field).doubleValue(); + break; + case DOUBLE_TYPE: + nulls.set(size, field == null); + doubleVars()[size] = field == null ? 0 : (Double)field; + break; + case BINARY_TYPE: + nulls.set(binaryVars.size(), field == null); + binaryVars.add(field == null ? EMPTY_BINARY : ByteBuffer.wrap((byte[])field)); + break; + default: + nulls.set(stringVars.size(), field == null); + stringVars.add(field == null ? EMPTY_STRING : String.valueOf(field)); + break; + } + size++; + } + + private boolean[] boolVars() { + if (boolVars.length == size) { + boolean[] newVars = new boolean[size << 1]; + System.arraycopy(boolVars, 0, newVars, 0, size); + return boolVars = newVars; + } + return boolVars; + } + + private byte[] byteVars() { + if (byteVars.length == size) { + byte[] newVars = new byte[size << 1]; + System.arraycopy(byteVars, 0, newVars, 0, size); + return byteVars = newVars; + } + return byteVars; + } + + private short[] shortVars() { + if (shortVars.length == size) { + short[] newVars = new short[size << 1]; + System.arraycopy(shortVars, 0, newVars, 0, size); + return shortVars = newVars; + } + return shortVars; + } + + private int[] intVars() { + if (intVars.length == size) { + int[] newVars = new int[size << 1]; + System.arraycopy(intVars, 0, newVars, 0, size); + return intVars = newVars; + } + return intVars; + } + + private long[] longVars() { + if (longVars.length == size) { + long[] newVars = new long[size << 1]; + System.arraycopy(longVars, 0, newVars, 0, size); + return longVars = newVars; + } + return longVars; + } + + private double[] doubleVars() { + if (doubleVars.length == size) { + double[] newVars = new double[size << 1]; + System.arraycopy(doubleVars, 0, newVars, 0, size); + return doubleVars = newVars; + } + return doubleVars; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java new file mode 100644 index 0000000000..47a582e222 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java @@ -0,0 +1,149 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.hive.service.cli.thrift.TColumn; +import org.apache.hive.service.cli.thrift.TRow; +import org.apache.hive.service.cli.thrift.TRowSet; + +/** + * ColumnBasedSet. + */ +public class ColumnBasedSet implements RowSet { + + private long startOffset; + + private final Type[] types; // non-null only for writing (server-side) + private final List columns; + + public ColumnBasedSet(TableSchema schema) { + types = schema.toTypes(); + columns = new ArrayList(); + for (ColumnDescriptor colDesc : schema.getColumnDescriptors()) { + columns.add(new Column(colDesc.getType())); + } + } + + public ColumnBasedSet(TRowSet tRowSet) { + types = null; + columns = new ArrayList(); + for (TColumn tvalue : tRowSet.getColumns()) { + columns.add(new Column(tvalue)); + } + startOffset = tRowSet.getStartRowOffset(); + } + + private ColumnBasedSet(Type[] types, List columns, long startOffset) { + this.types = types; + this.columns = columns; + this.startOffset = startOffset; + } + + @Override + public ColumnBasedSet addRow(Object[] fields) { + for (int i = 0; i < fields.length; i++) { + columns.get(i).addValue(types[i], fields[i]); + } + return this; + } + + public List getColumns() { + return columns; + } + + @Override + public int numColumns() { + return columns.size(); + } + + @Override + public int numRows() { + return columns.isEmpty() ? 0 : columns.get(0).size(); + } + + @Override + public ColumnBasedSet extractSubset(int maxRows) { + int numRows = Math.min(numRows(), maxRows); + + List subset = new ArrayList(); + for (int i = 0; i < columns.size(); i++) { + subset.add(columns.get(i).extractSubset(0, numRows)); + } + ColumnBasedSet result = new ColumnBasedSet(types, subset, startOffset); + startOffset += numRows; + return result; + } + + @Override + public long getStartOffset() { + return startOffset; + } + + @Override + public void setStartOffset(long startOffset) { + this.startOffset = startOffset; + } + + public TRowSet toTRowSet() { + TRowSet tRowSet = new TRowSet(startOffset, new ArrayList()); + for (int i = 0; i < columns.size(); i++) { + tRowSet.addToColumns(columns.get(i).toTColumn()); + } + return tRowSet; + } + + @Override + public Iterator iterator() { + return new Iterator() { + + private int index; + private final Object[] convey = new Object[numColumns()]; + + @Override + public boolean hasNext() { + return index < numRows(); + } + + @Override + public Object[] next() { + for (int i = 0; i < columns.size(); i++) { + convey[i] = columns.get(i).get(index); + } + index++; + return convey; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("remove"); + } + }; + } + + public Object[] fill(int index, Object[] convey) { + for (int i = 0; i < columns.size(); i++) { + convey[i] = columns.get(i).get(index); + } + return convey; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java new file mode 100644 index 0000000000..f0bbf14693 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java @@ -0,0 +1,99 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.service.cli.thrift.TColumnDesc; + + +/** + * ColumnDescriptor. + * + */ +public class ColumnDescriptor { + private final String name; + private final String comment; + private final TypeDescriptor type; + // ordinal position of this column in the schema + private final int position; + + public ColumnDescriptor(String name, String comment, TypeDescriptor type, int position) { + this.name = name; + this.comment = comment; + this.type = type; + this.position = position; + } + + public ColumnDescriptor(TColumnDesc tColumnDesc) { + name = tColumnDesc.getColumnName(); + comment = tColumnDesc.getComment(); + type = new TypeDescriptor(tColumnDesc.getTypeDesc()); + position = tColumnDesc.getPosition(); + } + + public ColumnDescriptor(FieldSchema column, int position) { + name = column.getName(); + comment = column.getComment(); + type = new TypeDescriptor(column.getType()); + this.position = position; + } + + public static ColumnDescriptor newPrimitiveColumnDescriptor(String name, String comment, Type type, int position) { + // Current usage looks like it's only for metadata columns, but if that changes then + // this method may need to require a type qualifiers aruments. + return new ColumnDescriptor(name, comment, new TypeDescriptor(type), position); + } + + public String getName() { + return name; + } + + public String getComment() { + return comment; + } + + public TypeDescriptor getTypeDescriptor() { + return type; + } + + public int getOrdinalPosition() { + return position; + } + + public TColumnDesc toTColumnDesc() { + TColumnDesc tColumnDesc = new TColumnDesc(); + tColumnDesc.setColumnName(name); + tColumnDesc.setComment(comment); + tColumnDesc.setTypeDesc(type.toTTypeDesc()); + tColumnDesc.setPosition(position); + return tColumnDesc; + } + + public Type getType() { + return type.getType(); + } + + public boolean isPrimitive() { + return type.getType().isPrimitiveType(); + } + + public String getTypeName() { + return type.getTypeName(); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnValue.java new file mode 100644 index 0000000000..40144cfe33 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnValue.java @@ -0,0 +1,307 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; + +import org.apache.hadoop.hive.common.type.HiveChar; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; +import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; +import org.apache.hadoop.hive.common.type.HiveVarchar; +import org.apache.hive.service.cli.thrift.TBoolValue; +import org.apache.hive.service.cli.thrift.TByteValue; +import org.apache.hive.service.cli.thrift.TColumnValue; +import org.apache.hive.service.cli.thrift.TDoubleValue; +import org.apache.hive.service.cli.thrift.TI16Value; +import org.apache.hive.service.cli.thrift.TI32Value; +import org.apache.hive.service.cli.thrift.TI64Value; +import org.apache.hive.service.cli.thrift.TStringValue; + +/** + * Protocols before HIVE_CLI_SERVICE_PROTOCOL_V6 (used by RowBasedSet) + * + */ +public class ColumnValue { + + private static TColumnValue booleanValue(Boolean value) { + TBoolValue tBoolValue = new TBoolValue(); + if (value != null) { + tBoolValue.setValue(value); + } + return TColumnValue.boolVal(tBoolValue); + } + + private static TColumnValue byteValue(Byte value) { + TByteValue tByteValue = new TByteValue(); + if (value != null) { + tByteValue.setValue(value); + } + return TColumnValue.byteVal(tByteValue); + } + + private static TColumnValue shortValue(Short value) { + TI16Value tI16Value = new TI16Value(); + if (value != null) { + tI16Value.setValue(value); + } + return TColumnValue.i16Val(tI16Value); + } + + private static TColumnValue intValue(Integer value) { + TI32Value tI32Value = new TI32Value(); + if (value != null) { + tI32Value.setValue(value); + } + return TColumnValue.i32Val(tI32Value); + } + + private static TColumnValue longValue(Long value) { + TI64Value tI64Value = new TI64Value(); + if (value != null) { + tI64Value.setValue(value); + } + return TColumnValue.i64Val(tI64Value); + } + + private static TColumnValue floatValue(Float value) { + TDoubleValue tDoubleValue = new TDoubleValue(); + if (value != null) { + tDoubleValue.setValue(value); + } + return TColumnValue.doubleVal(tDoubleValue); + } + + private static TColumnValue doubleValue(Double value) { + TDoubleValue tDoubleValue = new TDoubleValue(); + if (value != null) { + tDoubleValue.setValue(value); + } + return TColumnValue.doubleVal(tDoubleValue); + } + + private static TColumnValue stringValue(String value) { + TStringValue tStringValue = new TStringValue(); + if (value != null) { + tStringValue.setValue(value); + } + return TColumnValue.stringVal(tStringValue); + } + + private static TColumnValue stringValue(HiveChar value) { + TStringValue tStringValue = new TStringValue(); + if (value != null) { + tStringValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStringValue); + } + + private static TColumnValue stringValue(HiveVarchar value) { + TStringValue tStringValue = new TStringValue(); + if (value != null) { + tStringValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStringValue); + } + + private static TColumnValue dateValue(Date value) { + TStringValue tStringValue = new TStringValue(); + if (value != null) { + tStringValue.setValue(value.toString()); + } + return new TColumnValue(TColumnValue.stringVal(tStringValue)); + } + + private static TColumnValue timestampValue(Timestamp value) { + TStringValue tStringValue = new TStringValue(); + if (value != null) { + tStringValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStringValue); + } + + private static TColumnValue stringValue(HiveDecimal value) { + TStringValue tStrValue = new TStringValue(); + if (value != null) { + tStrValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStrValue); + } + + private static TColumnValue stringValue(HiveIntervalYearMonth value) { + TStringValue tStrValue = new TStringValue(); + if (value != null) { + tStrValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStrValue); + } + + private static TColumnValue stringValue(HiveIntervalDayTime value) { + TStringValue tStrValue = new TStringValue(); + if (value != null) { + tStrValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStrValue); + } + + public static TColumnValue toTColumnValue(Type type, Object value) { + switch (type) { + case BOOLEAN_TYPE: + return booleanValue((Boolean)value); + case TINYINT_TYPE: + return byteValue((Byte)value); + case SMALLINT_TYPE: + return shortValue((Short)value); + case INT_TYPE: + return intValue((Integer)value); + case BIGINT_TYPE: + return longValue((Long)value); + case FLOAT_TYPE: + return floatValue((Float)value); + case DOUBLE_TYPE: + return doubleValue((Double)value); + case STRING_TYPE: + return stringValue((String)value); + case CHAR_TYPE: + return stringValue((HiveChar)value); + case VARCHAR_TYPE: + return stringValue((HiveVarchar)value); + case DATE_TYPE: + return dateValue((Date)value); + case TIMESTAMP_TYPE: + return timestampValue((Timestamp)value); + case INTERVAL_YEAR_MONTH_TYPE: + return stringValue((HiveIntervalYearMonth) value); + case INTERVAL_DAY_TIME_TYPE: + return stringValue((HiveIntervalDayTime) value); + case DECIMAL_TYPE: + return stringValue(((HiveDecimal)value)); + case BINARY_TYPE: + return stringValue((String)value); + case ARRAY_TYPE: + case MAP_TYPE: + case STRUCT_TYPE: + case UNION_TYPE: + case USER_DEFINED_TYPE: + return stringValue((String)value); + default: + return null; + } + } + + private static Boolean getBooleanValue(TBoolValue tBoolValue) { + if (tBoolValue.isSetValue()) { + return tBoolValue.isValue(); + } + return null; + } + + private static Byte getByteValue(TByteValue tByteValue) { + if (tByteValue.isSetValue()) { + return tByteValue.getValue(); + } + return null; + } + + private static Short getShortValue(TI16Value tI16Value) { + if (tI16Value.isSetValue()) { + return tI16Value.getValue(); + } + return null; + } + + private static Integer getIntegerValue(TI32Value tI32Value) { + if (tI32Value.isSetValue()) { + return tI32Value.getValue(); + } + return null; + } + + private static Long getLongValue(TI64Value tI64Value) { + if (tI64Value.isSetValue()) { + return tI64Value.getValue(); + } + return null; + } + + private static Double getDoubleValue(TDoubleValue tDoubleValue) { + if (tDoubleValue.isSetValue()) { + return tDoubleValue.getValue(); + } + return null; + } + + private static String getStringValue(TStringValue tStringValue) { + if (tStringValue.isSetValue()) { + return tStringValue.getValue(); + } + return null; + } + + private static Date getDateValue(TStringValue tStringValue) { + if (tStringValue.isSetValue()) { + return Date.valueOf(tStringValue.getValue()); + } + return null; + } + + private static Timestamp getTimestampValue(TStringValue tStringValue) { + if (tStringValue.isSetValue()) { + return Timestamp.valueOf(tStringValue.getValue()); + } + return null; + } + + private static byte[] getBinaryValue(TStringValue tString) { + if (tString.isSetValue()) { + return tString.getValue().getBytes(); + } + return null; + } + + private static BigDecimal getBigDecimalValue(TStringValue tStringValue) { + if (tStringValue.isSetValue()) { + return new BigDecimal(tStringValue.getValue()); + } + return null; + } + + public static Object toColumnValue(TColumnValue value) { + TColumnValue._Fields field = value.getSetField(); + switch (field) { + case BOOL_VAL: + return getBooleanValue(value.getBoolVal()); + case BYTE_VAL: + return getByteValue(value.getByteVal()); + case I16_VAL: + return getShortValue(value.getI16Val()); + case I32_VAL: + return getIntegerValue(value.getI32Val()); + case I64_VAL: + return getLongValue(value.getI64Val()); + case DOUBLE_VAL: + return getDoubleValue(value.getDoubleVal()); + case STRING_VAL: + return getStringValue(value.getStringVal()); + } + throw new IllegalArgumentException("never"); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java new file mode 100644 index 0000000000..9cad5be198 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java @@ -0,0 +1,208 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.List; +import java.util.Map; + +import org.apache.hive.service.auth.HiveAuthFactory; + + +/** + * EmbeddedCLIServiceClient. + * + */ +public class EmbeddedCLIServiceClient extends CLIServiceClient { + private final ICLIService cliService; + + public EmbeddedCLIServiceClient(ICLIService cliService) { + this.cliService = cliService; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#openSession(java.lang.String, java.lang.String, java.util.Map) + */ + @Override + public SessionHandle openSession(String username, String password, + Map configuration) throws HiveSQLException { + return cliService.openSession(username, password, configuration); + } + + @Override + public SessionHandle openSessionWithImpersonation(String username, String password, + Map configuration, String delegationToken) throws HiveSQLException { + throw new HiveSQLException("Impersonated session is not supported in the embedded mode"); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#closeSession(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public void closeSession(SessionHandle sessionHandle) throws HiveSQLException { + cliService.closeSession(sessionHandle); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List) + */ + @Override + public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType getInfoType) + throws HiveSQLException { + return cliService.getInfo(sessionHandle, getInfoType); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#executeStatement(org.apache.hive.service.cli.SessionHandle, + * java.lang.String, java.util.Map) + */ + @Override + public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, + Map confOverlay) throws HiveSQLException { + return cliService.executeStatement(sessionHandle, statement, confOverlay); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#executeStatementAsync(org.apache.hive.service.cli.SessionHandle, + * java.lang.String, java.util.Map) + */ + @Override + public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, + Map confOverlay) throws HiveSQLException { + return cliService.executeStatementAsync(sessionHandle, statement, confOverlay); + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getTypeInfo(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTypeInfo(SessionHandle sessionHandle) throws HiveSQLException { + return cliService.getTypeInfo(sessionHandle); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getCatalogs(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getCatalogs(SessionHandle sessionHandle) throws HiveSQLException { + return cliService.getCatalogs(sessionHandle); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String) + */ + @Override + public OperationHandle getSchemas(SessionHandle sessionHandle, String catalogName, + String schemaName) throws HiveSQLException { + return cliService.getSchemas(sessionHandle, catalogName, schemaName); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List) + */ + @Override + public OperationHandle getTables(SessionHandle sessionHandle, String catalogName, + String schemaName, String tableName, List tableTypes) throws HiveSQLException { + return cliService.getTables(sessionHandle, catalogName, schemaName, tableName, tableTypes); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getTableTypes(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTableTypes(SessionHandle sessionHandle) throws HiveSQLException { + return cliService.getTableTypes(sessionHandle); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getColumns(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.lang.String) + */ + @Override + public OperationHandle getColumns(SessionHandle sessionHandle, String catalogName, + String schemaName, String tableName, String columnName) throws HiveSQLException { + return cliService.getColumns(sessionHandle, catalogName, schemaName, tableName, columnName); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getFunctions(org.apache.hive.service.cli.SessionHandle, java.lang.String) + */ + @Override + public OperationHandle getFunctions(SessionHandle sessionHandle, + String catalogName, String schemaName, String functionName) + throws HiveSQLException { + return cliService.getFunctions(sessionHandle, catalogName, schemaName, functionName); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getOperationStatus(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public OperationStatus getOperationStatus(OperationHandle opHandle) throws HiveSQLException { + return cliService.getOperationStatus(opHandle); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#cancelOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { + cliService.cancelOperation(opHandle); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#closeOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void closeOperation(OperationHandle opHandle) throws HiveSQLException { + cliService.closeOperation(opHandle); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public TableSchema getResultSetMetadata(OperationHandle opHandle) throws HiveSQLException { + return cliService.getResultSetMetadata(opHandle); + } + + @Override + public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, + long maxRows, FetchType fetchType) throws HiveSQLException { + return cliService.fetchResults(opHandle, orientation, maxRows, fetchType); + } + + + @Override + public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String owner, String renewer) throws HiveSQLException { + return cliService.getDelegationToken(sessionHandle, authFactory, owner, renewer); + } + + @Override + public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + cliService.cancelDelegationToken(sessionHandle, authFactory, tokenStr); + } + + @Override + public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + cliService.renewDelegationToken(sessionHandle, authFactory, tokenStr); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchOrientation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchOrientation.java new file mode 100644 index 0000000000..ffa6f2e1f3 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchOrientation.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TFetchOrientation; + +/** + * FetchOrientation. + * + */ +public enum FetchOrientation { + FETCH_NEXT(TFetchOrientation.FETCH_NEXT), + FETCH_PRIOR(TFetchOrientation.FETCH_PRIOR), + FETCH_RELATIVE(TFetchOrientation.FETCH_RELATIVE), + FETCH_ABSOLUTE(TFetchOrientation.FETCH_ABSOLUTE), + FETCH_FIRST(TFetchOrientation.FETCH_FIRST), + FETCH_LAST(TFetchOrientation.FETCH_LAST); + + private TFetchOrientation tFetchOrientation; + + FetchOrientation(TFetchOrientation tFetchOrientation) { + this.tFetchOrientation = tFetchOrientation; + } + + public static FetchOrientation getFetchOrientation(TFetchOrientation tFetchOrientation) { + for (FetchOrientation fetchOrientation : values()) { + if (tFetchOrientation.equals(fetchOrientation.toTFetchOrientation())) { + return fetchOrientation; + } + } + // TODO: Should this really default to FETCH_NEXT? + return FETCH_NEXT; + } + + public TFetchOrientation toTFetchOrientation() { + return tFetchOrientation; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchType.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchType.java new file mode 100644 index 0000000000..a8e7fe19b0 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchType.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +/** + * FetchType indicates the type of fetchResults request. + * It maps the TFetchType, which is generated from Thrift interface. + */ +public enum FetchType { + QUERY_OUTPUT((short)0), + LOG((short)1); + + private final short tFetchType; + + FetchType(short tFetchType) { + this.tFetchType = tFetchType; + } + + public static FetchType getFetchType(short tFetchType) { + for (FetchType fetchType : values()) { + if (tFetchType == fetchType.toTFetchType()) { + return fetchType; + } + } + return QUERY_OUTPUT; + } + + public short toTFetchType() { + return tFetchType; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoType.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoType.java new file mode 100644 index 0000000000..8dd33a88fd --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoType.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TGetInfoType; + +/** + * GetInfoType. + * + */ +public enum GetInfoType { + CLI_MAX_DRIVER_CONNECTIONS(TGetInfoType.CLI_MAX_DRIVER_CONNECTIONS), + CLI_MAX_CONCURRENT_ACTIVITIES(TGetInfoType.CLI_MAX_CONCURRENT_ACTIVITIES), + CLI_DATA_SOURCE_NAME(TGetInfoType.CLI_DATA_SOURCE_NAME), + CLI_FETCH_DIRECTION(TGetInfoType.CLI_FETCH_DIRECTION), + CLI_SERVER_NAME(TGetInfoType.CLI_SERVER_NAME), + CLI_SEARCH_PATTERN_ESCAPE(TGetInfoType.CLI_SEARCH_PATTERN_ESCAPE), + CLI_DBMS_NAME(TGetInfoType.CLI_DBMS_NAME), + CLI_DBMS_VER(TGetInfoType.CLI_DBMS_VER), + CLI_ACCESSIBLE_TABLES(TGetInfoType.CLI_ACCESSIBLE_TABLES), + CLI_ACCESSIBLE_PROCEDURES(TGetInfoType.CLI_ACCESSIBLE_PROCEDURES), + CLI_CURSOR_COMMIT_BEHAVIOR(TGetInfoType.CLI_CURSOR_COMMIT_BEHAVIOR), + CLI_DATA_SOURCE_READ_ONLY(TGetInfoType.CLI_DATA_SOURCE_READ_ONLY), + CLI_DEFAULT_TXN_ISOLATION(TGetInfoType.CLI_DEFAULT_TXN_ISOLATION), + CLI_IDENTIFIER_CASE(TGetInfoType.CLI_IDENTIFIER_CASE), + CLI_IDENTIFIER_QUOTE_CHAR(TGetInfoType.CLI_IDENTIFIER_QUOTE_CHAR), + CLI_MAX_COLUMN_NAME_LEN(TGetInfoType.CLI_MAX_COLUMN_NAME_LEN), + CLI_MAX_CURSOR_NAME_LEN(TGetInfoType.CLI_MAX_CURSOR_NAME_LEN), + CLI_MAX_SCHEMA_NAME_LEN(TGetInfoType.CLI_MAX_SCHEMA_NAME_LEN), + CLI_MAX_CATALOG_NAME_LEN(TGetInfoType.CLI_MAX_CATALOG_NAME_LEN), + CLI_MAX_TABLE_NAME_LEN(TGetInfoType.CLI_MAX_TABLE_NAME_LEN), + CLI_SCROLL_CONCURRENCY(TGetInfoType.CLI_SCROLL_CONCURRENCY), + CLI_TXN_CAPABLE(TGetInfoType.CLI_TXN_CAPABLE), + CLI_USER_NAME(TGetInfoType.CLI_USER_NAME), + CLI_TXN_ISOLATION_OPTION(TGetInfoType.CLI_TXN_ISOLATION_OPTION), + CLI_INTEGRITY(TGetInfoType.CLI_INTEGRITY), + CLI_GETDATA_EXTENSIONS(TGetInfoType.CLI_GETDATA_EXTENSIONS), + CLI_NULL_COLLATION(TGetInfoType.CLI_NULL_COLLATION), + CLI_ALTER_TABLE(TGetInfoType.CLI_ALTER_TABLE), + CLI_ORDER_BY_COLUMNS_IN_SELECT(TGetInfoType.CLI_ORDER_BY_COLUMNS_IN_SELECT), + CLI_SPECIAL_CHARACTERS(TGetInfoType.CLI_SPECIAL_CHARACTERS), + CLI_MAX_COLUMNS_IN_GROUP_BY(TGetInfoType.CLI_MAX_COLUMNS_IN_GROUP_BY), + CLI_MAX_COLUMNS_IN_INDEX(TGetInfoType.CLI_MAX_COLUMNS_IN_INDEX), + CLI_MAX_COLUMNS_IN_ORDER_BY(TGetInfoType.CLI_MAX_COLUMNS_IN_ORDER_BY), + CLI_MAX_COLUMNS_IN_SELECT(TGetInfoType.CLI_MAX_COLUMNS_IN_SELECT), + CLI_MAX_COLUMNS_IN_TABLE(TGetInfoType.CLI_MAX_COLUMNS_IN_TABLE), + CLI_MAX_INDEX_SIZE(TGetInfoType.CLI_MAX_INDEX_SIZE), + CLI_MAX_ROW_SIZE(TGetInfoType.CLI_MAX_ROW_SIZE), + CLI_MAX_STATEMENT_LEN(TGetInfoType.CLI_MAX_STATEMENT_LEN), + CLI_MAX_TABLES_IN_SELECT(TGetInfoType.CLI_MAX_TABLES_IN_SELECT), + CLI_MAX_USER_NAME_LEN(TGetInfoType.CLI_MAX_USER_NAME_LEN), + CLI_OJ_CAPABILITIES(TGetInfoType.CLI_OJ_CAPABILITIES), + + CLI_XOPEN_CLI_YEAR(TGetInfoType.CLI_XOPEN_CLI_YEAR), + CLI_CURSOR_SENSITIVITY(TGetInfoType.CLI_CURSOR_SENSITIVITY), + CLI_DESCRIBE_PARAMETER(TGetInfoType.CLI_DESCRIBE_PARAMETER), + CLI_CATALOG_NAME(TGetInfoType.CLI_CATALOG_NAME), + CLI_COLLATION_SEQ(TGetInfoType.CLI_COLLATION_SEQ), + CLI_MAX_IDENTIFIER_LEN(TGetInfoType.CLI_MAX_IDENTIFIER_LEN); + + private final TGetInfoType tInfoType; + + GetInfoType(TGetInfoType tInfoType) { + this.tInfoType = tInfoType; + } + + public static GetInfoType getGetInfoType(TGetInfoType tGetInfoType) { + for (GetInfoType infoType : values()) { + if (tGetInfoType.equals(infoType.tInfoType)) { + return infoType; + } + } + throw new IllegalArgumentException("Unrecognized Thrift TGetInfoType value: " + tGetInfoType); + } + + public TGetInfoType toTGetInfoType() { + return tInfoType; + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoValue.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoValue.java new file mode 100644 index 0000000000..ba92ff4ab5 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoValue.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TGetInfoValue; + +/** + * GetInfoValue. + * + */ +public class GetInfoValue { + private String stringValue = null; + private short shortValue; + private int intValue; + private long longValue; + + public GetInfoValue(String stringValue) { + this.stringValue = stringValue; + } + + public GetInfoValue(short shortValue) { + this.shortValue = shortValue; + } + + public GetInfoValue(int intValue) { + this.intValue = intValue; + } + + public GetInfoValue(long longValue) { + this.longValue = longValue; + } + + public GetInfoValue(TGetInfoValue tGetInfoValue) { + switch (tGetInfoValue.getSetField()) { + case STRING_VALUE: + stringValue = tGetInfoValue.getStringValue(); + break; + default: + throw new IllegalArgumentException("Unreconigzed TGetInfoValue"); + } + } + + public TGetInfoValue toTGetInfoValue() { + TGetInfoValue tInfoValue = new TGetInfoValue(); + if (stringValue != null) { + tInfoValue.setStringValue(stringValue); + } + return tInfoValue; + } + + public String getStringValue() { + return stringValue; + } + + public short getShortValue() { + return shortValue; + } + + public int getIntValue() { + return intValue; + } + + public long getLongValue() { + return longValue; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Handle.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Handle.java new file mode 100644 index 0000000000..cf3427ae20 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Handle.java @@ -0,0 +1,78 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.THandleIdentifier; + + + + +public abstract class Handle { + + private final HandleIdentifier handleId; + + public Handle() { + handleId = new HandleIdentifier(); + } + + public Handle(HandleIdentifier handleId) { + this.handleId = handleId; + } + + public Handle(THandleIdentifier tHandleIdentifier) { + this.handleId = new HandleIdentifier(tHandleIdentifier); + } + + public HandleIdentifier getHandleIdentifier() { + return handleId; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((handleId == null) ? 0 : handleId.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (!(obj instanceof Handle)) { + return false; + } + Handle other = (Handle) obj; + if (handleId == null) { + if (other.handleId != null) { + return false; + } + } else if (!handleId.equals(other.handleId)) { + return false; + } + return true; + } + + @Override + public abstract String toString(); + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java new file mode 100644 index 0000000000..4dc80da8dc --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java @@ -0,0 +1,113 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.nio.ByteBuffer; +import java.util.UUID; + +import org.apache.hive.service.cli.thrift.THandleIdentifier; + +/** + * HandleIdentifier. + * + */ +public class HandleIdentifier { + private final UUID publicId; + private final UUID secretId; + + public HandleIdentifier() { + publicId = UUID.randomUUID(); + secretId = UUID.randomUUID(); + } + + public HandleIdentifier(UUID publicId, UUID secretId) { + this.publicId = publicId; + this.secretId = secretId; + } + + public HandleIdentifier(THandleIdentifier tHandleId) { + ByteBuffer bb = ByteBuffer.wrap(tHandleId.getGuid()); + this.publicId = new UUID(bb.getLong(), bb.getLong()); + bb = ByteBuffer.wrap(tHandleId.getSecret()); + this.secretId = new UUID(bb.getLong(), bb.getLong()); + } + + public UUID getPublicId() { + return publicId; + } + + public UUID getSecretId() { + return secretId; + } + + public THandleIdentifier toTHandleIdentifier() { + byte[] guid = new byte[16]; + byte[] secret = new byte[16]; + ByteBuffer guidBB = ByteBuffer.wrap(guid); + ByteBuffer secretBB = ByteBuffer.wrap(secret); + guidBB.putLong(publicId.getMostSignificantBits()); + guidBB.putLong(publicId.getLeastSignificantBits()); + secretBB.putLong(secretId.getMostSignificantBits()); + secretBB.putLong(secretId.getLeastSignificantBits()); + return new THandleIdentifier(ByteBuffer.wrap(guid), ByteBuffer.wrap(secret)); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((publicId == null) ? 0 : publicId.hashCode()); + result = prime * result + ((secretId == null) ? 0 : secretId.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (!(obj instanceof HandleIdentifier)) { + return false; + } + HandleIdentifier other = (HandleIdentifier) obj; + if (publicId == null) { + if (other.publicId != null) { + return false; + } + } else if (!publicId.equals(other.publicId)) { + return false; + } + if (secretId == null) { + if (other.secretId != null) { + return false; + } + } else if (!secretId.equals(other.secretId)) { + return false; + } + return true; + } + + @Override + public String toString() { + return publicId.toString(); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java new file mode 100644 index 0000000000..1334dde663 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java @@ -0,0 +1,248 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hive.service.cli.thrift.TStatus; +import org.apache.hive.service.cli.thrift.TStatusCode; + +/** + * HiveSQLException. + * + */ +public class HiveSQLException extends SQLException { + + /** + * + */ + private static final long serialVersionUID = -6095254671958748094L; + + /** + * + */ + public HiveSQLException() { + super(); + } + + /** + * @param reason + */ + public HiveSQLException(String reason) { + super(reason); + } + + /** + * @param cause + */ + public HiveSQLException(Throwable cause) { + super(cause); + } + + /** + * @param reason + * @param sqlState + */ + public HiveSQLException(String reason, String sqlState) { + super(reason, sqlState); + } + + /** + * @param reason + * @param cause + */ + public HiveSQLException(String reason, Throwable cause) { + super(reason, cause); + } + + /** + * @param reason + * @param sqlState + * @param vendorCode + */ + public HiveSQLException(String reason, String sqlState, int vendorCode) { + super(reason, sqlState, vendorCode); + } + + /** + * @param reason + * @param sqlState + * @param cause + */ + public HiveSQLException(String reason, String sqlState, Throwable cause) { + super(reason, sqlState, cause); + } + + /** + * @param reason + * @param sqlState + * @param vendorCode + * @param cause + */ + public HiveSQLException(String reason, String sqlState, int vendorCode, Throwable cause) { + super(reason, sqlState, vendorCode, cause); + } + + public HiveSQLException(TStatus status) { + // TODO: set correct vendorCode field + super(status.getErrorMessage(), status.getSqlState(), status.getErrorCode()); + if (status.getInfoMessages() != null) { + initCause(toCause(status.getInfoMessages())); + } + } + + /** + * Converts current object to a {@link TStatus} object + * @return a {@link TStatus} object + */ + public TStatus toTStatus() { + // TODO: convert sqlState, etc. + TStatus tStatus = new TStatus(TStatusCode.ERROR_STATUS); + tStatus.setSqlState(getSQLState()); + tStatus.setErrorCode(getErrorCode()); + tStatus.setErrorMessage(getMessage()); + tStatus.setInfoMessages(toString(this)); + return tStatus; + } + + /** + * Converts the specified {@link Exception} object into a {@link TStatus} object + * @param e a {@link Exception} object + * @return a {@link TStatus} object + */ + public static TStatus toTStatus(Exception e) { + if (e instanceof HiveSQLException) { + return ((HiveSQLException)e).toTStatus(); + } + TStatus tStatus = new TStatus(TStatusCode.ERROR_STATUS); + tStatus.setErrorMessage(e.getMessage()); + tStatus.setInfoMessages(toString(e)); + return tStatus; + } + + /** + * Converts a {@link Throwable} object into a flattened list of texts including its stack trace + * and the stack traces of the nested causes. + * @param ex a {@link Throwable} object + * @return a flattened list of texts including the {@link Throwable} object's stack trace + * and the stack traces of the nested causes. + */ + public static List toString(Throwable ex) { + return toString(ex, null); + } + + private static List toString(Throwable cause, StackTraceElement[] parent) { + StackTraceElement[] trace = cause.getStackTrace(); + int m = trace.length - 1; + if (parent != null) { + int n = parent.length - 1; + while (m >= 0 && n >= 0 && trace[m].equals(parent[n])) { + m--; n--; + } + } + List detail = enroll(cause, trace, m); + cause = cause.getCause(); + if (cause != null) { + detail.addAll(toString(cause, trace)); + } + return detail; + } + + private static List enroll(Throwable ex, StackTraceElement[] trace, int max) { + List details = new ArrayList(); + StringBuilder builder = new StringBuilder(); + builder.append('*').append(ex.getClass().getName()).append(':'); + builder.append(ex.getMessage()).append(':'); + builder.append(trace.length).append(':').append(max); + details.add(builder.toString()); + for (int i = 0; i <= max; i++) { + builder.setLength(0); + builder.append(trace[i].getClassName()).append(':'); + builder.append(trace[i].getMethodName()).append(':'); + String fileName = trace[i].getFileName(); + builder.append(fileName == null ? "" : fileName).append(':'); + builder.append(trace[i].getLineNumber()); + details.add(builder.toString()); + } + return details; + } + + /** + * Converts a flattened list of texts including the stack trace and the stack + * traces of the nested causes into a {@link Throwable} object. + * @param details a flattened list of texts including the stack trace and the stack + * traces of the nested causes + * @return a {@link Throwable} object + */ + public static Throwable toCause(List details) { + return toStackTrace(details, null, 0); + } + + private static Throwable toStackTrace(List details, StackTraceElement[] parent, int index) { + String detail = details.get(index++); + if (!detail.startsWith("*")) { + return null; // should not be happened. ignore remaining + } + int i1 = detail.indexOf(':'); + int i3 = detail.lastIndexOf(':'); + int i2 = detail.substring(0, i3).lastIndexOf(':'); + String exceptionClass = detail.substring(1, i1); + String exceptionMessage = detail.substring(i1 + 1, i2); + Throwable ex = newInstance(exceptionClass, exceptionMessage); + + Integer length = Integer.valueOf(detail.substring(i2 + 1, i3)); + Integer unique = Integer.valueOf(detail.substring(i3 + 1)); + + int i = 0; + StackTraceElement[] trace = new StackTraceElement[length]; + for (; i <= unique; i++) { + detail = details.get(index++); + int j1 = detail.indexOf(':'); + int j3 = detail.lastIndexOf(':'); + int j2 = detail.substring(0, j3).lastIndexOf(':'); + String className = detail.substring(0, j1); + String methodName = detail.substring(j1 + 1, j2); + String fileName = detail.substring(j2 + 1, j3); + if (fileName.isEmpty()) { + fileName = null; + } + int lineNumber = Integer.valueOf(detail.substring(j3 + 1)); + trace[i] = new StackTraceElement(className, methodName, fileName, lineNumber); + } + int common = trace.length - i; + if (common > 0) { + System.arraycopy(parent, parent.length - common, trace, trace.length - common, common); + } + if (details.size() > index) { + ex.initCause(toStackTrace(details, trace, index)); + } + ex.setStackTrace(trace); + return ex; + } + + private static Throwable newInstance(String className, String message) { + try { + return (Throwable)Class.forName(className).getConstructor(String.class).newInstance(message); + } catch (Exception e) { + return new RuntimeException(className + ":" + message); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ICLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ICLIService.java new file mode 100644 index 0000000000..c9cc1f4da5 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ICLIService.java @@ -0,0 +1,105 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.cli; + +import java.util.List; +import java.util.Map; + + + + +import org.apache.hive.service.auth.HiveAuthFactory; + +public interface ICLIService { + + SessionHandle openSession(String username, String password, + Map configuration) + throws HiveSQLException; + + SessionHandle openSessionWithImpersonation(String username, String password, + Map configuration, String delegationToken) + throws HiveSQLException; + + void closeSession(SessionHandle sessionHandle) + throws HiveSQLException; + + GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType infoType) + throws HiveSQLException; + + OperationHandle executeStatement(SessionHandle sessionHandle, String statement, + Map confOverlay) + throws HiveSQLException; + + OperationHandle executeStatementAsync(SessionHandle sessionHandle, + String statement, Map confOverlay) + throws HiveSQLException; + + OperationHandle getTypeInfo(SessionHandle sessionHandle) + throws HiveSQLException; + + OperationHandle getCatalogs(SessionHandle sessionHandle) + throws HiveSQLException; + + OperationHandle getSchemas(SessionHandle sessionHandle, + String catalogName, String schemaName) + throws HiveSQLException; + + OperationHandle getTables(SessionHandle sessionHandle, + String catalogName, String schemaName, String tableName, List tableTypes) + throws HiveSQLException; + + OperationHandle getTableTypes(SessionHandle sessionHandle) + throws HiveSQLException; + + OperationHandle getColumns(SessionHandle sessionHandle, + String catalogName, String schemaName, String tableName, String columnName) + throws HiveSQLException; + + OperationHandle getFunctions(SessionHandle sessionHandle, + String catalogName, String schemaName, String functionName) + throws HiveSQLException; + + OperationStatus getOperationStatus(OperationHandle opHandle) + throws HiveSQLException; + + void cancelOperation(OperationHandle opHandle) + throws HiveSQLException; + + void closeOperation(OperationHandle opHandle) + throws HiveSQLException; + + TableSchema getResultSetMetadata(OperationHandle opHandle) + throws HiveSQLException; + + RowSet fetchResults(OperationHandle opHandle) + throws HiveSQLException; + + RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, + long maxRows, FetchType fetchType) throws HiveSQLException; + + String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String owner, String renewer) throws HiveSQLException; + + void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException; + + void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException; + + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationHandle.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationHandle.java new file mode 100644 index 0000000000..5426e28471 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationHandle.java @@ -0,0 +1,102 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TOperationHandle; +import org.apache.hive.service.cli.thrift.TProtocolVersion; + +public class OperationHandle extends Handle { + + private final OperationType opType; + private final TProtocolVersion protocol; + private boolean hasResultSet = false; + + public OperationHandle(OperationType opType, TProtocolVersion protocol) { + super(); + this.opType = opType; + this.protocol = protocol; + } + + // dummy handle for ThriftCLIService + public OperationHandle(TOperationHandle tOperationHandle) { + this(tOperationHandle, TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V1); + } + + public OperationHandle(TOperationHandle tOperationHandle, TProtocolVersion protocol) { + super(tOperationHandle.getOperationId()); + this.opType = OperationType.getOperationType(tOperationHandle.getOperationType()); + this.hasResultSet = tOperationHandle.isHasResultSet(); + this.protocol = protocol; + } + + public OperationType getOperationType() { + return opType; + } + + public void setHasResultSet(boolean hasResultSet) { + this.hasResultSet = hasResultSet; + } + + public boolean hasResultSet() { + return hasResultSet; + } + + public TOperationHandle toTOperationHandle() { + TOperationHandle tOperationHandle = new TOperationHandle(); + tOperationHandle.setOperationId(getHandleIdentifier().toTHandleIdentifier()); + tOperationHandle.setOperationType(opType.toTOperationType()); + tOperationHandle.setHasResultSet(hasResultSet); + return tOperationHandle; + } + + public TProtocolVersion getProtocolVersion() { + return protocol; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = super.hashCode(); + result = prime * result + ((opType == null) ? 0 : opType.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!super.equals(obj)) { + return false; + } + if (!(obj instanceof OperationHandle)) { + return false; + } + OperationHandle other = (OperationHandle) obj; + if (opType != other.opType) { + return false; + } + return true; + } + + @Override + public String toString() { + return "OperationHandle [opType=" + opType + ", getHandleIdentifier()=" + getHandleIdentifier() + + "]"; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java new file mode 100644 index 0000000000..51ffb40369 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java @@ -0,0 +1,107 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TOperationState; + +/** + * OperationState. + * + */ +public enum OperationState { + INITIALIZED(TOperationState.INITIALIZED_STATE, false), + RUNNING(TOperationState.RUNNING_STATE, false), + FINISHED(TOperationState.FINISHED_STATE, true), + CANCELED(TOperationState.CANCELED_STATE, true), + CLOSED(TOperationState.CLOSED_STATE, true), + ERROR(TOperationState.ERROR_STATE, true), + UNKNOWN(TOperationState.UKNOWN_STATE, false), + PENDING(TOperationState.PENDING_STATE, false); + + private final TOperationState tOperationState; + private final boolean terminal; + + OperationState(TOperationState tOperationState, boolean terminal) { + this.tOperationState = tOperationState; + this.terminal = terminal; + } + + // must be sync with TOperationState in order + public static OperationState getOperationState(TOperationState tOperationState) { + return OperationState.values()[tOperationState.getValue()]; + } + + public static void validateTransition(OperationState oldState, + OperationState newState) + throws HiveSQLException { + switch (oldState) { + case INITIALIZED: + switch (newState) { + case PENDING: + case RUNNING: + case CANCELED: + case CLOSED: + return; + } + break; + case PENDING: + switch (newState) { + case RUNNING: + case FINISHED: + case CANCELED: + case ERROR: + case CLOSED: + return; + } + break; + case RUNNING: + switch (newState) { + case FINISHED: + case CANCELED: + case ERROR: + case CLOSED: + return; + } + break; + case FINISHED: + case CANCELED: + case ERROR: + if (OperationState.CLOSED.equals(newState)) { + return; + } + default: + // fall-through + } + throw new HiveSQLException("Illegal Operation state transition " + + "from " + oldState + " to " + newState); + } + + public void validateTransition(OperationState newState) + throws HiveSQLException { + validateTransition(this, newState); + } + + public TOperationState toTOperationState() { + return tOperationState; + } + + public boolean isTerminal() { + return terminal; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationStatus.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationStatus.java new file mode 100644 index 0000000000..e45b828193 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationStatus.java @@ -0,0 +1,43 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +/** + * OperationStatus + * + */ +public class OperationStatus { + + private final OperationState state; + private final HiveSQLException operationException; + + public OperationStatus(OperationState state, HiveSQLException operationException) { + this.state = state; + this.operationException = operationException; + } + + public OperationState getState() { + return state; + } + + public HiveSQLException getOperationException() { + return operationException; + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationType.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationType.java new file mode 100644 index 0000000000..429d9a4c25 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationType.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TOperationType; + +/** + * OperationType. + * + */ +public enum OperationType { + + UNKNOWN_OPERATION(TOperationType.UNKNOWN), + EXECUTE_STATEMENT(TOperationType.EXECUTE_STATEMENT), + GET_TYPE_INFO(TOperationType.GET_TYPE_INFO), + GET_CATALOGS(TOperationType.GET_CATALOGS), + GET_SCHEMAS(TOperationType.GET_SCHEMAS), + GET_TABLES(TOperationType.GET_TABLES), + GET_TABLE_TYPES(TOperationType.GET_TABLE_TYPES), + GET_COLUMNS(TOperationType.GET_COLUMNS), + GET_FUNCTIONS(TOperationType.GET_FUNCTIONS); + + private TOperationType tOperationType; + + OperationType(TOperationType tOpType) { + this.tOperationType = tOpType; + } + + public static OperationType getOperationType(TOperationType tOperationType) { + // TODO: replace this with a Map? + for (OperationType opType : values()) { + if (tOperationType.equals(opType.tOperationType)) { + return opType; + } + } + return OperationType.UNKNOWN_OPERATION; + } + + public TOperationType toTOperationType() { + return tOperationType; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java new file mode 100644 index 0000000000..6e4d43fd5d --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +/** + * PatternOrIdentifier. + * + */ +public class PatternOrIdentifier { + + boolean isPattern = false; + String text; + + public PatternOrIdentifier(String tpoi) { + text = tpoi; + isPattern = false; + } + + public boolean isPattern() { + return isPattern; + } + + public boolean isIdentifier() { + return !isPattern; + } + + @Override + public String toString() { + return text; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java new file mode 100644 index 0000000000..a0ee2109dc --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java @@ -0,0 +1,140 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.hive.service.cli.thrift.TColumnValue; +import org.apache.hive.service.cli.thrift.TRow; +import org.apache.hive.service.cli.thrift.TRowSet; + +/** + * RowBasedSet + */ +public class RowBasedSet implements RowSet { + + private long startOffset; + + private final Type[] types; // non-null only for writing (server-side) + private final RemovableList rows; + + public RowBasedSet(TableSchema schema) { + types = schema.toTypes(); + rows = new RemovableList(); + } + + public RowBasedSet(TRowSet tRowSet) { + types = null; + rows = new RemovableList(tRowSet.getRows()); + startOffset = tRowSet.getStartRowOffset(); + } + + private RowBasedSet(Type[] types, List rows, long startOffset) { + this.types = types; + this.rows = new RemovableList(rows); + this.startOffset = startOffset; + } + + @Override + public RowBasedSet addRow(Object[] fields) { + TRow tRow = new TRow(); + for (int i = 0; i < fields.length; i++) { + tRow.addToColVals(ColumnValue.toTColumnValue(types[i], fields[i])); + } + rows.add(tRow); + return this; + } + + @Override + public int numColumns() { + return rows.isEmpty() ? 0 : rows.get(0).getColVals().size(); + } + + @Override + public int numRows() { + return rows.size(); + } + + public RowBasedSet extractSubset(int maxRows) { + int numRows = Math.min(numRows(), maxRows); + RowBasedSet result = new RowBasedSet(types, rows.subList(0, numRows), startOffset); + rows.removeRange(0, numRows); + startOffset += numRows; + return result; + } + + public long getStartOffset() { + return startOffset; + } + + public void setStartOffset(long startOffset) { + this.startOffset = startOffset; + } + + public int getSize() { + return rows.size(); + } + + public TRowSet toTRowSet() { + TRowSet tRowSet = new TRowSet(); + tRowSet.setStartRowOffset(startOffset); + tRowSet.setRows(new ArrayList(rows)); + return tRowSet; + } + + @Override + public Iterator iterator() { + return new Iterator() { + + final Iterator iterator = rows.iterator(); + final Object[] convey = new Object[numColumns()]; + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public Object[] next() { + TRow row = iterator.next(); + List values = row.getColVals(); + for (int i = 0; i < values.size(); i++) { + convey[i] = ColumnValue.toColumnValue(values.get(i)); + } + return convey; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("remove"); + } + }; + } + + private static class RemovableList extends ArrayList { + public RemovableList() { super(); } + public RemovableList(List rows) { super(rows); } + @Override + public void removeRange(int fromIndex, int toIndex) { + super.removeRange(fromIndex, toIndex); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSet.java new file mode 100644 index 0000000000..ab0787e1d3 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSet.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TRowSet; + +public interface RowSet extends Iterable { + + RowSet addRow(Object[] fields); + + RowSet extractSubset(int maxRows); + + int numColumns(); + + int numRows(); + + long getStartOffset(); + + void setStartOffset(long startOffset); + + TRowSet toTRowSet(); +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSetFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSetFactory.java new file mode 100644 index 0000000000..e8f68eaaf9 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSetFactory.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TProtocolVersion; +import org.apache.hive.service.cli.thrift.TRowSet; + +import static org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V6; + +public class RowSetFactory { + + public static RowSet create(TableSchema schema, TProtocolVersion version) { + if (version.getValue() >= HIVE_CLI_SERVICE_PROTOCOL_V6.getValue()) { + return new ColumnBasedSet(schema); + } + return new RowBasedSet(schema); + } + + public static RowSet create(TRowSet results, TProtocolVersion version) { + if (version.getValue() >= HIVE_CLI_SERVICE_PROTOCOL_V6.getValue()) { + return new ColumnBasedSet(results); + } + return new RowBasedSet(results); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/SessionHandle.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/SessionHandle.java new file mode 100644 index 0000000000..52e0ad4834 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/SessionHandle.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.UUID; + +import org.apache.hive.service.cli.thrift.TProtocolVersion; +import org.apache.hive.service.cli.thrift.TSessionHandle; + + +/** + * SessionHandle. + * + */ +public class SessionHandle extends Handle { + + private final TProtocolVersion protocol; + + public SessionHandle(TProtocolVersion protocol) { + this.protocol = protocol; + } + + // dummy handle for ThriftCLIService + public SessionHandle(TSessionHandle tSessionHandle) { + this(tSessionHandle, TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V1); + } + + public SessionHandle(TSessionHandle tSessionHandle, TProtocolVersion protocol) { + super(tSessionHandle.getSessionId()); + this.protocol = protocol; + } + + public UUID getSessionId() { + return getHandleIdentifier().getPublicId(); + } + + public TSessionHandle toTSessionHandle() { + TSessionHandle tSessionHandle = new TSessionHandle(); + tSessionHandle.setSessionId(getHandleIdentifier().toTHandleIdentifier()); + return tSessionHandle; + } + + public TProtocolVersion getProtocolVersion() { + return protocol; + } + + @Override + public String toString() { + return "SessionHandle [" + getHandleIdentifier() + "]"; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TableSchema.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TableSchema.java new file mode 100644 index 0000000000..ee019bc737 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TableSchema.java @@ -0,0 +1,102 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Schema; +import org.apache.hive.service.cli.thrift.TColumnDesc; +import org.apache.hive.service.cli.thrift.TTableSchema; + +/** + * TableSchema. + * + */ +public class TableSchema { + private final List columns = new ArrayList(); + + public TableSchema() { + } + + public TableSchema(int numColumns) { + // TODO: remove this constructor + } + + public TableSchema(TTableSchema tTableSchema) { + for (TColumnDesc tColumnDesc : tTableSchema.getColumns()) { + columns.add(new ColumnDescriptor(tColumnDesc)); + } + } + + public TableSchema(List fieldSchemas) { + int pos = 1; + for (FieldSchema field : fieldSchemas) { + columns.add(new ColumnDescriptor(field, pos++)); + } + } + + public TableSchema(Schema schema) { + this(schema.getFieldSchemas()); + } + + public List getColumnDescriptors() { + return new ArrayList(columns); + } + + public ColumnDescriptor getColumnDescriptorAt(int pos) { + return columns.get(pos); + } + + public int getSize() { + return columns.size(); + } + + public void clear() { + columns.clear(); + } + + + public TTableSchema toTTableSchema() { + TTableSchema tTableSchema = new TTableSchema(); + for (ColumnDescriptor col : columns) { + tTableSchema.addToColumns(col.toTColumnDesc()); + } + return tTableSchema; + } + + public Type[] toTypes() { + Type[] types = new Type[columns.size()]; + for (int i = 0; i < types.length; i++) { + types[i] = columns.get(i).getType(); + } + return types; + } + + public TableSchema addPrimitiveColumn(String columnName, Type columnType, String columnComment) { + columns.add(ColumnDescriptor.newPrimitiveColumnDescriptor(columnName, columnComment, columnType, columns.size() + 1)); + return this; + } + + public TableSchema addStringColumn(String columnName, String columnComment) { + columns.add(ColumnDescriptor.newPrimitiveColumnDescriptor(columnName, columnComment, Type.STRING_TYPE, columns.size() + 1)); + return this; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Type.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Type.java new file mode 100644 index 0000000000..a96d2ac371 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Type.java @@ -0,0 +1,348 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.sql.DatabaseMetaData; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hive.service.cli.thrift.TTypeId; + +/** + * Type. + * + */ +public enum Type { + NULL_TYPE("VOID", + java.sql.Types.NULL, + TTypeId.NULL_TYPE), + BOOLEAN_TYPE("BOOLEAN", + java.sql.Types.BOOLEAN, + TTypeId.BOOLEAN_TYPE), + TINYINT_TYPE("TINYINT", + java.sql.Types.TINYINT, + TTypeId.TINYINT_TYPE), + SMALLINT_TYPE("SMALLINT", + java.sql.Types.SMALLINT, + TTypeId.SMALLINT_TYPE), + INT_TYPE("INT", + java.sql.Types.INTEGER, + TTypeId.INT_TYPE), + BIGINT_TYPE("BIGINT", + java.sql.Types.BIGINT, + TTypeId.BIGINT_TYPE), + FLOAT_TYPE("FLOAT", + java.sql.Types.FLOAT, + TTypeId.FLOAT_TYPE), + DOUBLE_TYPE("DOUBLE", + java.sql.Types.DOUBLE, + TTypeId.DOUBLE_TYPE), + STRING_TYPE("STRING", + java.sql.Types.VARCHAR, + TTypeId.STRING_TYPE), + CHAR_TYPE("CHAR", + java.sql.Types.CHAR, + TTypeId.CHAR_TYPE, + true, false, false), + VARCHAR_TYPE("VARCHAR", + java.sql.Types.VARCHAR, + TTypeId.VARCHAR_TYPE, + true, false, false), + DATE_TYPE("DATE", + java.sql.Types.DATE, + TTypeId.DATE_TYPE), + TIMESTAMP_TYPE("TIMESTAMP", + java.sql.Types.TIMESTAMP, + TTypeId.TIMESTAMP_TYPE), + INTERVAL_YEAR_MONTH_TYPE("INTERVAL_YEAR_MONTH", + java.sql.Types.OTHER, + TTypeId.INTERVAL_YEAR_MONTH_TYPE), + INTERVAL_DAY_TIME_TYPE("INTERVAL_DAY_TIME", + java.sql.Types.OTHER, + TTypeId.INTERVAL_DAY_TIME_TYPE), + BINARY_TYPE("BINARY", + java.sql.Types.BINARY, + TTypeId.BINARY_TYPE), + DECIMAL_TYPE("DECIMAL", + java.sql.Types.DECIMAL, + TTypeId.DECIMAL_TYPE, + true, false, false), + ARRAY_TYPE("ARRAY", + java.sql.Types.ARRAY, + TTypeId.ARRAY_TYPE, + true, true), + MAP_TYPE("MAP", + java.sql.Types.JAVA_OBJECT, + TTypeId.MAP_TYPE, + true, true), + STRUCT_TYPE("STRUCT", + java.sql.Types.STRUCT, + TTypeId.STRUCT_TYPE, + true, false), + UNION_TYPE("UNIONTYPE", + java.sql.Types.OTHER, + TTypeId.UNION_TYPE, + true, false), + USER_DEFINED_TYPE("USER_DEFINED", + java.sql.Types.OTHER, + TTypeId.USER_DEFINED_TYPE, + true, false); + + private final String name; + private final TTypeId tType; + private final int javaSQLType; + private final boolean isQualified; + private final boolean isComplex; + private final boolean isCollection; + + Type(String name, int javaSQLType, TTypeId tType, boolean isQualified, boolean isComplex, boolean isCollection) { + this.name = name; + this.javaSQLType = javaSQLType; + this.tType = tType; + this.isQualified = isQualified; + this.isComplex = isComplex; + this.isCollection = isCollection; + } + + Type(String name, int javaSQLType, TTypeId tType, boolean isComplex, boolean isCollection) { + this(name, javaSQLType, tType, false, isComplex, isCollection); + } + + Type(String name, int javaSqlType, TTypeId tType) { + this(name, javaSqlType, tType, false, false, false); + } + + public boolean isPrimitiveType() { + return !isComplex; + } + + public boolean isQualifiedType() { + return isQualified; + } + + public boolean isComplexType() { + return isComplex; + } + + public boolean isCollectionType() { + return isCollection; + } + + public static Type getType(TTypeId tType) { + for (Type type : values()) { + if (tType.equals(type.tType)) { + return type; + } + } + throw new IllegalArgumentException("Unregonized Thrift TTypeId value: " + tType); + } + + public static Type getType(String name) { + if (name == null) { + throw new IllegalArgumentException("Invalid type name: null"); + } + for (Type type : values()) { + if (name.equalsIgnoreCase(type.name)) { + return type; + } else if (type.isQualifiedType() || type.isComplexType()) { + if (name.toUpperCase().startsWith(type.name)) { + return type; + } + } + } + throw new IllegalArgumentException("Unrecognized type name: " + name); + } + + /** + * Radix for this type (typically either 2 or 10) + * Null is returned for data types where this is not applicable. + */ + public Integer getNumPrecRadix() { + if (this.isNumericType()) { + return 10; + } + return null; + } + + /** + * Maximum precision for numeric types. + * Returns null for non-numeric types. + * @return + */ + public Integer getMaxPrecision() { + switch (this) { + case TINYINT_TYPE: + return 3; + case SMALLINT_TYPE: + return 5; + case INT_TYPE: + return 10; + case BIGINT_TYPE: + return 19; + case FLOAT_TYPE: + return 7; + case DOUBLE_TYPE: + return 15; + case DECIMAL_TYPE: + return HiveDecimal.MAX_PRECISION; + default: + return null; + } + } + + public boolean isNumericType() { + switch (this) { + case TINYINT_TYPE: + case SMALLINT_TYPE: + case INT_TYPE: + case BIGINT_TYPE: + case FLOAT_TYPE: + case DOUBLE_TYPE: + case DECIMAL_TYPE: + return true; + default: + return false; + } + } + + /** + * Prefix used to quote a literal of this type (may be null) + */ + public String getLiteralPrefix() { + return null; + } + + /** + * Suffix used to quote a literal of this type (may be null) + * @return + */ + public String getLiteralSuffix() { + return null; + } + + /** + * Can you use NULL for this type? + * @return + * DatabaseMetaData.typeNoNulls - does not allow NULL values + * DatabaseMetaData.typeNullable - allows NULL values + * DatabaseMetaData.typeNullableUnknown - nullability unknown + */ + public Short getNullable() { + // All Hive types are nullable + return DatabaseMetaData.typeNullable; + } + + /** + * Is the type case sensitive? + * @return + */ + public Boolean isCaseSensitive() { + switch (this) { + case STRING_TYPE: + return true; + default: + return false; + } + } + + /** + * Parameters used in creating the type (may be null) + * @return + */ + public String getCreateParams() { + return null; + } + + /** + * Can you use WHERE based on this type? + * @return + * DatabaseMetaData.typePredNone - No support + * DatabaseMetaData.typePredChar - Only support with WHERE .. LIKE + * DatabaseMetaData.typePredBasic - Supported except for WHERE .. LIKE + * DatabaseMetaData.typeSearchable - Supported for all WHERE .. + */ + public Short getSearchable() { + if (isPrimitiveType()) { + return DatabaseMetaData.typeSearchable; + } + return DatabaseMetaData.typePredNone; + } + + /** + * Is this type unsigned? + * @return + */ + public Boolean isUnsignedAttribute() { + if (isNumericType()) { + return false; + } + return true; + } + + /** + * Can this type represent money? + * @return + */ + public Boolean isFixedPrecScale() { + return false; + } + + /** + * Can this type be used for an auto-increment value? + * @return + */ + public Boolean isAutoIncrement() { + return false; + } + + /** + * Localized version of type name (may be null). + * @return + */ + public String getLocalizedName() { + return null; + } + + /** + * Minimum scale supported for this type + * @return + */ + public Short getMinimumScale() { + return 0; + } + + /** + * Maximum scale supported for this type + * @return + */ + public Short getMaximumScale() { + return 0; + } + + public TTypeId toTType() { + return tType; + } + + public int toJavaSQLType() { + return javaSQLType; + } + + public String getName() { + return name; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java new file mode 100644 index 0000000000..562b3f5e67 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java @@ -0,0 +1,159 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.List; + +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hive.service.cli.thrift.TPrimitiveTypeEntry; +import org.apache.hive.service.cli.thrift.TTypeDesc; +import org.apache.hive.service.cli.thrift.TTypeEntry; + +/** + * TypeDescriptor. + * + */ +public class TypeDescriptor { + + private final Type type; + private String typeName = null; + private TypeQualifiers typeQualifiers = null; + + public TypeDescriptor(Type type) { + this.type = type; + } + + public TypeDescriptor(TTypeDesc tTypeDesc) { + List tTypeEntries = tTypeDesc.getTypes(); + TPrimitiveTypeEntry top = tTypeEntries.get(0).getPrimitiveEntry(); + this.type = Type.getType(top.getType()); + if (top.isSetTypeQualifiers()) { + setTypeQualifiers(TypeQualifiers.fromTTypeQualifiers(top.getTypeQualifiers())); + } + } + + public TypeDescriptor(String typeName) { + this.type = Type.getType(typeName); + if (this.type.isComplexType()) { + this.typeName = typeName; + } else if (this.type.isQualifiedType()) { + PrimitiveTypeInfo pti = TypeInfoFactory.getPrimitiveTypeInfo(typeName); + setTypeQualifiers(TypeQualifiers.fromTypeInfo(pti)); + } + } + + public Type getType() { + return type; + } + + public TTypeDesc toTTypeDesc() { + TPrimitiveTypeEntry primitiveEntry = new TPrimitiveTypeEntry(type.toTType()); + if (getTypeQualifiers() != null) { + primitiveEntry.setTypeQualifiers(getTypeQualifiers().toTTypeQualifiers()); + } + TTypeEntry entry = TTypeEntry.primitiveEntry(primitiveEntry); + + TTypeDesc desc = new TTypeDesc(); + desc.addToTypes(entry); + return desc; + } + + public String getTypeName() { + if (typeName != null) { + return typeName; + } else { + return type.getName(); + } + } + + public TypeQualifiers getTypeQualifiers() { + return typeQualifiers; + } + + public void setTypeQualifiers(TypeQualifiers typeQualifiers) { + this.typeQualifiers = typeQualifiers; + } + + /** + * The column size for this type. + * For numeric data this is the maximum precision. + * For character data this is the length in characters. + * For datetime types this is the length in characters of the String representation + * (assuming the maximum allowed precision of the fractional seconds component). + * For binary data this is the length in bytes. + * Null is returned for for data types where the column size is not applicable. + */ + public Integer getColumnSize() { + if (type.isNumericType()) { + return getPrecision(); + } + switch (type) { + case STRING_TYPE: + case BINARY_TYPE: + return Integer.MAX_VALUE; + case CHAR_TYPE: + case VARCHAR_TYPE: + return typeQualifiers.getCharacterMaximumLength(); + case DATE_TYPE: + return 10; + case TIMESTAMP_TYPE: + return 29; + default: + return null; + } + } + + /** + * Maximum precision for numeric types. + * Returns null for non-numeric types. + * @return + */ + public Integer getPrecision() { + if (this.type == Type.DECIMAL_TYPE) { + return typeQualifiers.getPrecision(); + } + return this.type.getMaxPrecision(); + } + + /** + * The number of fractional digits for this type. + * Null is returned for data types where this is not applicable. + */ + public Integer getDecimalDigits() { + switch (this.type) { + case BOOLEAN_TYPE: + case TINYINT_TYPE: + case SMALLINT_TYPE: + case INT_TYPE: + case BIGINT_TYPE: + return 0; + case FLOAT_TYPE: + return 7; + case DOUBLE_TYPE: + return 15; + case DECIMAL_TYPE: + return typeQualifiers.getScale(); + case TIMESTAMP_TYPE: + return 9; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java new file mode 100644 index 0000000000..c6da52c15a --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java @@ -0,0 +1,133 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; +import org.apache.hive.service.cli.thrift.TCLIServiceConstants; +import org.apache.hive.service.cli.thrift.TTypeQualifierValue; +import org.apache.hive.service.cli.thrift.TTypeQualifiers; + +/** + * This class holds type qualifier information for a primitive type, + * such as char/varchar length or decimal precision/scale. + */ +public class TypeQualifiers { + private Integer characterMaximumLength; + private Integer precision; + private Integer scale; + + public TypeQualifiers() {} + + public Integer getCharacterMaximumLength() { + return characterMaximumLength; + } + public void setCharacterMaximumLength(int characterMaximumLength) { + this.characterMaximumLength = characterMaximumLength; + } + + public TTypeQualifiers toTTypeQualifiers() { + TTypeQualifiers ret = null; + + Map qMap = new HashMap(); + if (getCharacterMaximumLength() != null) { + TTypeQualifierValue val = new TTypeQualifierValue(); + val.setI32Value(getCharacterMaximumLength().intValue()); + qMap.put(TCLIServiceConstants.CHARACTER_MAXIMUM_LENGTH, val); + } + + if (precision != null) { + TTypeQualifierValue val = new TTypeQualifierValue(); + val.setI32Value(precision.intValue()); + qMap.put(TCLIServiceConstants.PRECISION, val); + } + + if (scale != null) { + TTypeQualifierValue val = new TTypeQualifierValue(); + val.setI32Value(scale.intValue()); + qMap.put(TCLIServiceConstants.SCALE, val); + } + + if (qMap.size() > 0) { + ret = new TTypeQualifiers(qMap); + } + + return ret; + } + + public static TypeQualifiers fromTTypeQualifiers(TTypeQualifiers ttq) { + TypeQualifiers ret = null; + if (ttq != null) { + ret = new TypeQualifiers(); + Map tqMap = ttq.getQualifiers(); + + if (tqMap.containsKey(TCLIServiceConstants.CHARACTER_MAXIMUM_LENGTH)) { + ret.setCharacterMaximumLength( + tqMap.get(TCLIServiceConstants.CHARACTER_MAXIMUM_LENGTH).getI32Value()); + } + + if (tqMap.containsKey(TCLIServiceConstants.PRECISION)) { + ret.setPrecision(tqMap.get(TCLIServiceConstants.PRECISION).getI32Value()); + } + + if (tqMap.containsKey(TCLIServiceConstants.SCALE)) { + ret.setScale(tqMap.get(TCLIServiceConstants.SCALE).getI32Value()); + } + } + return ret; + } + + public static TypeQualifiers fromTypeInfo(PrimitiveTypeInfo pti) { + TypeQualifiers result = null; + if (pti instanceof VarcharTypeInfo) { + result = new TypeQualifiers(); + result.setCharacterMaximumLength(((VarcharTypeInfo)pti).getLength()); + } else if (pti instanceof CharTypeInfo) { + result = new TypeQualifiers(); + result.setCharacterMaximumLength(((CharTypeInfo)pti).getLength()); + } else if (pti instanceof DecimalTypeInfo) { + result = new TypeQualifiers(); + result.setPrecision(((DecimalTypeInfo)pti).precision()); + result.setScale(((DecimalTypeInfo)pti).scale()); + } + return result; + } + + public Integer getPrecision() { + return precision; + } + + public void setPrecision(Integer precision) { + this.precision = precision; + } + + public Integer getScale() { + return scale; + } + + public void setScale(Integer scale) { + this.scale = scale; + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java new file mode 100644 index 0000000000..87ac39b051 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java @@ -0,0 +1,86 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.hive.metastore.TableType; + +/** + * ClassicTableTypeMapping. + * Classic table type mapping : + * Managed Table ==> Table + * External Table ==> Table + * Virtual View ==> View + */ +public class ClassicTableTypeMapping implements TableTypeMapping { + + public enum ClassicTableTypes { + TABLE, + VIEW, + } + + private final Map hiveToClientMap = new HashMap(); + private final Map clientToHiveMap = new HashMap(); + + public ClassicTableTypeMapping () { + hiveToClientMap.put(TableType.MANAGED_TABLE.toString(), + ClassicTableTypes.TABLE.toString()); + hiveToClientMap.put(TableType.EXTERNAL_TABLE.toString(), + ClassicTableTypes.TABLE.toString()); + hiveToClientMap.put(TableType.VIRTUAL_VIEW.toString(), + ClassicTableTypes.VIEW.toString()); + + clientToHiveMap.put(ClassicTableTypes.TABLE.toString(), + TableType.MANAGED_TABLE.toString()); + clientToHiveMap.put(ClassicTableTypes.VIEW.toString(), + TableType.VIRTUAL_VIEW.toString()); + } + + @Override + public String mapToHiveType(String clientTypeName) { + if (clientToHiveMap.containsKey(clientTypeName)) { + return clientToHiveMap.get(clientTypeName); + } else { + return clientTypeName; + } + } + + @Override + public String mapToClientType(String hiveTypeName) { + if (hiveToClientMap.containsKey(hiveTypeName)) { + return hiveToClientMap.get(hiveTypeName); + } else { + return hiveTypeName; + } + } + + @Override + public Set getTableTypeNames() { + Set typeNameSet = new HashSet(); + for (ClassicTableTypes typeNames : ClassicTableTypes.values()) { + typeNameSet.add(typeNames.toString()); + } + return typeNameSet; + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java new file mode 100644 index 0000000000..3f2de108f0 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.cli.operation; + +import java.sql.SQLException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.hadoop.hive.ql.processors.CommandProcessor; +import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.session.HiveSession; + +public abstract class ExecuteStatementOperation extends Operation { + protected String statement = null; + protected Map confOverlay = new HashMap(); + + public ExecuteStatementOperation(HiveSession parentSession, String statement, + Map confOverlay, boolean runInBackground) { + super(parentSession, OperationType.EXECUTE_STATEMENT, runInBackground); + this.statement = statement; + setConfOverlay(confOverlay); + } + + public String getStatement() { + return statement; + } + + public static ExecuteStatementOperation newExecuteStatementOperation( + HiveSession parentSession, String statement, Map confOverlay, boolean runAsync) + throws HiveSQLException { + String[] tokens = statement.trim().split("\\s+"); + CommandProcessor processor = null; + try { + processor = CommandProcessorFactory.getForHiveCommand(tokens, parentSession.getHiveConf()); + } catch (SQLException e) { + throw new HiveSQLException(e.getMessage(), e.getSQLState(), e); + } + if (processor == null) { + return new SQLOperation(parentSession, statement, confOverlay, runAsync); + } + return new HiveCommandOperation(parentSession, statement, processor, confOverlay); + } + + protected Map getConfOverlay() { + return confOverlay; + } + + protected void setConfOverlay(Map confOverlay) { + if (confOverlay != null) { + this.confOverlay = confOverlay; + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java new file mode 100644 index 0000000000..8868ec18e0 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java @@ -0,0 +1,81 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetCatalogsOperation. + * + */ +public class GetCatalogsOperation extends MetadataOperation { + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addStringColumn("TABLE_CAT", "Catalog name. NULL if not applicable."); + + private final RowSet rowSet; + + protected GetCatalogsOperation(HiveSession parentSession) { + super(parentSession, OperationType.GET_CATALOGS); + rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + try { + if (isAuthV2Enabled()) { + authorizeMetaGets(HiveOperationType.GET_CATALOGS, null); + } + setState(OperationState.FINISHED); + } catch (HiveSQLException e) { + setState(OperationState.ERROR); + throw e; + } + + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java new file mode 100644 index 0000000000..309f10f640 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java @@ -0,0 +1,236 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.sql.DatabaseMetaData; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.regex.Pattern; + +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.plan.HiveOperation; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType; +import org.apache.hive.service.cli.ColumnDescriptor; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.Type; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetColumnsOperation. + * + */ +public class GetColumnsOperation extends MetadataOperation { + + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addPrimitiveColumn("TABLE_CAT", Type.STRING_TYPE, + "Catalog name. NULL if not applicable") + .addPrimitiveColumn("TABLE_SCHEM", Type.STRING_TYPE, + "Schema name") + .addPrimitiveColumn("TABLE_NAME", Type.STRING_TYPE, + "Table name") + .addPrimitiveColumn("COLUMN_NAME", Type.STRING_TYPE, + "Column name") + .addPrimitiveColumn("DATA_TYPE", Type.INT_TYPE, + "SQL type from java.sql.Types") + .addPrimitiveColumn("TYPE_NAME", Type.STRING_TYPE, + "Data source dependent type name, for a UDT the type name is fully qualified") + .addPrimitiveColumn("COLUMN_SIZE", Type.INT_TYPE, + "Column size. For char or date types this is the maximum number of characters," + + " for numeric or decimal types this is precision.") + .addPrimitiveColumn("BUFFER_LENGTH", Type.TINYINT_TYPE, + "Unused") + .addPrimitiveColumn("DECIMAL_DIGITS", Type.INT_TYPE, + "The number of fractional digits") + .addPrimitiveColumn("NUM_PREC_RADIX", Type.INT_TYPE, + "Radix (typically either 10 or 2)") + .addPrimitiveColumn("NULLABLE", Type.INT_TYPE, + "Is NULL allowed") + .addPrimitiveColumn("REMARKS", Type.STRING_TYPE, + "Comment describing column (may be null)") + .addPrimitiveColumn("COLUMN_DEF", Type.STRING_TYPE, + "Default value (may be null)") + .addPrimitiveColumn("SQL_DATA_TYPE", Type.INT_TYPE, + "Unused") + .addPrimitiveColumn("SQL_DATETIME_SUB", Type.INT_TYPE, + "Unused") + .addPrimitiveColumn("CHAR_OCTET_LENGTH", Type.INT_TYPE, + "For char types the maximum number of bytes in the column") + .addPrimitiveColumn("ORDINAL_POSITION", Type.INT_TYPE, + "Index of column in table (starting at 1)") + .addPrimitiveColumn("IS_NULLABLE", Type.STRING_TYPE, + "\"NO\" means column definitely does not allow NULL values; " + + "\"YES\" means the column might allow NULL values. An empty " + + "string means nobody knows.") + .addPrimitiveColumn("SCOPE_CATALOG", Type.STRING_TYPE, + "Catalog of table that is the scope of a reference attribute " + + "(null if DATA_TYPE isn't REF)") + .addPrimitiveColumn("SCOPE_SCHEMA", Type.STRING_TYPE, + "Schema of table that is the scope of a reference attribute " + + "(null if the DATA_TYPE isn't REF)") + .addPrimitiveColumn("SCOPE_TABLE", Type.STRING_TYPE, + "Table name that this the scope of a reference attribure " + + "(null if the DATA_TYPE isn't REF)") + .addPrimitiveColumn("SOURCE_DATA_TYPE", Type.SMALLINT_TYPE, + "Source type of a distinct type or user-generated Ref type, " + + "SQL type from java.sql.Types (null if DATA_TYPE isn't DISTINCT or user-generated REF)") + .addPrimitiveColumn("IS_AUTO_INCREMENT", Type.STRING_TYPE, + "Indicates whether this column is auto incremented."); + + private final String catalogName; + private final String schemaName; + private final String tableName; + private final String columnName; + + private final RowSet rowSet; + + protected GetColumnsOperation(HiveSession parentSession, String catalogName, String schemaName, + String tableName, String columnName) { + super(parentSession, OperationType.GET_COLUMNS); + this.catalogName = catalogName; + this.schemaName = schemaName; + this.tableName = tableName; + this.columnName = columnName; + this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + try { + IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); + String schemaPattern = convertSchemaPattern(schemaName); + String tablePattern = convertIdentifierPattern(tableName, true); + + Pattern columnPattern = null; + if (columnName != null) { + columnPattern = Pattern.compile(convertIdentifierPattern(columnName, false)); + } + + List dbNames = metastoreClient.getDatabases(schemaPattern); + Collections.sort(dbNames); + Map> db2Tabs = new HashMap<>(); + + for (String dbName : dbNames) { + List tableNames = metastoreClient.getTables(dbName, tablePattern); + Collections.sort(tableNames); + db2Tabs.put(dbName, tableNames); + } + + if (isAuthV2Enabled()) { + List privObjs = getPrivObjs(db2Tabs); + String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName + + ", tablePattern : " + tableName; + authorizeMetaGets(HiveOperationType.GET_COLUMNS, privObjs, cmdStr); + } + + for (Entry> dbTabs : db2Tabs.entrySet()) { + String dbName = dbTabs.getKey(); + List tableNames = dbTabs.getValue(); + for (Table table : metastoreClient.getTableObjectsByName(dbName, tableNames)) { + TableSchema schema = new TableSchema(metastoreClient.getSchema(dbName, table.getTableName())); + for (ColumnDescriptor column : schema.getColumnDescriptors()) { + if (columnPattern != null && !columnPattern.matcher(column.getName()).matches()) { + continue; + } + Object[] rowData = new Object[] { + null, // TABLE_CAT + table.getDbName(), // TABLE_SCHEM + table.getTableName(), // TABLE_NAME + column.getName(), // COLUMN_NAME + column.getType().toJavaSQLType(), // DATA_TYPE + column.getTypeName(), // TYPE_NAME + column.getTypeDescriptor().getColumnSize(), // COLUMN_SIZE + null, // BUFFER_LENGTH, unused + column.getTypeDescriptor().getDecimalDigits(), // DECIMAL_DIGITS + column.getType().getNumPrecRadix(), // NUM_PREC_RADIX + DatabaseMetaData.columnNullable, // NULLABLE + column.getComment(), // REMARKS + null, // COLUMN_DEF + null, // SQL_DATA_TYPE + null, // SQL_DATETIME_SUB + null, // CHAR_OCTET_LENGTH + column.getOrdinalPosition(), // ORDINAL_POSITION + "YES", // IS_NULLABLE + null, // SCOPE_CATALOG + null, // SCOPE_SCHEMA + null, // SCOPE_TABLE + null, // SOURCE_DATA_TYPE + "NO", // IS_AUTO_INCREMENT + }; + rowSet.addRow(rowData); + } + } + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + + } + + + private List getPrivObjs(Map> db2Tabs) { + List privObjs = new ArrayList<>(); + for (Entry> dbTabs : db2Tabs.entrySet()) { + for (String tabName : dbTabs.getValue()) { + privObjs.add(new HivePrivilegeObject(HivePrivilegeObjectType.TABLE_OR_VIEW, dbTabs.getKey(), + tabName)); + } + } + return privObjs; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java new file mode 100644 index 0000000000..6df1e8a227 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java @@ -0,0 +1,148 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.sql.DatabaseMetaData; +import java.util.List; +import java.util.Set; + +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.exec.FunctionInfo; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils; +import org.apache.hive.service.cli.CLIServiceUtils; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.Type; +import org.apache.hive.service.cli.session.HiveSession; +import org.apache.thrift.TException; + +/** + * GetFunctionsOperation. + * + */ +public class GetFunctionsOperation extends MetadataOperation { + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addPrimitiveColumn("FUNCTION_CAT", Type.STRING_TYPE, + "Function catalog (may be null)") + .addPrimitiveColumn("FUNCTION_SCHEM", Type.STRING_TYPE, + "Function schema (may be null)") + .addPrimitiveColumn("FUNCTION_NAME", Type.STRING_TYPE, + "Function name. This is the name used to invoke the function") + .addPrimitiveColumn("REMARKS", Type.STRING_TYPE, + "Explanatory comment on the function") + .addPrimitiveColumn("FUNCTION_TYPE", Type.INT_TYPE, + "Kind of function.") + .addPrimitiveColumn("SPECIFIC_NAME", Type.STRING_TYPE, + "The name which uniquely identifies this function within its schema"); + + private final String catalogName; + private final String schemaName; + private final String functionName; + + private final RowSet rowSet; + + public GetFunctionsOperation(HiveSession parentSession, + String catalogName, String schemaName, String functionName) { + super(parentSession, OperationType.GET_FUNCTIONS); + this.catalogName = catalogName; + this.schemaName = schemaName; + this.functionName = functionName; + this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + if (isAuthV2Enabled()) { + // get databases for schema pattern + IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); + String schemaPattern = convertSchemaPattern(schemaName); + List matchingDbs; + try { + matchingDbs = metastoreClient.getDatabases(schemaPattern); + } catch (TException e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + // authorize this call on the schema objects + List privObjs = HivePrivilegeObjectUtils + .getHivePrivDbObjects(matchingDbs); + String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName; + authorizeMetaGets(HiveOperationType.GET_FUNCTIONS, privObjs, cmdStr); + } + + try { + if ((null == catalogName || "".equals(catalogName)) + && (null == schemaName || "".equals(schemaName))) { + Set functionNames = FunctionRegistry + .getFunctionNames(CLIServiceUtils.patternToRegex(functionName)); + for (String functionName : functionNames) { + FunctionInfo functionInfo = FunctionRegistry.getFunctionInfo(functionName); + Object rowData[] = new Object[] { + null, // FUNCTION_CAT + null, // FUNCTION_SCHEM + functionInfo.getDisplayName(), // FUNCTION_NAME + "", // REMARKS + (functionInfo.isGenericUDTF() ? + DatabaseMetaData.functionReturnsTable + : DatabaseMetaData.functionNoTable), // FUNCTION_TYPE + functionInfo.getClass().getCanonicalName() + }; + rowSet.addRow(rowData); + } + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java new file mode 100644 index 0000000000..e56686abb7 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetSchemasOperation. + * + */ +public class GetSchemasOperation extends MetadataOperation { + private final String catalogName; + private final String schemaName; + + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addStringColumn("TABLE_SCHEM", "Schema name.") + .addStringColumn("TABLE_CATALOG", "Catalog name."); + + private RowSet rowSet; + + protected GetSchemasOperation(HiveSession parentSession, + String catalogName, String schemaName) { + super(parentSession, OperationType.GET_SCHEMAS); + this.catalogName = catalogName; + this.schemaName = schemaName; + this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + if (isAuthV2Enabled()) { + String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName; + authorizeMetaGets(HiveOperationType.GET_SCHEMAS, null, cmdStr); + } + try { + IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); + String schemaPattern = convertSchemaPattern(schemaName); + for (String dbName : metastoreClient.getDatabases(schemaPattern)) { + rowSet.addRow(new Object[] {dbName, DEFAULT_HIVE_CATALOG}); + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java new file mode 100644 index 0000000000..a09b39a4e0 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java @@ -0,0 +1,93 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetTableTypesOperation. + * + */ +public class GetTableTypesOperation extends MetadataOperation { + + protected static TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addStringColumn("TABLE_TYPE", "Table type name."); + + private final RowSet rowSet; + private final TableTypeMapping tableTypeMapping; + + protected GetTableTypesOperation(HiveSession parentSession) { + super(parentSession, OperationType.GET_TABLE_TYPES); + String tableMappingStr = getParentSession().getHiveConf(). + getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); + tableTypeMapping = + TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); + rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + if (isAuthV2Enabled()) { + authorizeMetaGets(HiveOperationType.GET_TABLETYPES, null); + } + try { + for (TableType type : TableType.values()) { + rowSet.addRow(new String[] {tableTypeMapping.mapToClientType(type.toString())}); + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java new file mode 100644 index 0000000000..0e2fdc657c --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java @@ -0,0 +1,135 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetTablesOperation. + * + */ +public class GetTablesOperation extends MetadataOperation { + + private final String catalogName; + private final String schemaName; + private final String tableName; + private final List tableTypes = new ArrayList(); + private final RowSet rowSet; + private final TableTypeMapping tableTypeMapping; + + + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addStringColumn("TABLE_CAT", "Catalog name. NULL if not applicable.") + .addStringColumn("TABLE_SCHEM", "Schema name.") + .addStringColumn("TABLE_NAME", "Table name.") + .addStringColumn("TABLE_TYPE", "The table type, e.g. \"TABLE\", \"VIEW\", etc.") + .addStringColumn("REMARKS", "Comments about the table."); + + protected GetTablesOperation(HiveSession parentSession, + String catalogName, String schemaName, String tableName, + List tableTypes) { + super(parentSession, OperationType.GET_TABLES); + this.catalogName = catalogName; + this.schemaName = schemaName; + this.tableName = tableName; + String tableMappingStr = getParentSession().getHiveConf(). + getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); + tableTypeMapping = + TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); + if (tableTypes != null) { + this.tableTypes.addAll(tableTypes); + } + this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + try { + IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); + String schemaPattern = convertSchemaPattern(schemaName); + List matchingDbs = metastoreClient.getDatabases(schemaPattern); + if(isAuthV2Enabled()){ + List privObjs = HivePrivilegeObjectUtils.getHivePrivDbObjects(matchingDbs); + String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName; + authorizeMetaGets(HiveOperationType.GET_TABLES, privObjs, cmdStr); + } + + String tablePattern = convertIdentifierPattern(tableName, true); + for (String dbName : metastoreClient.getDatabases(schemaPattern)) { + List tableNames = metastoreClient.getTables(dbName, tablePattern); + for (Table table : metastoreClient.getTableObjectsByName(dbName, tableNames)) { + Object[] rowData = new Object[] { + DEFAULT_HIVE_CATALOG, + table.getDbName(), + table.getTableName(), + tableTypeMapping.mapToClientType(table.getTableType()), + table.getParameters().get("comment") + }; + if (tableTypes.isEmpty() || tableTypes.contains( + tableTypeMapping.mapToClientType(table.getTableType()))) { + rowSet.addRow(rowData); + } + } + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java new file mode 100644 index 0000000000..2a0fec2771 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java @@ -0,0 +1,142 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.Type; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetTypeInfoOperation. + * + */ +public class GetTypeInfoOperation extends MetadataOperation { + + private final static TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addPrimitiveColumn("TYPE_NAME", Type.STRING_TYPE, + "Type name") + .addPrimitiveColumn("DATA_TYPE", Type.INT_TYPE, + "SQL data type from java.sql.Types") + .addPrimitiveColumn("PRECISION", Type.INT_TYPE, + "Maximum precision") + .addPrimitiveColumn("LITERAL_PREFIX", Type.STRING_TYPE, + "Prefix used to quote a literal (may be null)") + .addPrimitiveColumn("LITERAL_SUFFIX", Type.STRING_TYPE, + "Suffix used to quote a literal (may be null)") + .addPrimitiveColumn("CREATE_PARAMS", Type.STRING_TYPE, + "Parameters used in creating the type (may be null)") + .addPrimitiveColumn("NULLABLE", Type.SMALLINT_TYPE, + "Can you use NULL for this type") + .addPrimitiveColumn("CASE_SENSITIVE", Type.BOOLEAN_TYPE, + "Is it case sensitive") + .addPrimitiveColumn("SEARCHABLE", Type.SMALLINT_TYPE, + "Can you use \"WHERE\" based on this type") + .addPrimitiveColumn("UNSIGNED_ATTRIBUTE", Type.BOOLEAN_TYPE, + "Is it unsigned") + .addPrimitiveColumn("FIXED_PREC_SCALE", Type.BOOLEAN_TYPE, + "Can it be a money value") + .addPrimitiveColumn("AUTO_INCREMENT", Type.BOOLEAN_TYPE, + "Can it be used for an auto-increment value") + .addPrimitiveColumn("LOCAL_TYPE_NAME", Type.STRING_TYPE, + "Localized version of type name (may be null)") + .addPrimitiveColumn("MINIMUM_SCALE", Type.SMALLINT_TYPE, + "Minimum scale supported") + .addPrimitiveColumn("MAXIMUM_SCALE", Type.SMALLINT_TYPE, + "Maximum scale supported") + .addPrimitiveColumn("SQL_DATA_TYPE", Type.INT_TYPE, + "Unused") + .addPrimitiveColumn("SQL_DATETIME_SUB", Type.INT_TYPE, + "Unused") + .addPrimitiveColumn("NUM_PREC_RADIX", Type.INT_TYPE, + "Usually 2 or 10"); + + private final RowSet rowSet; + + protected GetTypeInfoOperation(HiveSession parentSession) { + super(parentSession, OperationType.GET_TYPE_INFO); + rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + if (isAuthV2Enabled()) { + authorizeMetaGets(HiveOperationType.GET_TYPEINFO, null); + } + try { + for (Type type : Type.values()) { + Object[] rowData = new Object[] { + type.getName(), // TYPE_NAME + type.toJavaSQLType(), // DATA_TYPE + type.getMaxPrecision(), // PRECISION + type.getLiteralPrefix(), // LITERAL_PREFIX + type.getLiteralSuffix(), // LITERAL_SUFFIX + type.getCreateParams(), // CREATE_PARAMS + type.getNullable(), // NULLABLE + type.isCaseSensitive(), // CASE_SENSITIVE + type.getSearchable(), // SEARCHABLE + type.isUnsignedAttribute(), // UNSIGNED_ATTRIBUTE + type.isFixedPrecScale(), // FIXED_PREC_SCALE + type.isAutoIncrement(), // AUTO_INCREMENT + type.getLocalizedName(), // LOCAL_TYPE_NAME + type.getMinimumScale(), // MINIMUM_SCALE + type.getMaximumScale(), // MAXIMUM_SCALE + null, // SQL_DATA_TYPE, unused + null, // SQL_DATETIME_SUB, unused + type.getNumPrecRadix() //NUM_PREC_RADIX + }; + rowSet.addRow(rowData); + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java new file mode 100644 index 0000000000..bcc66cf811 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java @@ -0,0 +1,213 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FileReader; +import java.io.IOException; +import java.io.PrintStream; +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hive.metastore.api.Schema; +import org.apache.hadoop.hive.ql.processors.CommandProcessor; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.io.IOUtils; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * Executes a HiveCommand + */ +public class HiveCommandOperation extends ExecuteStatementOperation { + private CommandProcessor commandProcessor; + private TableSchema resultSchema = null; + + /** + * For processors other than Hive queries (Driver), they output to session.out (a temp file) + * first and the fetchOne/fetchN/fetchAll functions get the output from pipeIn. + */ + private BufferedReader resultReader; + + + protected HiveCommandOperation(HiveSession parentSession, String statement, + CommandProcessor commandProcessor, Map confOverlay) { + super(parentSession, statement, confOverlay, false); + this.commandProcessor = commandProcessor; + setupSessionIO(parentSession.getSessionState()); + } + + private void setupSessionIO(SessionState sessionState) { + try { + LOG.info("Putting temp output to file " + sessionState.getTmpOutputFile().toString()); + sessionState.in = null; // hive server's session input stream is not used + // open a per-session file in auto-flush mode for writing temp results + sessionState.out = new PrintStream(new FileOutputStream(sessionState.getTmpOutputFile()), true, "UTF-8"); + // TODO: for hadoop jobs, progress is printed out to session.err, + // we should find a way to feed back job progress to client + sessionState.err = new PrintStream(System.err, true, "UTF-8"); + } catch (IOException e) { + LOG.error("Error in creating temp output file ", e); + try { + sessionState.in = null; + sessionState.out = new PrintStream(System.out, true, "UTF-8"); + sessionState.err = new PrintStream(System.err, true, "UTF-8"); + } catch (UnsupportedEncodingException ee) { + LOG.error("Error creating PrintStream", e); + ee.printStackTrace(); + sessionState.out = null; + sessionState.err = null; + } + } + } + + + private void tearDownSessionIO() { + IOUtils.cleanup(LOG, parentSession.getSessionState().out); + IOUtils.cleanup(LOG, parentSession.getSessionState().err); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + try { + String command = getStatement().trim(); + String[] tokens = statement.split("\\s"); + String commandArgs = command.substring(tokens[0].length()).trim(); + + CommandProcessorResponse response = commandProcessor.run(commandArgs); + int returnCode = response.getResponseCode(); + if (returnCode != 0) { + throw toSQLException("Error while processing statement", response); + } + Schema schema = response.getSchema(); + if (schema != null) { + setHasResultSet(true); + resultSchema = new TableSchema(schema); + } else { + setHasResultSet(false); + resultSchema = new TableSchema(); + } + } catch (HiveSQLException e) { + setState(OperationState.ERROR); + throw e; + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException("Error running query: " + e.toString(), e); + } + setState(OperationState.FINISHED); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.operation.Operation#close() + */ + @Override + public void close() throws HiveSQLException { + setState(OperationState.CLOSED); + tearDownSessionIO(); + cleanTmpFile(); + cleanupOperationLog(); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.operation.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + return resultSchema; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.operation.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + resetResultReader(); + } + List rows = readResults((int) maxRows); + RowSet rowSet = RowSetFactory.create(resultSchema, getProtocolVersion()); + + for (String row : rows) { + rowSet.addRow(new String[] {row}); + } + return rowSet; + } + + /** + * Reads the temporary results for non-Hive (non-Driver) commands to the + * resulting List of strings. + * @param nLines number of lines read at once. If it is <= 0, then read all lines. + */ + private List readResults(int nLines) throws HiveSQLException { + if (resultReader == null) { + SessionState sessionState = getParentSession().getSessionState(); + File tmp = sessionState.getTmpOutputFile(); + try { + resultReader = new BufferedReader(new FileReader(tmp)); + } catch (FileNotFoundException e) { + LOG.error("File " + tmp + " not found. ", e); + throw new HiveSQLException(e); + } + } + List results = new ArrayList(); + + for (int i = 0; i < nLines || nLines <= 0; ++i) { + try { + String line = resultReader.readLine(); + if (line == null) { + // reached the end of the result file + break; + } else { + results.add(line); + } + } catch (IOException e) { + LOG.error("Reading temp results encountered an exception: ", e); + throw new HiveSQLException(e); + } + } + return results; + } + + private void cleanTmpFile() { + resetResultReader(); + SessionState sessionState = getParentSession().getSessionState(); + File tmp = sessionState.getTmpOutputFile(); + tmp.delete(); + } + + private void resetResultReader() { + if (resultReader != null) { + IOUtils.cleanup(LOG, resultReader); + resultReader = null; + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java new file mode 100644 index 0000000000..b530f21712 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.util.HashSet; +import java.util.Set; + +import org.apache.hadoop.hive.metastore.TableType; + +/** + * HiveTableTypeMapping. + * Default table type mapping + * + */ +public class HiveTableTypeMapping implements TableTypeMapping { + + @Override + public String mapToHiveType(String clientTypeName) { + return clientTypeName; + } + + @Override + public String mapToClientType(String hiveTypeName) { + return hiveTypeName; + } + + @Override + public Set getTableTypeNames() { + Set typeNameSet = new HashSet(); + for (TableType typeNames : TableType.values()) { + typeNameSet.add(typeNames.toString()); + } + return typeNameSet; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java new file mode 100644 index 0000000000..70340bd13c --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java @@ -0,0 +1,209 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; +import java.io.CharArrayWriter; +import java.util.Enumeration; +import java.util.regex.Pattern; + +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.log.PerfLogger; +import org.apache.hadoop.hive.ql.session.OperationLog; +import org.apache.hadoop.hive.ql.session.OperationLog.LoggingLevel; +import org.apache.hive.service.cli.CLIServiceUtils; +import org.apache.log4j.Appender; +import org.apache.log4j.ConsoleAppender; +import org.apache.log4j.Layout; +import org.apache.log4j.Logger; +import org.apache.log4j.WriterAppender; +import org.apache.log4j.spi.Filter; +import org.apache.log4j.spi.LoggingEvent; + +import com.google.common.base.Joiner; + +/** + * An Appender to divert logs from individual threads to the LogObject they belong to. + */ +public class LogDivertAppender extends WriterAppender { + private static final Logger LOG = Logger.getLogger(LogDivertAppender.class.getName()); + private final OperationManager operationManager; + private boolean isVerbose; + private Layout verboseLayout; + + /** + * A log filter that filters messages coming from the logger with the given names. + * It be used as a white list filter or a black list filter. + * We apply black list filter on the Loggers used by the log diversion stuff, so that + * they don't generate more logs for themselves when they process logs. + * White list filter is used for less verbose log collection + */ + private static class NameFilter extends Filter { + private Pattern namePattern; + private LoggingLevel loggingMode; + private OperationManager operationManager; + + /* Patterns that are excluded in verbose logging level. + * Filter out messages coming from log processing classes, or we'll run an infinite loop. + */ + private static final Pattern verboseExcludeNamePattern = Pattern.compile(Joiner.on("|"). + join(new String[] {LOG.getName(), OperationLog.class.getName(), + OperationManager.class.getName()})); + + /* Patterns that are included in execution logging level. + * In execution mode, show only select logger messages. + */ + private static final Pattern executionIncludeNamePattern = Pattern.compile(Joiner.on("|"). + join(new String[] {"org.apache.hadoop.mapreduce.JobSubmitter", + "org.apache.hadoop.mapreduce.Job", "SessionState", Task.class.getName(), + "org.apache.hadoop.hive.ql.exec.spark.status.SparkJobMonitor"})); + + /* Patterns that are included in performance logging level. + * In performance mode, show execution and performance logger messages. + */ + private static final Pattern performanceIncludeNamePattern = Pattern.compile( + executionIncludeNamePattern.pattern() + "|" + PerfLogger.class.getName()); + + private void setCurrentNamePattern(OperationLog.LoggingLevel mode) { + if (mode == OperationLog.LoggingLevel.VERBOSE) { + this.namePattern = verboseExcludeNamePattern; + } else if (mode == OperationLog.LoggingLevel.EXECUTION) { + this.namePattern = executionIncludeNamePattern; + } else if (mode == OperationLog.LoggingLevel.PERFORMANCE) { + this.namePattern = performanceIncludeNamePattern; + } + } + + public NameFilter( + OperationLog.LoggingLevel loggingMode, OperationManager op) { + this.operationManager = op; + this.loggingMode = loggingMode; + setCurrentNamePattern(loggingMode); + } + + @Override + public int decide(LoggingEvent ev) { + OperationLog log = operationManager.getOperationLogByThread(); + boolean excludeMatches = (loggingMode == OperationLog.LoggingLevel.VERBOSE); + + if (log == null) { + return Filter.DENY; + } + + OperationLog.LoggingLevel currentLoggingMode = log.getOpLoggingLevel(); + // If logging is disabled, deny everything. + if (currentLoggingMode == OperationLog.LoggingLevel.NONE) { + return Filter.DENY; + } + // Look at the current session's setting + // and set the pattern and excludeMatches accordingly. + if (currentLoggingMode != loggingMode) { + loggingMode = currentLoggingMode; + setCurrentNamePattern(loggingMode); + } + + boolean isMatch = namePattern.matcher(ev.getLoggerName()).matches(); + + if (excludeMatches == isMatch) { + // Deny if this is black-list filter (excludeMatches = true) and it + // matched + // or if this is whitelist filter and it didn't match + return Filter.DENY; + } + return Filter.NEUTRAL; + } + } + + /** This is where the log message will go to */ + private final CharArrayWriter writer = new CharArrayWriter(); + + private void setLayout (boolean isVerbose, Layout lo) { + if (isVerbose) { + if (lo == null) { + lo = CLIServiceUtils.verboseLayout; + LOG.info("Cannot find a Layout from a ConsoleAppender. Using default Layout pattern."); + } + } else { + lo = CLIServiceUtils.nonVerboseLayout; + } + setLayout(lo); + } + + private void initLayout(boolean isVerbose) { + // There should be a ConsoleAppender. Copy its Layout. + Logger root = Logger.getRootLogger(); + Layout layout = null; + + Enumeration appenders = root.getAllAppenders(); + while (appenders.hasMoreElements()) { + Appender ap = (Appender) appenders.nextElement(); + if (ap.getClass().equals(ConsoleAppender.class)) { + layout = ap.getLayout(); + break; + } + } + setLayout(isVerbose, layout); + } + + public LogDivertAppender(OperationManager operationManager, + OperationLog.LoggingLevel loggingMode) { + isVerbose = (loggingMode == OperationLog.LoggingLevel.VERBOSE); + initLayout(isVerbose); + setWriter(writer); + setName("LogDivertAppender"); + this.operationManager = operationManager; + this.verboseLayout = isVerbose ? layout : CLIServiceUtils.verboseLayout; + addFilter(new NameFilter(loggingMode, operationManager)); + } + + @Override + public void doAppend(LoggingEvent event) { + OperationLog log = operationManager.getOperationLogByThread(); + + // Set current layout depending on the verbose/non-verbose mode. + if (log != null) { + boolean isCurrModeVerbose = (log.getOpLoggingLevel() == OperationLog.LoggingLevel.VERBOSE); + + // If there is a logging level change from verbose->non-verbose or vice-versa since + // the last subAppend call, change the layout to preserve consistency. + if (isCurrModeVerbose != isVerbose) { + isVerbose = isCurrModeVerbose; + setLayout(isVerbose, verboseLayout); + } + } + super.doAppend(event); + } + + /** + * Overrides WriterAppender.subAppend(), which does the real logging. No need + * to worry about concurrency since log4j calls this synchronously. + */ + @Override + protected void subAppend(LoggingEvent event) { + super.subAppend(event); + // That should've gone into our writer. Notify the LogContext. + String logOutput = writer.toString(); + writer.reset(); + + OperationLog log = operationManager.getOperationLogByThread(); + if (log == null) { + LOG.debug(" ---+++=== Dropped log event from thread " + event.getThreadName()); + return; + } + log.writeOperationLog(logOutput); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java new file mode 100644 index 0000000000..4595ef56fc --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java @@ -0,0 +1,135 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * MetadataOperation. + * + */ +public abstract class MetadataOperation extends Operation { + + protected static final String DEFAULT_HIVE_CATALOG = ""; + protected static TableSchema RESULT_SET_SCHEMA; + private static final char SEARCH_STRING_ESCAPE = '\\'; + + protected MetadataOperation(HiveSession parentSession, OperationType opType) { + super(parentSession, opType, false); + setHasResultSet(true); + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#close() + */ + @Override + public void close() throws HiveSQLException { + setState(OperationState.CLOSED); + cleanupOperationLog(); + } + + /** + * Convert wildchars and escape sequence from JDBC format to datanucleous/regex + */ + protected String convertIdentifierPattern(final String pattern, boolean datanucleusFormat) { + if (pattern == null) { + return convertPattern("%", true); + } else { + return convertPattern(pattern, datanucleusFormat); + } + } + + /** + * Convert wildchars and escape sequence of schema pattern from JDBC format to datanucleous/regex + * The schema pattern treats empty string also as wildchar + */ + protected String convertSchemaPattern(final String pattern) { + if ((pattern == null) || pattern.isEmpty()) { + return convertPattern("%", true); + } else { + return convertPattern(pattern, true); + } + } + + /** + * Convert a pattern containing JDBC catalog search wildcards into + * Java regex patterns. + * + * @param pattern input which may contain '%' or '_' wildcard characters, or + * these characters escaped using {@link #getSearchStringEscape()}. + * @return replace %/_ with regex search characters, also handle escaped + * characters. + * + * The datanucleus module expects the wildchar as '*'. The columns search on the + * other hand is done locally inside the hive code and that requires the regex wildchar + * format '.*' This is driven by the datanucleusFormat flag. + */ + private String convertPattern(final String pattern, boolean datanucleusFormat) { + String wStr; + if (datanucleusFormat) { + wStr = "*"; + } else { + wStr = ".*"; + } + return pattern + .replaceAll("([^\\\\])%", "$1" + wStr).replaceAll("\\\\%", "%").replaceAll("^%", wStr) + .replaceAll("([^\\\\])_", "$1.").replaceAll("\\\\_", "_").replaceAll("^_", "."); + } + + protected boolean isAuthV2Enabled(){ + SessionState ss = SessionState.get(); + return (ss.isAuthorizationModeV2() && + HiveConf.getBoolVar(ss.getConf(), HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)); + } + + protected void authorizeMetaGets(HiveOperationType opType, List inpObjs) + throws HiveSQLException { + authorizeMetaGets(opType, inpObjs, null); + } + + protected void authorizeMetaGets(HiveOperationType opType, List inpObjs, + String cmdString) throws HiveSQLException { + SessionState ss = SessionState.get(); + HiveAuthzContext.Builder ctxBuilder = new HiveAuthzContext.Builder(); + ctxBuilder.setUserIpAddress(ss.getUserIpAddress()); + ctxBuilder.setCommandString(cmdString); + try { + ss.getAuthorizerV2().checkPrivileges(opType, inpObjs, null, + ctxBuilder.build()); + } catch (HiveAuthzPluginException | HiveAccessControlException e) { + throw new HiveSQLException(e.getMessage(), e); + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java new file mode 100644 index 0000000000..19153b654b --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java @@ -0,0 +1,322 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.cli.operation; + +import java.io.File; +import java.io.FileNotFoundException; +import java.util.EnumSet; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.OperationLog; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationHandle; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationStatus; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; +import org.apache.hive.service.cli.thrift.TProtocolVersion; + +public abstract class Operation { + protected final HiveSession parentSession; + private OperationState state = OperationState.INITIALIZED; + private final OperationHandle opHandle; + private HiveConf configuration; + public static final Log LOG = LogFactory.getLog(Operation.class.getName()); + public static final FetchOrientation DEFAULT_FETCH_ORIENTATION = FetchOrientation.FETCH_NEXT; + 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; + + private long operationTimeout; + private long lastAccessTime; + + 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 = parentSession; + this.runAsync = runInBackground; + this.opHandle = new OperationHandle(opType, parentSession.getProtocolVersion()); + lastAccessTime = System.currentTimeMillis(); + operationTimeout = HiveConf.getTimeVar(parentSession.getHiveConf(), + HiveConf.ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT, TimeUnit.MILLISECONDS); + } + + public Future getBackgroundHandle() { + return backgroundHandle; + } + + protected void setBackgroundHandle(Future backgroundHandle) { + this.backgroundHandle = backgroundHandle; + } + + public boolean shouldRunAsync() { + return runAsync; + } + + public void setConfiguration(HiveConf configuration) { + this.configuration = new HiveConf(configuration); + } + + public HiveConf getConfiguration() { + return new HiveConf(configuration); + } + + public HiveSession getParentSession() { + return parentSession; + } + + public OperationHandle getHandle() { + return opHandle; + } + + public TProtocolVersion getProtocolVersion() { + return opHandle.getProtocolVersion(); + } + + public OperationType getType() { + return opHandle.getOperationType(); + } + + public OperationStatus getStatus() { + return new OperationStatus(state, operationException); + } + + public boolean hasResultSet() { + return hasResultSet; + } + + protected void setHasResultSet(boolean hasResultSet) { + this.hasResultSet = hasResultSet; + opHandle.setHasResultSet(hasResultSet); + } + + public OperationLog getOperationLog() { + return operationLog; + } + + protected final OperationState setState(OperationState newState) throws HiveSQLException { + state.validateTransition(newState); + this.state = newState; + this.lastAccessTime = System.currentTimeMillis(); + return this.state; + } + + public boolean isTimedOut(long current) { + if (operationTimeout == 0) { + return false; + } + if (operationTimeout > 0) { + // check only when it's in terminal state + return state.isTerminal() && lastAccessTime + operationTimeout <= current; + } + return lastAccessTime + -operationTimeout <= current; + } + + public long getLastAccessTime() { + return lastAccessTime; + } + + public long getOperationTimeout() { + return operationTimeout; + } + + public void setOperationTimeout(long operationTimeout) { + this.operationTimeout = operationTimeout; + } + + protected void setOperationException(HiveSQLException operationException) { + this.operationException = operationException; + } + + protected final void assertState(OperationState state) throws HiveSQLException { + if (this.state != state) { + throw new HiveSQLException("Expected state " + state + ", but found " + this.state); + } + this.lastAccessTime = System.currentTimeMillis(); + } + + public boolean isRunning() { + return OperationState.RUNNING.equals(state); + } + + public boolean isFinished() { + return OperationState.FINISHED.equals(state); + } + + public boolean isCanceled() { + return OperationState.CANCELED.equals(state); + } + + public boolean isFailed() { + return OperationState.ERROR.equals(state); + } + + protected void createOperationLog() { + if (parentSession.isOperationLogEnabled()) { + File operationLogFile = new File(parentSession.getOperationLogSessionDir(), + opHandle.getHandleIdentifier().toString()); + isOperationLogEnabled = true; + + // create log file + try { + if (operationLogFile.exists()) { + LOG.warn("The operation log file should not exist, but it is already there: " + + operationLogFile.getAbsolutePath()); + operationLogFile.delete(); + } + if (!operationLogFile.createNewFile()) { + // the log file already exists and cannot be deleted. + // If it can be read/written, keep its contents and use it. + if (!operationLogFile.canRead() || !operationLogFile.canWrite()) { + LOG.warn("The already existed operation log file cannot be recreated, " + + "and it cannot be read or written: " + operationLogFile.getAbsolutePath()); + isOperationLogEnabled = false; + return; + } + } + } catch (Exception e) { + LOG.warn("Unable to create operation log file: " + operationLogFile.getAbsolutePath(), e); + isOperationLogEnabled = false; + return; + } + + // create OperationLog object with above log file + try { + operationLog = new OperationLog(opHandle.toString(), operationLogFile, parentSession.getHiveConf()); + } catch (FileNotFoundException e) { + LOG.warn("Unable to instantiate OperationLog object for operation: " + + opHandle, e); + isOperationLogEnabled = false; + return; + } + + // register this operationLog to current thread + OperationLog.setCurrentOperationLog(operationLog); + } + } + + protected void unregisterOperationLog() { + if (isOperationLogEnabled) { + OperationLog.removeCurrentOperationLog(); + } + } + + /** + * Invoked before runInternal(). + * Set up some preconditions, or configurations. + */ + protected void beforeRun() { + createOperationLog(); + } + + /** + * Invoked after runInternal(), even if an exception is thrown in runInternal(). + * Clean up resources, which was set up in beforeRun(). + */ + protected void afterRun() { + unregisterOperationLog(); + } + + /** + * Implemented by subclass of Operation class to execute specific behaviors. + * @throws HiveSQLException + */ + protected abstract void runInternal() throws HiveSQLException; + + public void run() throws HiveSQLException { + beforeRun(); + try { + runInternal(); + } finally { + afterRun(); + } + } + + protected void cleanupOperationLog() { + if (isOperationLogEnabled) { + if (operationLog == null) { + LOG.error("Operation [ " + opHandle.getHandleIdentifier() + " ] " + + "logging is enabled, but its OperationLog object cannot be found."); + } else { + operationLog.close(); + } + } + } + + // TODO: make this abstract and implement in subclasses. + public void cancel() throws HiveSQLException { + setState(OperationState.CANCELED); + throw new UnsupportedOperationException("SQLOperation.cancel()"); + } + + public abstract void close() throws HiveSQLException; + + public abstract TableSchema getResultSetSchema() throws HiveSQLException; + + public abstract RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException; + + public RowSet getNextRowSet() throws HiveSQLException { + return getNextRowSet(FetchOrientation.FETCH_NEXT, DEFAULT_FETCH_MAX_ROWS); + } + + /** + * Verify if the given fetch orientation is part of the default orientation types. + * @param orientation + * @throws HiveSQLException + */ + protected void validateDefaultFetchOrientation(FetchOrientation orientation) + throws HiveSQLException { + validateFetchOrientation(orientation, DEFAULT_FETCH_ORIENTATION_SET); + } + + /** + * Verify if the given fetch orientation is part of the supported orientation types. + * @param orientation + * @param supportedOrientations + * @throws HiveSQLException + */ + protected void validateFetchOrientation(FetchOrientation orientation, + EnumSet supportedOrientations) throws HiveSQLException { + if (!supportedOrientations.contains(orientation)) { + throw new HiveSQLException("The fetch type " + orientation.toString() + + " is not supported for this resultset", "HY106"); + } + } + + protected HiveSQLException toSQLException(String prefix, CommandProcessorResponse response) { + HiveSQLException ex = new HiveSQLException(prefix + ": " + response.getErrorMessage(), + response.getSQLState(), response.getResponseCode()); + if (response.getException() != null) { + ex.initCause(response.getException()); + } + return ex; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java new file mode 100644 index 0000000000..92c340a29c --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java @@ -0,0 +1,284 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Schema; +import org.apache.hadoop.hive.ql.session.OperationLog; +import org.apache.hive.service.AbstractService; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationHandle; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationStatus; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; +import org.apache.log4j.Appender; +import org.apache.log4j.Logger; + +/** + * OperationManager. + * + */ +public class OperationManager extends AbstractService { + private final Log LOG = LogFactory.getLog(OperationManager.class.getName()); + + private final Map handleToOperation = + new HashMap(); + + public OperationManager() { + super(OperationManager.class.getSimpleName()); + } + + @Override + public synchronized void init(HiveConf hiveConf) { + if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED)) { + initOperationLogCapture(hiveConf.getVar( + HiveConf.ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL)); + } else { + LOG.debug("Operation level logging is turned off"); + } + super.init(hiveConf); + } + + @Override + public synchronized void start() { + super.start(); + // TODO + } + + @Override + public synchronized void stop() { + // TODO + super.stop(); + } + + private void initOperationLogCapture(String loggingMode) { + // Register another Appender (with the same layout) that talks to us. + Appender ap = new LogDivertAppender(this, OperationLog.getLoggingLevel(loggingMode)); + Logger.getRootLogger().addAppender(ap); + } + + public ExecuteStatementOperation newExecuteStatementOperation(HiveSession parentSession, + String statement, Map confOverlay, boolean runAsync) + throws HiveSQLException { + ExecuteStatementOperation executeStatementOperation = ExecuteStatementOperation + .newExecuteStatementOperation(parentSession, statement, confOverlay, runAsync); + addOperation(executeStatementOperation); + return executeStatementOperation; + } + + public GetTypeInfoOperation newGetTypeInfoOperation(HiveSession parentSession) { + GetTypeInfoOperation operation = new GetTypeInfoOperation(parentSession); + addOperation(operation); + return operation; + } + + public GetCatalogsOperation newGetCatalogsOperation(HiveSession parentSession) { + GetCatalogsOperation operation = new GetCatalogsOperation(parentSession); + addOperation(operation); + return operation; + } + + public GetSchemasOperation newGetSchemasOperation(HiveSession parentSession, + String catalogName, String schemaName) { + GetSchemasOperation operation = new GetSchemasOperation(parentSession, catalogName, schemaName); + addOperation(operation); + return operation; + } + + public MetadataOperation newGetTablesOperation(HiveSession parentSession, + String catalogName, String schemaName, String tableName, + List tableTypes) { + MetadataOperation operation = + new GetTablesOperation(parentSession, catalogName, schemaName, tableName, tableTypes); + addOperation(operation); + return operation; + } + + public GetTableTypesOperation newGetTableTypesOperation(HiveSession parentSession) { + GetTableTypesOperation operation = new GetTableTypesOperation(parentSession); + addOperation(operation); + return operation; + } + + public GetColumnsOperation newGetColumnsOperation(HiveSession parentSession, + String catalogName, String schemaName, String tableName, String columnName) { + GetColumnsOperation operation = new GetColumnsOperation(parentSession, + catalogName, schemaName, tableName, columnName); + addOperation(operation); + return operation; + } + + public GetFunctionsOperation newGetFunctionsOperation(HiveSession parentSession, + String catalogName, String schemaName, String functionName) { + GetFunctionsOperation operation = new GetFunctionsOperation(parentSession, + catalogName, schemaName, functionName); + addOperation(operation); + return operation; + } + + public Operation getOperation(OperationHandle operationHandle) throws HiveSQLException { + Operation operation = getOperationInternal(operationHandle); + if (operation == null) { + throw new HiveSQLException("Invalid OperationHandle: " + operationHandle); + } + return operation; + } + + private synchronized Operation getOperationInternal(OperationHandle operationHandle) { + return handleToOperation.get(operationHandle); + } + + private synchronized Operation removeTimedOutOperation(OperationHandle operationHandle) { + Operation operation = handleToOperation.get(operationHandle); + if (operation != null && operation.isTimedOut(System.currentTimeMillis())) { + handleToOperation.remove(operationHandle); + return operation; + } + return null; + } + + private synchronized void addOperation(Operation operation) { + handleToOperation.put(operation.getHandle(), operation); + } + + private synchronized Operation removeOperation(OperationHandle opHandle) { + return handleToOperation.remove(opHandle); + } + + public OperationStatus getOperationStatus(OperationHandle opHandle) + throws HiveSQLException { + return getOperation(opHandle).getStatus(); + } + + public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { + Operation operation = getOperation(opHandle); + OperationState opState = operation.getStatus().getState(); + if (opState == OperationState.CANCELED || + opState == OperationState.CLOSED || + opState == OperationState.FINISHED || + opState == OperationState.ERROR || + opState == OperationState.UNKNOWN) { + // Cancel should be a no-op in either cases + LOG.debug(opHandle + ": Operation is already aborted in state - " + opState); + } + else { + LOG.debug(opHandle + ": Attempting to cancel from state - " + opState); + operation.cancel(); + } + } + + public void closeOperation(OperationHandle opHandle) throws HiveSQLException { + Operation operation = removeOperation(opHandle); + if (operation == null) { + throw new HiveSQLException("Operation does not exist!"); + } + operation.close(); + } + + public TableSchema getOperationResultSetSchema(OperationHandle opHandle) + throws HiveSQLException { + return getOperation(opHandle).getResultSetSchema(); + } + + public RowSet getOperationNextRowSet(OperationHandle opHandle) + throws HiveSQLException { + return getOperation(opHandle).getNextRowSet(); + } + + public RowSet getOperationNextRowSet(OperationHandle opHandle, + FetchOrientation orientation, long maxRows) + throws HiveSQLException { + return getOperation(opHandle).getNextRowSet(orientation, maxRows); + } + + public RowSet getOperationLogRowSet(OperationHandle opHandle, + FetchOrientation orientation, long maxRows) + throws HiveSQLException { + // get the OperationLog object from the operation + OperationLog operationLog = getOperation(opHandle).getOperationLog(); + if (operationLog == null) { + throw new HiveSQLException("Couldn't find log associated with operation handle: " + opHandle); + } + + // read logs + List logs; + try { + logs = operationLog.readOperationLog(isFetchFirst(orientation), maxRows); + } catch (SQLException e) { + throw new HiveSQLException(e.getMessage(), e.getCause()); + } + + + // convert logs to RowSet + TableSchema tableSchema = new TableSchema(getLogSchema()); + RowSet rowSet = RowSetFactory.create(tableSchema, getOperation(opHandle).getProtocolVersion()); + for (String log : logs) { + rowSet.addRow(new String[] {log}); + } + + return rowSet; + } + + private boolean isFetchFirst(FetchOrientation fetchOrientation) { + //TODO: Since OperationLog is moved to package o.a.h.h.ql.session, + // we may add a Enum there and map FetchOrientation to it. + if (fetchOrientation.equals(FetchOrientation.FETCH_FIRST)) { + return true; + } + return false; + } + + private Schema getLogSchema() { + Schema schema = new Schema(); + FieldSchema fieldSchema = new FieldSchema(); + fieldSchema.setName("operation_log"); + fieldSchema.setType("string"); + schema.addToFieldSchemas(fieldSchema); + return schema; + } + + public OperationLog getOperationLogByThread() { + return OperationLog.getCurrentOperationLog(); + } + + public List removeExpiredOperations(OperationHandle[] handles) { + List removed = new ArrayList(); + for (OperationHandle handle : handles) { + Operation operation = removeTimedOutOperation(handle); + if (operation != null) { + LOG.warn("Operation " + handle + " is timed-out and will be closed"); + removed.add(operation); + } + } + return removed; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java new file mode 100644 index 0000000000..33ee16b80b --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java @@ -0,0 +1,473 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UnsupportedEncodingException; +import java.security.PrivilegedExceptionAction; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.Future; +import java.util.concurrent.RejectedExecutionException; + +import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Schema; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.exec.ExplainTask; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.VariableSubstitution; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.OperationLog; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.SerDeUtils; +import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.shims.Utils; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; +import org.apache.hive.service.server.ThreadWithGarbageCleanup; + +/** + * SQLOperation. + * + */ +public class SQLOperation extends ExecuteStatementOperation { + + private Driver driver = null; + private CommandProcessorResponse response; + private TableSchema resultSchema = null; + private Schema mResultSchema = null; + private SerDe serde = null; + private boolean fetchStarted = false; + + public SQLOperation(HiveSession parentSession, String statement, Map confOverlay, boolean runInBackground) { + // TODO: call setRemoteUser in ExecuteStatementOperation or higher. + super(parentSession, statement, confOverlay, runInBackground); + } + + /*** + * Compile the query and extract metadata + * @param sqlOperationConf + * @throws HiveSQLException + */ + public void prepare(HiveConf sqlOperationConf) throws HiveSQLException { + setState(OperationState.RUNNING); + + try { + driver = new Driver(sqlOperationConf, getParentSession().getUserName()); + + // set the operation handle information in Driver, so that thrift API users + // can use the operation handle they receive, to lookup query information in + // Yarn ATS + String guid64 = Base64.encodeBase64URLSafeString(getHandle().getHandleIdentifier() + .toTHandleIdentifier().getGuid()).trim(); + driver.setOperationId(guid64); + + // In Hive server mode, we are not able to retry in the FetchTask + // case, when calling fetch queries since execute() has returned. + // For now, we disable the test attempts. + driver.setTryCount(Integer.MAX_VALUE); + + String subStatement = new VariableSubstitution().substitute(sqlOperationConf, statement); + response = driver.compileAndRespond(subStatement); + if (0 != response.getResponseCode()) { + throw toSQLException("Error while compiling statement", response); + } + + mResultSchema = driver.getSchema(); + + // hasResultSet should be true only if the query has a FetchTask + // "explain" is an exception for now + if(driver.getPlan().getFetchTask() != null) { + //Schema has to be set + if (mResultSchema == null || !mResultSchema.isSetFieldSchemas()) { + throw new HiveSQLException("Error compiling query: Schema and FieldSchema " + + "should be set when query plan has a FetchTask"); + } + resultSchema = new TableSchema(mResultSchema); + setHasResultSet(true); + } else { + setHasResultSet(false); + } + // Set hasResultSet true if the plan has ExplainTask + // TODO explain should use a FetchTask for reading + for (Task task: driver.getPlan().getRootTasks()) { + if (task.getClass() == ExplainTask.class) { + resultSchema = new TableSchema(mResultSchema); + setHasResultSet(true); + break; + } + } + } catch (HiveSQLException e) { + setState(OperationState.ERROR); + throw e; + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException("Error running query: " + e.toString(), e); + } + } + + private void runQuery(HiveConf sqlOperationConf) 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. + // For now, we disable the test attempts. + driver.setTryCount(Integer.MAX_VALUE); + response = driver.run(); + if (0 != response.getResponseCode()) { + throw toSQLException("Error while processing statement", response); + } + } catch (HiveSQLException e) { + // If the operation was cancelled by another thread, + // Driver#run will return a non-zero response code. + // We will simply return if the operation state is CANCELED, + // otherwise throw an exception + if (getStatus().getState() == OperationState.CANCELED) { + return; + } + else { + setState(OperationState.ERROR); + throw e; + } + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException("Error running query: " + e.toString(), e); + } + setState(OperationState.FINISHED); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.PENDING); + final HiveConf opConfig = getConfigForOperation(); + prepare(opConfig); + if (!shouldRunAsync()) { + runQuery(opConfig); + } else { + // We'll pass ThreadLocals in the background thread from the foreground (handler) thread + final SessionState parentSessionState = SessionState.get(); + // ThreadLocal Hive object needs to be set in background thread. + // The metastore client in Hive is associated with right user. + final Hive parentHive = 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); + // Runnable impl to call runInternal asynchronously, + // from a different thread + Runnable backgroundOperation = new Runnable() { + @Override + public void run() { + PrivilegedExceptionAction doAsAction = new PrivilegedExceptionAction() { + @Override + public Object run() throws HiveSQLException { + Hive.set(parentHive); + SessionState.setCurrentSessionState(parentSessionState); + // Set current OperationLog in this async thread for keeping on saving query log. + registerCurrentOperationLog(); + try { + runQuery(opConfig); + } catch (HiveSQLException e) { + setOperationException(e); + LOG.error("Error running hive query: ", e); + } finally { + 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().getSessionManager().submitBackgroundOperation(backgroundOperation); + setBackgroundHandle(backgroundHandle); + } catch (RejectedExecutionException rejected) { + setState(OperationState.ERROR); + throw new HiveSQLException("The background threadpool cannot accept" + + " new task for execution, please retry the operation", rejected); + } + } + } + + /** + * Returns the current UGI on the stack + * @param opConfig + * @return UserGroupInformation + * @throws HiveSQLException + */ + private UserGroupInformation getCurrentUGI(HiveConf opConfig) throws HiveSQLException { + try { + return Utils.getUGI(); + } catch (Exception e) { + throw new HiveSQLException("Unable to get current user", e); + } + } + + /** + * Returns the ThreadLocal Hive for the current thread + * @return Hive + * @throws HiveSQLException + */ + private Hive getSessionHive() throws HiveSQLException { + try { + return Hive.get(); + } catch (HiveException e) { + throw new HiveSQLException("Failed to get ThreadLocal Hive object", e); + } + } + + private void registerCurrentOperationLog() { + if (isOperationLogEnabled) { + if (operationLog == null) { + LOG.warn("Failed to get current OperationLog object of Operation: " + + getHandle().getHandleIdentifier()); + isOperationLogEnabled = false; + return; + } + OperationLog.setCurrentOperationLog(operationLog); + } + } + + private void cleanup(OperationState state) throws HiveSQLException { + setState(state); + if (shouldRunAsync()) { + Future backgroundHandle = getBackgroundHandle(); + if (backgroundHandle != null) { + backgroundHandle.cancel(true); + } + } + if (driver != null) { + driver.close(); + driver.destroy(); + } + driver = null; + + SessionState ss = SessionState.get(); + if (ss.getTmpOutputFile() != null) { + ss.getTmpOutputFile().delete(); + } + } + + @Override + public void cancel() throws HiveSQLException { + cleanup(OperationState.CANCELED); + } + + @Override + public void close() throws HiveSQLException { + cleanup(OperationState.CLOSED); + cleanupOperationLog(); + } + + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + if (resultSchema == null) { + resultSchema = new TableSchema(driver.getSchema()); + } + return resultSchema; + } + + private transient final List convey = new ArrayList(); + + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + validateDefaultFetchOrientation(orientation); + assertState(OperationState.FINISHED); + + RowSet rowSet = RowSetFactory.create(resultSchema, getProtocolVersion()); + + try { + /* if client is requesting fetch-from-start and its not the first time reading from this operation + * then reset the fetch position to beginning + */ + if (orientation.equals(FetchOrientation.FETCH_FIRST) && fetchStarted) { + driver.resetFetch(); + } + fetchStarted = true; + driver.setMaxRows((int) maxRows); + if (driver.getResults(convey)) { + return decode(convey, rowSet); + } + return rowSet; + } catch (IOException e) { + throw new HiveSQLException(e); + } catch (CommandNeedRetryException e) { + throw new HiveSQLException(e); + } catch (Exception e) { + throw new HiveSQLException(e); + } finally { + convey.clear(); + } + } + + private RowSet decode(List rows, RowSet rowSet) throws Exception { + if (driver.isFetchingTable()) { + return prepareFromRow(rows, rowSet); + } + return decodeFromString(rows, rowSet); + } + + // already encoded to thrift-able object in ThriftFormatter + private RowSet prepareFromRow(List rows, RowSet rowSet) throws Exception { + for (Object row : rows) { + rowSet.addRow((Object[]) row); + } + return rowSet; + } + + private RowSet decodeFromString(List rows, RowSet rowSet) + throws SQLException, SerDeException { + getSerDe(); + StructObjectInspector soi = (StructObjectInspector) serde.getObjectInspector(); + List fieldRefs = soi.getAllStructFieldRefs(); + + Object[] deserializedFields = new Object[fieldRefs.size()]; + Object rowObj; + ObjectInspector fieldOI; + + int protocol = getProtocolVersion().getValue(); + for (Object rowString : rows) { + try { + rowObj = serde.deserialize(new BytesWritable(((String)rowString).getBytes("UTF-8"))); + } catch (UnsupportedEncodingException e) { + throw new SerDeException(e); + } + for (int i = 0; i < fieldRefs.size(); i++) { + StructField fieldRef = fieldRefs.get(i); + fieldOI = fieldRef.getFieldObjectInspector(); + Object fieldData = soi.getStructFieldData(rowObj, fieldRef); + deserializedFields[i] = SerDeUtils.toThriftPayload(fieldData, fieldOI, protocol); + } + rowSet.addRow(deserializedFields); + } + return rowSet; + } + + private SerDe getSerDe() throws SQLException { + if (serde != null) { + return serde; + } + try { + List fieldSchemas = mResultSchema.getFieldSchemas(); + StringBuilder namesSb = new StringBuilder(); + StringBuilder typesSb = new StringBuilder(); + + if (fieldSchemas != null && !fieldSchemas.isEmpty()) { + for (int pos = 0; pos < fieldSchemas.size(); pos++) { + if (pos != 0) { + namesSb.append(","); + typesSb.append(","); + } + namesSb.append(fieldSchemas.get(pos).getName()); + typesSb.append(fieldSchemas.get(pos).getType()); + } + } + String names = namesSb.toString(); + String types = typesSb.toString(); + + serde = new LazySimpleSerDe(); + Properties props = new Properties(); + if (names.length() > 0) { + LOG.debug("Column names: " + names); + props.setProperty(serdeConstants.LIST_COLUMNS, names); + } + if (types.length() > 0) { + LOG.debug("Column types: " + types); + props.setProperty(serdeConstants.LIST_COLUMN_TYPES, types); + } + SerDeUtils.initializeSerDe(serde, new HiveConf(), props, null); + + } catch (Exception ex) { + ex.printStackTrace(); + throw new SQLException("Could not create ResultSet: " + ex.getMessage(), ex); + } + return serde; + } + + /** + * 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 + */ + private HiveConf getConfigForOperation() throws HiveSQLException { + HiveConf sqlOperationConf = getParentSession().getHiveConf(); + if (!getConfOverlay().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 : getConfOverlay().entrySet()) { + try { + sqlOperationConf.verifyAndSet(confEntry.getKey(), confEntry.getValue()); + } catch (IllegalArgumentException e) { + throw new HiveSQLException("Error applying statement specific settings", e); + } + } + } + return sqlOperationConf; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java new file mode 100644 index 0000000000..3a8a07f44f --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.util.Set; + + +public interface TableTypeMapping { + /** + * Map client's table type name to hive's table type + * @param clientTypeName + * @return + */ + public String mapToHiveType (String clientTypeName); + + /** + * Map hive's table type name to client's table type + * @param clientTypeName + * @return + */ + public String mapToClientType (String hiveTypeName); + + /** + * Get all the table types of this mapping + * @return + */ + public Set getTableTypeNames(); +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMappingFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMappingFactory.java new file mode 100644 index 0000000000..d8ac2696b3 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMappingFactory.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +public class TableTypeMappingFactory { + + public enum TableTypeMappings { + HIVE, + CLASSIC + } + private static TableTypeMapping hiveTableTypeMapping = new HiveTableTypeMapping(); + private static TableTypeMapping classicTableTypeMapping = new ClassicTableTypeMapping(); + + public static TableTypeMapping getTableTypeMapping(String mappingType) { + if (TableTypeMappings.CLASSIC.toString().equalsIgnoreCase(mappingType)) { + return classicTableTypeMapping; + } else { + return hiveTableTypeMapping; + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSession.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSession.java new file mode 100644 index 0000000000..65f9b298bf --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSession.java @@ -0,0 +1,156 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.*; + +public interface HiveSession extends HiveSessionBase { + + void open(Map sessionConfMap) throws Exception; + + IMetaStoreClient getMetaStoreClient() throws HiveSQLException; + + /** + * getInfo operation handler + * @param getInfoType + * @return + * @throws HiveSQLException + */ + GetInfoValue getInfo(GetInfoType getInfoType) throws HiveSQLException; + + /** + * execute operation handler + * @param statement + * @param confOverlay + * @return + * @throws HiveSQLException + */ + OperationHandle executeStatement(String statement, + Map confOverlay) throws HiveSQLException; + + /** + * execute operation handler + * @param statement + * @param confOverlay + * @return + * @throws HiveSQLException + */ + OperationHandle executeStatementAsync(String statement, + Map confOverlay) throws HiveSQLException; + + /** + * getTypeInfo operation handler + * @return + * @throws HiveSQLException + */ + OperationHandle getTypeInfo() throws HiveSQLException; + + /** + * getCatalogs operation handler + * @return + * @throws HiveSQLException + */ + OperationHandle getCatalogs() throws HiveSQLException; + + /** + * getSchemas operation handler + * @param catalogName + * @param schemaName + * @return + * @throws HiveSQLException + */ + OperationHandle getSchemas(String catalogName, String schemaName) + throws HiveSQLException; + + /** + * getTables operation handler + * @param catalogName + * @param schemaName + * @param tableName + * @param tableTypes + * @return + * @throws HiveSQLException + */ + OperationHandle getTables(String catalogName, String schemaName, + String tableName, List tableTypes) throws HiveSQLException; + + /** + * getTableTypes operation handler + * @return + * @throws HiveSQLException + */ + OperationHandle getTableTypes() throws HiveSQLException ; + + /** + * getColumns operation handler + * @param catalogName + * @param schemaName + * @param tableName + * @param columnName + * @return + * @throws HiveSQLException + */ + OperationHandle getColumns(String catalogName, String schemaName, + String tableName, String columnName) throws HiveSQLException; + + /** + * getFunctions operation handler + * @param catalogName + * @param schemaName + * @param functionName + * @return + * @throws HiveSQLException + */ + OperationHandle getFunctions(String catalogName, String schemaName, + String functionName) throws HiveSQLException; + + /** + * close the session + * @throws HiveSQLException + */ + void close() throws HiveSQLException; + + void cancelOperation(OperationHandle opHandle) throws HiveSQLException; + + void closeOperation(OperationHandle opHandle) throws HiveSQLException; + + TableSchema getResultSetMetadata(OperationHandle opHandle) + throws HiveSQLException; + + RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, + long maxRows, FetchType fetchType) throws HiveSQLException; + + String getDelegationToken(HiveAuthFactory authFactory, String owner, + String renewer) throws HiveSQLException; + + void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException; + + void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException; + + void closeExpiredOperations(); + + long getNoOperationTime(); +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java new file mode 100644 index 0000000000..9b04d679df --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java @@ -0,0 +1,93 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +import java.util.Map; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hive.service.cli.SessionHandle; +import org.apache.hive.service.cli.operation.OperationManager; +import org.apache.hive.service.cli.thrift.TProtocolVersion; + +import java.io.File; +import java.util.Map; + +/** + * Methods that don't need to be executed under a doAs + * context are here. Rest of them in HiveSession interface + */ +public interface HiveSessionBase { + + TProtocolVersion getProtocolVersion(); + + /** + * Set the session manager for the session + * @param sessionManager + */ + void setSessionManager(SessionManager sessionManager); + + /** + * Get the session manager for the session + */ + SessionManager getSessionManager(); + + /** + * Set operation manager for the session + * @param operationManager + */ + void setOperationManager(OperationManager operationManager); + + /** + * Check whether operation logging is enabled and session dir is created successfully + */ + boolean isOperationLogEnabled(); + + /** + * Get the session dir, which is the parent dir of operation logs + * @return a file representing the parent directory of operation logs + */ + File getOperationLogSessionDir(); + + /** + * Set the session dir, which is the parent dir of operation logs + * @param operationLogRootDir the parent dir of the session dir + */ + void setOperationLogSessionDir(File operationLogRootDir); + + SessionHandle getSessionHandle(); + + String getUsername(); + + String getPassword(); + + HiveConf getHiveConf(); + + SessionState getSessionState(); + + String getUserName(); + + void setUserName(String userName); + + String getIpAddress(); + + void setIpAddress(String ipAddress); + + long getLastAccessTime(); +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java new file mode 100644 index 0000000000..06388cc795 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +import org.apache.hadoop.hive.ql.hooks.Hook; +import org.apache.hive.service.cli.HiveSQLException; + +/** + * HiveSessionHook. + * HiveServer2 session level Hook interface. The run method is executed + * when session manager starts a new session + * + */ +public interface HiveSessionHook extends Hook { + + /** + * @param sessionHookContext context + * @throws HiveSQLException + */ + public void run(HiveSessionHookContext sessionHookContext) throws HiveSQLException; +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java new file mode 100644 index 0000000000..156c8147f9 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +import org.apache.hadoop.hive.conf.HiveConf; +/** + * HiveSessionHookContext. + * Interface passed to the HiveServer2 session hook execution. This enables + * the hook implementation to accesss session config, user and session handle + */ +public interface HiveSessionHookContext { + + /** + * Retrieve session conf + * @return + */ + public HiveConf getSessionConf(); + + /** + * The get the username starting the session + * @return + */ + public String getSessionUser(); + + /** + * Retrieve handle for the session + * @return + */ + public String getSessionHandle(); +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java new file mode 100644 index 0000000000..1ee4ac8a1d --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +import org.apache.hadoop.hive.conf.HiveConf; + +/** + * + * HiveSessionHookContextImpl. + * Session hook context implementation which is created by session manager + * and passed to hook invocation. + */ +public class HiveSessionHookContextImpl implements HiveSessionHookContext { + + private final HiveSession hiveSession; + + HiveSessionHookContextImpl(HiveSession hiveSession) { + this.hiveSession = hiveSession; + } + + @Override + public HiveConf getSessionConf() { + return hiveSession.getHiveConf(); + } + + + @Override + public String getSessionUser() { + return hiveSession.getUserName(); + } + + @Override + public String getSessionHandle() { + return hiveSession.getSessionHandle().toString(); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java new file mode 100644 index 0000000000..cc3e807e7a --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java @@ -0,0 +1,734 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.common.cli.HiveFileProcessor; +import org.apache.hadoop.hive.common.cli.IHiveFileProcessor; +import org.apache.hadoop.hive.conf.HiveConf; +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.exec.FetchFormatter; +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; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.processors.SetProcessor; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hive.common.util.HiveVersionInfo; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.FetchType; +import org.apache.hive.service.cli.GetInfoType; +import org.apache.hive.service.cli.GetInfoValue; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationHandle; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.SessionHandle; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.operation.ExecuteStatementOperation; +import org.apache.hive.service.cli.operation.GetCatalogsOperation; +import org.apache.hive.service.cli.operation.GetColumnsOperation; +import org.apache.hive.service.cli.operation.GetFunctionsOperation; +import org.apache.hive.service.cli.operation.GetSchemasOperation; +import org.apache.hive.service.cli.operation.GetTableTypesOperation; +import org.apache.hive.service.cli.operation.GetTypeInfoOperation; +import org.apache.hive.service.cli.operation.MetadataOperation; +import org.apache.hive.service.cli.operation.Operation; +import org.apache.hive.service.cli.operation.OperationManager; +import org.apache.hive.service.cli.thrift.TProtocolVersion; +import org.apache.hive.service.server.ThreadWithGarbageCleanup; + +/** + * HiveSession + * + */ +public class HiveSessionImpl implements HiveSession { + private final SessionHandle sessionHandle; + private String username; + private final String password; + private HiveConf hiveConf; + private SessionState sessionState; + private String ipAddress; + private static final String FETCH_WORK_SERDE_CLASS = + "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"; + private static final Log LOG = LogFactory.getLog(HiveSessionImpl.class); + private SessionManager sessionManager; + private OperationManager operationManager; + private final Set opHandleSet = new HashSet(); + private boolean isOperationLogEnabled; + private File sessionLogDir; + private volatile long lastAccessTime; + private volatile long lastIdleTime; + + public HiveSessionImpl(TProtocolVersion protocol, String username, String password, + HiveConf serverhiveConf, String ipAddress) { + this.username = username; + this.password = password; + this.sessionHandle = new SessionHandle(protocol); + this.hiveConf = 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); + } + } 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, + sessionHandle.getHandleIdentifier().toString()); + // Use thrift transportable formatter + hiveConf.set(ListSinkOperator.OUTPUT_FORMATTER, + FetchFormatter.ThriftFormatter.class.getName()); + hiveConf.setInt(ListSinkOperator.OUTPUT_PROTOCOL, protocol.getValue()); + } + + @Override + /** + * Opens a new HiveServer2 session for the client connection. + * Creates a new SessionState object that will be associated with this HiveServer2 session. + * When the server executes multiple queries in the same session, + * this SessionState object is reused across multiple queries. + * Note that if doAs is true, this call goes through a proxy object, + * which wraps the method logic in a UserGroupInformation#doAs. + * 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.setUserIpAddress(ipAddress); + sessionState.setIsHiveServerQuery(true); + SessionState.start(sessionState); + try { + sessionState.reloadAuxJars(); + } catch (IOException e) { + String msg = "Failed to load reloadable jar file path: " + e; + LOG.error(msg, e); + throw new HiveSQLException(msg, e); + } + // Process global init file: .hiverc + processGlobalInitFile(); + if (sessionConfMap != null) { + configureSession(sessionConfMap); + } + lastAccessTime = System.currentTimeMillis(); + lastIdleTime = lastAccessTime; + } + + /** + * It is used for processing hiverc file from HiveServer2 side. + */ + private class GlobalHivercFileProcessor extends HiveFileProcessor { + @Override + protected BufferedReader loadFile(String fileName) throws IOException { + FileInputStream initStream = null; + BufferedReader bufferedReader = null; + initStream = new FileInputStream(fileName); + bufferedReader = new BufferedReader(new InputStreamReader(initStream)); + return bufferedReader; + } + + @Override + protected int processCmd(String cmd) { + int rc = 0; + String cmd_trimed = cmd.trim(); + try { + executeStatementInternal(cmd_trimed, null, false); + } catch (HiveSQLException e) { + rc = -1; + LOG.warn("Failed to execute HQL command in global .hiverc file.", e); + } + return rc; + } + } + + private void processGlobalInitFile() { + IHiveFileProcessor processor = new GlobalHivercFileProcessor(); + + try { + String hiverc = hiveConf.getVar(ConfVars.HIVE_SERVER2_GLOBAL_INIT_FILE_LOCATION); + if (hiverc != null) { + File hivercFile = new File(hiverc); + if (hivercFile.isDirectory()) { + hivercFile = new File(hivercFile, SessionManager.HIVERCFILE); + } + if (hivercFile.isFile()) { + LOG.info("Running global init file: " + hivercFile); + int rc = processor.processFile(hivercFile.getAbsolutePath()); + if (rc != 0) { + LOG.error("Failed on initializing global .hiverc file"); + } + } else { + LOG.debug("Global init file " + hivercFile + " does not exist"); + } + } + } catch (IOException e) { + LOG.warn("Failed on initializing global .hiverc file", e); + } + } + + private void configureSession(Map sessionConfMap) throws HiveSQLException { + SessionState.setCurrentSessionState(sessionState); + for (Map.Entry entry : sessionConfMap.entrySet()) { + String key = entry.getKey(); + if (key.startsWith("set:")) { + try { + SetProcessor.setVariable(key.substring(4), entry.getValue()); + } catch (Exception e) { + throw new HiveSQLException(e); + } + } else if (key.startsWith("use:")) { + SessionState.get().setCurrentDatabase(entry.getValue()); + } else { + hiveConf.verifyAndSet(key, entry.getValue()); + } + } + } + + @Override + public void setOperationLogSessionDir(File operationLogRootDir) { + sessionLogDir = new File(operationLogRootDir, sessionHandle.getHandleIdentifier().toString()); + isOperationLogEnabled = true; + if (!sessionLogDir.exists()) { + if (!sessionLogDir.mkdir()) { + LOG.warn("Unable to create operation log session directory: " + + sessionLogDir.getAbsolutePath()); + isOperationLogEnabled = false; + } + } + if (isOperationLogEnabled) { + LOG.info("Operation log session directory is created: " + sessionLogDir.getAbsolutePath()); + } + } + + @Override + public boolean isOperationLogEnabled() { + return isOperationLogEnabled; + } + + @Override + public File getOperationLogSessionDir() { + return sessionLogDir; + } + + @Override + public TProtocolVersion getProtocolVersion() { + return sessionHandle.getProtocolVersion(); + } + + @Override + public SessionManager getSessionManager() { + return sessionManager; + } + + @Override + public void setSessionManager(SessionManager sessionManager) { + this.sessionManager = sessionManager; + } + + private OperationManager getOperationManager() { + return operationManager; + } + + @Override + public void setOperationManager(OperationManager operationManager) { + this.operationManager = operationManager; + } + + 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); + if (userAccess) { + lastAccessTime = System.currentTimeMillis(); + } + } + + /** + * 1. We'll remove the ThreadLocal SessionState as this thread might now serve + * other requests. + * 2. 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() + */ + protected synchronized void release(boolean userAccess) { + SessionState.detachSession(); + if (ThreadWithGarbageCleanup.currentThread() instanceof ThreadWithGarbageCleanup) { + ThreadWithGarbageCleanup currentThread = + (ThreadWithGarbageCleanup) ThreadWithGarbageCleanup.currentThread(); + currentThread.cacheThreadLocalRawStore(); + } + if (userAccess) { + lastAccessTime = System.currentTimeMillis(); + } + if (opHandleSet.isEmpty()) { + lastIdleTime = System.currentTimeMillis(); + } else { + lastIdleTime = 0; + } + } + + @Override + public SessionHandle getSessionHandle() { + return sessionHandle; + } + + @Override + public String getUsername() { + return username; + } + + @Override + public String getPassword() { + return password; + } + + @Override + public HiveConf getHiveConf() { + hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHOUTPUTSERDE, FETCH_WORK_SERDE_CLASS); + return hiveConf; + } + + @Override + public IMetaStoreClient getMetaStoreClient() throws HiveSQLException { + try { + return Hive.get(getHiveConf()).getMSC(); + } catch (HiveException e) { + throw new HiveSQLException("Failed to get metastore connection", e); + } catch (MetaException e) { + throw new HiveSQLException("Failed to get metastore connection", e); + } + } + + @Override + public GetInfoValue getInfo(GetInfoType getInfoType) + throws HiveSQLException { + acquire(true); + try { + switch (getInfoType) { + case CLI_SERVER_NAME: + return new GetInfoValue("Hive"); + case CLI_DBMS_NAME: + return new GetInfoValue("Apache Hive"); + case CLI_DBMS_VER: + return new GetInfoValue(HiveVersionInfo.getVersion()); + case CLI_MAX_COLUMN_NAME_LEN: + return new GetInfoValue(128); + case CLI_MAX_SCHEMA_NAME_LEN: + return new GetInfoValue(128); + case CLI_MAX_TABLE_NAME_LEN: + return new GetInfoValue(128); + case CLI_TXN_CAPABLE: + default: + throw new HiveSQLException("Unrecognized GetInfoType value: " + getInfoType.toString()); + } + } finally { + release(true); + } + } + + @Override + public OperationHandle executeStatement(String statement, Map confOverlay) + throws HiveSQLException { + return executeStatementInternal(statement, confOverlay, false); + } + + @Override + public OperationHandle executeStatementAsync(String statement, Map confOverlay) + throws HiveSQLException { + return executeStatementInternal(statement, confOverlay, true); + } + + private OperationHandle executeStatementInternal(String statement, Map confOverlay, + boolean runAsync) + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + ExecuteStatementOperation operation = operationManager + .newExecuteStatementOperation(getSession(), statement, confOverlay, runAsync); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + // Refering to SQLOperation.java,there is no chance that a HiveSQLException throws and the asyn + // background operation submits to thread pool successfully at the same time. So, Cleanup + // opHandle directly when got HiveSQLException + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getTypeInfo() + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetTypeInfoOperation operation = operationManager.newGetTypeInfoOperation(getSession()); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getCatalogs() + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetCatalogsOperation operation = operationManager.newGetCatalogsOperation(getSession()); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getSchemas(String catalogName, String schemaName) + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetSchemasOperation operation = + operationManager.newGetSchemasOperation(getSession(), catalogName, schemaName); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getTables(String catalogName, String schemaName, String tableName, + List tableTypes) + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + MetadataOperation operation = + operationManager.newGetTablesOperation(getSession(), catalogName, schemaName, tableName, tableTypes); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getTableTypes() + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetTableTypesOperation operation = operationManager.newGetTableTypesOperation(getSession()); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getColumns(String catalogName, String schemaName, + String tableName, String columnName) throws HiveSQLException { + acquire(true); + String addedJars = Utilities.getResourceFiles(hiveConf, SessionState.ResourceType.JAR); + if (StringUtils.isNotBlank(addedJars)) { + IMetaStoreClient metastoreClient = getSession().getMetaStoreClient(); + metastoreClient.setHiveAddedJars(addedJars); + } + OperationManager operationManager = getOperationManager(); + GetColumnsOperation operation = operationManager.newGetColumnsOperation(getSession(), + catalogName, schemaName, tableName, columnName); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getFunctions(String catalogName, String schemaName, String functionName) + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetFunctionsOperation operation = operationManager + .newGetFunctionsOperation(getSession(), catalogName, schemaName, functionName); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public void close() throws HiveSQLException { + try { + acquire(true); + // Iterate through the opHandles and close their operations + for (OperationHandle opHandle : opHandleSet) { + operationManager.closeOperation(opHandle); + } + opHandleSet.clear(); + // Cleanup session log directory. + cleanupSessionLogDir(); + HiveHistory hiveHist = sessionState.getHiveHistory(); + if (null != hiveHist) { + hiveHist.closeStream(); + } + try { + sessionState.close(); + } finally { + sessionState = null; + } + } catch (IOException ioe) { + throw new HiveSQLException("Failure to close", ioe); + } finally { + if (sessionState != null) { + try { + sessionState.close(); + } catch (Throwable t) { + LOG.warn("Error closing session", t); + } + sessionState = null; + } + release(true); + } + } + + private void cleanupSessionLogDir() { + if (isOperationLogEnabled) { + try { + FileUtils.forceDelete(sessionLogDir); + } catch (Exception e) { + LOG.error("Failed to cleanup session log dir: " + sessionHandle, e); + } + } + } + + @Override + public SessionState getSessionState() { + return sessionState; + } + + @Override + public String getUserName() { + return username; + } + + @Override + public void setUserName(String userName) { + this.username = userName; + } + + @Override + public long getLastAccessTime() { + return lastAccessTime; + } + + @Override + public void closeExpiredOperations() { + OperationHandle[] handles = opHandleSet.toArray(new OperationHandle[opHandleSet.size()]); + if (handles.length > 0) { + List operations = operationManager.removeExpiredOperations(handles); + if (!operations.isEmpty()) { + closeTimedOutOperations(operations); + } + } + } + + @Override + public long getNoOperationTime() { + return lastIdleTime > 0 ? System.currentTimeMillis() - lastIdleTime : 0; + } + + private void closeTimedOutOperations(List operations) { + acquire(false); + try { + for (Operation operation : operations) { + opHandleSet.remove(operation.getHandle()); + try { + operation.close(); + } catch (Exception e) { + LOG.warn("Exception is thrown closing timed-out operation " + operation.getHandle(), e); + } + } + } finally { + release(false); + } + } + + @Override + public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { + acquire(true); + try { + sessionManager.getOperationManager().cancelOperation(opHandle); + } finally { + release(true); + } + } + + @Override + public void closeOperation(OperationHandle opHandle) throws HiveSQLException { + acquire(true); + try { + operationManager.closeOperation(opHandle); + opHandleSet.remove(opHandle); + } finally { + release(true); + } + } + + @Override + public TableSchema getResultSetMetadata(OperationHandle opHandle) throws HiveSQLException { + acquire(true); + try { + return sessionManager.getOperationManager().getOperationResultSetSchema(opHandle); + } finally { + release(true); + } + } + + @Override + public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, + long maxRows, FetchType fetchType) throws HiveSQLException { + acquire(true); + try { + if (fetchType == FetchType.QUERY_OUTPUT) { + return operationManager.getOperationNextRowSet(opHandle, orientation, maxRows); + } + return operationManager.getOperationLogRowSet(opHandle, orientation, maxRows); + } finally { + release(true); + } + } + + protected HiveSession getSession() { + return this; + } + + @Override + public String getIpAddress() { + return ipAddress; + } + + @Override + public void setIpAddress(String ipAddress) { + this.ipAddress = ipAddress; + } + + @Override + public String getDelegationToken(HiveAuthFactory authFactory, String owner, String renewer) + throws HiveSQLException { + HiveAuthFactory.verifyProxyAccess(getUsername(), owner, getIpAddress(), getHiveConf()); + return authFactory.getDelegationToken(owner, renewer); + } + + @Override + public void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException { + HiveAuthFactory.verifyProxyAccess(getUsername(), getUserFromToken(authFactory, tokenStr), + getIpAddress(), getHiveConf()); + authFactory.cancelDelegationToken(tokenStr); + } + + @Override + public void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException { + HiveAuthFactory.verifyProxyAccess(getUsername(), getUserFromToken(authFactory, tokenStr), + getIpAddress(), getHiveConf()); + authFactory.renewDelegationToken(tokenStr); + } + + // extract the real user from the given token string + private String getUserFromToken(HiveAuthFactory authFactory, String tokenStr) throws HiveSQLException { + return authFactory.getUserFromToken(tokenStr); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java new file mode 100644 index 0000000000..a29e5d1d81 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java @@ -0,0 +1,183 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.hive.shims.Utils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.thrift.TProtocolVersion; + +/** + * + * HiveSessionImplwithUGI. + * HiveSession with connecting user's UGI and delegation token if required + */ +public class HiveSessionImplwithUGI extends HiveSessionImpl { + public static final String HS2TOKEN = "HiveServer2ImpersonationToken"; + + private UserGroupInformation sessionUgi = null; + private String delegationTokenStr = null; + private Hive sessionHive = null; + private HiveSession proxySession = null; + static final Log LOG = LogFactory.getLog(HiveSessionImplwithUGI.class); + + public HiveSessionImplwithUGI(TProtocolVersion protocol, String username, String password, + HiveConf hiveConf, String ipAddress, String delegationToken) throws HiveSQLException { + super(protocol, username, password, hiveConf, ipAddress); + setSessionUGI(username); + setDelegationToken(delegationToken); + + // create a new metastore connection for this particular user session + Hive.set(null); + try { + sessionHive = Hive.get(getHiveConf()); + } catch (HiveException e) { + throw new HiveSQLException("Failed to setup metastore connection", e); + } + } + + // setup appropriate UGI for the session + public void setSessionUGI(String owner) throws HiveSQLException { + if (owner == null) { + throw new HiveSQLException("No username provided for impersonation"); + } + if (UserGroupInformation.isSecurityEnabled()) { + try { + sessionUgi = UserGroupInformation.createProxyUser( + owner, UserGroupInformation.getLoginUser()); + } catch (IOException e) { + throw new HiveSQLException("Couldn't setup proxy user", e); + } + } else { + sessionUgi = UserGroupInformation.createRemoteUser(owner); + } + } + + public UserGroupInformation getSessionUgi() { + return this.sessionUgi; + } + + public String getDelegationToken () { + return this.delegationTokenStr; + } + + @Override + protected synchronized void acquire(boolean userAccess) { + super.acquire(userAccess); + // if we have a metastore connection with impersonation, then set it first + if (sessionHive != null) { + Hive.set(sessionHive); + } + } + + /** + * Close the file systems for the session and remove it from the FileSystem cache. + * Cancel the session's delegation token and close the metastore connection + */ + @Override + public void close() throws HiveSQLException { + try { + acquire(true); + cancelDelegationToken(); + } finally { + try { + super.close(); + } finally { + try { + FileSystem.closeAllForUGI(sessionUgi); + } catch (IOException ioe) { + throw new HiveSQLException("Could not clean up file-system handles for UGI: " + + sessionUgi, ioe); + } + } + } + } + + /** + * Enable delegation token for the session + * save the token string and set the token.signature in hive conf. The metastore client uses + * this token.signature to determine where to use kerberos or delegation token + * @throws HiveException + * @throws IOException + */ + private void setDelegationToken(String delegationTokenStr) throws HiveSQLException { + this.delegationTokenStr = delegationTokenStr; + if (delegationTokenStr != null) { + getHiveConf().set("hive.metastore.token.signature", HS2TOKEN); + try { + Utils.setTokenStr(sessionUgi, delegationTokenStr, HS2TOKEN); + } catch (IOException e) { + throw new HiveSQLException("Couldn't setup delegation token in the ugi", e); + } + } + } + + // If the session has a delegation token obtained from the metastore, then cancel it + private void cancelDelegationToken() throws HiveSQLException { + if (delegationTokenStr != null) { + try { + Hive.get(getHiveConf()).cancelDelegationToken(delegationTokenStr); + } catch (HiveException e) { + throw new HiveSQLException("Couldn't cancel delegation token", e); + } + // close the metastore connection created with this delegation token + Hive.closeCurrent(); + } + } + + @Override + protected HiveSession getSession() { + assert proxySession != null; + + return proxySession; + } + + public void setProxySession(HiveSession proxySession) { + this.proxySession = proxySession; + } + + @Override + public String getDelegationToken(HiveAuthFactory authFactory, String owner, + String renewer) throws HiveSQLException { + return authFactory.getDelegationToken(owner, renewer); + } + + @Override + public void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException { + authFactory.cancelDelegationToken(tokenStr); + } + + @Override + public void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException { + authFactory.renewDelegationToken(tokenStr); + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java new file mode 100644 index 0000000000..5b10521feb --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java @@ -0,0 +1,91 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +/** + * Proxy wrapper on HiveSession to execute operations + * by impersonating given user + */ +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; +import java.lang.reflect.UndeclaredThrowableException; +import java.security.PrivilegedActionException; +import java.security.PrivilegedExceptionAction; + +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.service.cli.HiveSQLException; + +public class HiveSessionProxy implements InvocationHandler { + private final HiveSession base; + private final UserGroupInformation ugi; + + public HiveSessionProxy(HiveSession hiveSession, UserGroupInformation ugi) { + this.base = hiveSession; + this.ugi = ugi; + } + + public static HiveSession getProxy(HiveSession hiveSession, UserGroupInformation ugi) + throws IllegalArgumentException, HiveSQLException { + return (HiveSession)Proxy.newProxyInstance(HiveSession.class.getClassLoader(), + new Class[] {HiveSession.class}, + new HiveSessionProxy(hiveSession, ugi)); + } + + @Override + public Object invoke(Object arg0, final Method method, final Object[] args) + throws Throwable { + try { + if (method.getDeclaringClass() == HiveSessionBase.class) { + return invoke(method, args); + } + return ugi.doAs( + new PrivilegedExceptionAction () { + @Override + public Object run() throws HiveSQLException { + return invoke(method, args); + } + }); + } catch (UndeclaredThrowableException e) { + Throwable innerException = e.getCause(); + if (innerException instanceof PrivilegedActionException) { + throw innerException.getCause(); + } else { + throw e.getCause(); + } + } + } + + private Object invoke(final Method method, final Object[] args) throws HiveSQLException { + try { + return method.invoke(base, args); + } catch (InvocationTargetException e) { + if (e.getCause() instanceof HiveSQLException) { + throw (HiveSQLException)e.getCause(); + } + throw new RuntimeException(e.getCause()); + } catch (IllegalArgumentException e) { + throw new RuntimeException(e); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + } +} + diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java new file mode 100644 index 0000000000..36a30b1e28 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java @@ -0,0 +1,402 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.ql.hooks.HookUtils; +import org.apache.hive.service.CompositeService; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.SessionHandle; +import org.apache.hive.service.cli.operation.OperationManager; +import org.apache.hive.service.cli.thrift.TProtocolVersion; +import org.apache.hive.service.server.HiveServer2; +import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; + +/** + * SessionManager. + * + */ +public class SessionManager extends CompositeService { + + private static final Log LOG = LogFactory.getLog(CompositeService.class); + public static final String HIVERCFILE = ".hiverc"; + private HiveConf hiveConf; + private final Map handleToSession = + new ConcurrentHashMap(); + private final OperationManager operationManager = new OperationManager(); + private ThreadPoolExecutor backgroundOperationPool; + private boolean isOperationLogEnabled; + private File operationLogRootDir; + + private long checkInterval; + private long sessionTimeout; + private boolean checkOperation; + + private volatile boolean shutdown; + // The HiveServer2 instance running this service + private final HiveServer2 hiveServer2; + + public SessionManager(HiveServer2 hiveServer2) { + super(SessionManager.class.getSimpleName()); + this.hiveServer2 = hiveServer2; + } + + @Override + public synchronized void init(HiveConf hiveConf) { + this.hiveConf = hiveConf; + //Create operation log root directory, if operation logging is enabled + if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED)) { + initOperationLogRootDir(); + } + createBackgroundOperationPool(); + addService(operationManager); + super.init(hiveConf); + } + + private void createBackgroundOperationPool() { + int poolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS); + LOG.info("HiveServer2: Background operation thread pool size: " + poolSize); + int poolQueueSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_WAIT_QUEUE_SIZE); + LOG.info("HiveServer2: Background operation thread wait queue size: " + poolQueueSize); + long keepAliveTime = HiveConf.getTimeVar( + hiveConf, ConfVars.HIVE_SERVER2_ASYNC_EXEC_KEEPALIVE_TIME, TimeUnit.SECONDS); + LOG.info( + "HiveServer2: Background operation thread keepalive time: " + keepAliveTime + " seconds"); + + // Create a thread pool with #poolSize threads + // Threads terminate when they are idle for more than the keepAliveTime + // A bounded blocking queue is used to queue incoming operations, if #operations > poolSize + String threadPoolName = "HiveServer2-Background-Pool"; + backgroundOperationPool = new ThreadPoolExecutor(poolSize, poolSize, + keepAliveTime, TimeUnit.SECONDS, new LinkedBlockingQueue(poolQueueSize), + new ThreadFactoryWithGarbageCleanup(threadPoolName)); + backgroundOperationPool.allowCoreThreadTimeOut(true); + + checkInterval = HiveConf.getTimeVar( + hiveConf, ConfVars.HIVE_SERVER2_SESSION_CHECK_INTERVAL, TimeUnit.MILLISECONDS); + sessionTimeout = HiveConf.getTimeVar( + hiveConf, ConfVars.HIVE_SERVER2_IDLE_SESSION_TIMEOUT, TimeUnit.MILLISECONDS); + checkOperation = HiveConf.getBoolVar(hiveConf, + ConfVars.HIVE_SERVER2_IDLE_SESSION_CHECK_OPERATION); + } + + private void initOperationLogRootDir() { + operationLogRootDir = new File( + hiveConf.getVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LOG_LOCATION)); + isOperationLogEnabled = true; + + if (operationLogRootDir.exists() && !operationLogRootDir.isDirectory()) { + LOG.warn("The operation log root directory exists, but it is not a directory: " + + operationLogRootDir.getAbsolutePath()); + isOperationLogEnabled = false; + } + + if (!operationLogRootDir.exists()) { + if (!operationLogRootDir.mkdirs()) { + LOG.warn("Unable to create operation log root directory: " + + operationLogRootDir.getAbsolutePath()); + isOperationLogEnabled = false; + } + } + + if (isOperationLogEnabled) { + LOG.info("Operation log root directory is created: " + operationLogRootDir.getAbsolutePath()); + try { + FileUtils.forceDeleteOnExit(operationLogRootDir); + } catch (IOException e) { + LOG.warn("Failed to schedule cleanup HS2 operation logging root dir: " + + operationLogRootDir.getAbsolutePath(), e); + } + } + } + + @Override + public synchronized void start() { + super.start(); + if (checkInterval > 0) { + startTimeoutChecker(); + } + } + + private void startTimeoutChecker() { + final long interval = Math.max(checkInterval, 3000l); // minimum 3 seconds + Runnable timeoutChecker = new Runnable() { + @Override + public void run() { + for (sleepInterval(interval); !shutdown; sleepInterval(interval)) { + long current = System.currentTimeMillis(); + for (HiveSession session : new ArrayList(handleToSession.values())) { + if (sessionTimeout > 0 && session.getLastAccessTime() + sessionTimeout <= current + && (!checkOperation || session.getNoOperationTime() > sessionTimeout)) { + SessionHandle handle = session.getSessionHandle(); + LOG.warn("Session " + handle + " is Timed-out (last access : " + + new Date(session.getLastAccessTime()) + ") and will be closed"); + try { + closeSession(handle); + } catch (HiveSQLException e) { + LOG.warn("Exception is thrown closing session " + handle, e); + } + } else { + session.closeExpiredOperations(); + } + } + } + } + + private void sleepInterval(long interval) { + try { + Thread.sleep(interval); + } catch (InterruptedException e) { + // ignore + } + } + }; + backgroundOperationPool.execute(timeoutChecker); + } + + @Override + public synchronized void stop() { + super.stop(); + shutdown = true; + if (backgroundOperationPool != null) { + backgroundOperationPool.shutdown(); + long timeout = hiveConf.getTimeVar( + ConfVars.HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT, TimeUnit.SECONDS); + try { + backgroundOperationPool.awaitTermination(timeout, TimeUnit.SECONDS); + } catch (InterruptedException e) { + LOG.warn("HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT = " + timeout + + " seconds has been exceeded. RUNNING background operations will be shut down", e); + } + backgroundOperationPool = null; + } + cleanupLoggingRootDir(); + } + + private void cleanupLoggingRootDir() { + if (isOperationLogEnabled) { + try { + FileUtils.forceDelete(operationLogRootDir); + } catch (Exception e) { + LOG.warn("Failed to cleanup root dir of HS2 logging: " + operationLogRootDir + .getAbsolutePath(), e); + } + } + } + + public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress, + Map sessionConf) throws HiveSQLException { + return openSession(protocol, username, password, ipAddress, sessionConf, false, null); + } + + /** + * Opens a new session and creates a session handle. + * The username passed to this method is the effective username. + * If withImpersonation is true (==doAs true) we wrap all the calls in HiveSession + * within a UGI.doAs, where UGI corresponds to the effective user. + * @see org.apache.hive.service.cli.thrift.ThriftCLIService#getUserName() + * + * @param protocol + * @param username + * @param password + * @param ipAddress + * @param sessionConf + * @param withImpersonation + * @param delegationToken + * @return + * @throws HiveSQLException + */ + public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress, + Map sessionConf, boolean withImpersonation, String delegationToken) + throws HiveSQLException { + HiveSession session; + // If doAs is set to true for HiveServer2, we will create a proxy object for the session impl. + // Within the proxy object, we wrap the method call in a UserGroupInformation#doAs + if (withImpersonation) { + HiveSessionImplwithUGI sessionWithUGI = new HiveSessionImplwithUGI(protocol, username, password, + hiveConf, ipAddress, delegationToken); + session = HiveSessionProxy.getProxy(sessionWithUGI, sessionWithUGI.getSessionUgi()); + sessionWithUGI.setProxySession(session); + } else { + session = new HiveSessionImpl(protocol, username, password, hiveConf, ipAddress); + } + session.setSessionManager(this); + session.setOperationManager(operationManager); + try { + session.open(sessionConf); + } catch (Exception e) { + try { + session.close(); + } catch (Throwable t) { + LOG.warn("Error closing session", t); + } + session = null; + throw new HiveSQLException("Failed to open new session: " + e, e); + } + if (isOperationLogEnabled) { + session.setOperationLogSessionDir(operationLogRootDir); + } + try { + executeSessionHooks(session); + } catch (Exception e) { + try { + session.close(); + } catch (Throwable t) { + LOG.warn("Error closing session", t); + } + session = null; + throw new HiveSQLException("Failed to execute session hooks", e); + } + handleToSession.put(session.getSessionHandle(), session); + return session.getSessionHandle(); + } + + public void closeSession(SessionHandle sessionHandle) throws HiveSQLException { + HiveSession session = handleToSession.remove(sessionHandle); + if (session == null) { + throw new HiveSQLException("Session does not exist!"); + } + try { + session.close(); + } finally { + // Shutdown HiveServer2 if it has been deregistered from ZooKeeper and has no active sessions + if (!(hiveServer2 == null) && (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_SUPPORT_DYNAMIC_SERVICE_DISCOVERY)) + && (!hiveServer2.isRegisteredWithZooKeeper())) { + // Asynchronously shutdown this instance of HiveServer2, + // if there are no active client sessions + if (getOpenSessionCount() == 0) { + LOG.info("This instance of HiveServer2 has been removed from the list of server " + + "instances available for dynamic service discovery. " + + "The last client session has ended - will shutdown now."); + Thread shutdownThread = new Thread() { + @Override + public void run() { + hiveServer2.stop(); + } + }; + shutdownThread.start(); + } + } + } + } + + public HiveSession getSession(SessionHandle sessionHandle) throws HiveSQLException { + HiveSession session = handleToSession.get(sessionHandle); + if (session == null) { + throw new HiveSQLException("Invalid SessionHandle: " + sessionHandle); + } + return session; + } + + public OperationManager getOperationManager() { + return operationManager; + } + + private static ThreadLocal threadLocalIpAddress = new ThreadLocal() { + @Override + protected synchronized String initialValue() { + return null; + } + }; + + public static void setIpAddress(String ipAddress) { + threadLocalIpAddress.set(ipAddress); + } + + public static void clearIpAddress() { + threadLocalIpAddress.remove(); + } + + public static String getIpAddress() { + return threadLocalIpAddress.get(); + } + + private static ThreadLocal threadLocalUserName = new ThreadLocal(){ + @Override + protected synchronized String initialValue() { + return null; + } + }; + + public static void setUserName(String userName) { + threadLocalUserName.set(userName); + } + + public static void clearUserName() { + threadLocalUserName.remove(); + } + + public static String getUserName() { + return threadLocalUserName.get(); + } + + private static ThreadLocal threadLocalProxyUserName = new ThreadLocal(){ + @Override + protected synchronized String initialValue() { + return null; + } + }; + + public static void setProxyUserName(String userName) { + LOG.debug("setting proxy user name based on query param to: " + userName); + threadLocalProxyUserName.set(userName); + } + + public static String getProxyUserName() { + return threadLocalProxyUserName.get(); + } + + public static void clearProxyUserName() { + threadLocalProxyUserName.remove(); + } + + // execute session hooks + private void executeSessionHooks(HiveSession session) throws Exception { + List sessionHooks = HookUtils.getHooks(hiveConf, + HiveConf.ConfVars.HIVE_SERVER2_SESSION_HOOK, HiveSessionHook.class); + for (HiveSessionHook sessionHook : sessionHooks) { + sessionHook.run(new HiveSessionHookContextImpl(session)); + } + } + + public Future submitBackgroundOperation(Runnable r) { + return backgroundOperationPool.submit(r); + } + + public int getOpenSessionCount() { + return handleToSession.size(); + } +} + diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/EmbeddedThriftBinaryCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/EmbeddedThriftBinaryCLIService.java new file mode 100644 index 0000000000..ac63537337 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/EmbeddedThriftBinaryCLIService.java @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.thrift; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hive.service.cli.CLIService; +import org.apache.hive.service.cli.ICLIService; + + +/** + * EmbeddedThriftBinaryCLIService. + * + */ +public class EmbeddedThriftBinaryCLIService extends ThriftBinaryCLIService { + + public EmbeddedThriftBinaryCLIService() { + super(new CLIService(null)); + isEmbedded = true; + HiveConf.setLoadHiveServer2Config(true); + } + + @Override + public synchronized void init(HiveConf hiveConf) { + cliService.init(hiveConf); + cliService.start(); + super.init(hiveConf); + } + + public ICLIService getService() { + return cliService; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java new file mode 100644 index 0000000000..6c9efba9e5 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.thrift; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.CLIService; +import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; +import org.apache.thrift.TProcessorFactory; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.server.TThreadPoolServer; +import org.apache.thrift.transport.TServerSocket; +import org.apache.thrift.transport.TTransportFactory; + + +public class ThriftBinaryCLIService extends ThriftCLIService { + + public ThriftBinaryCLIService(CLIService cliService) { + super(cliService, ThriftBinaryCLIService.class.getSimpleName()); + } + + @Override + public void run() { + try { + // Server thread pool + String threadPoolName = "HiveServer2-Handler-Pool"; + ExecutorService executorService = new ThreadPoolExecutor(minWorkerThreads, maxWorkerThreads, + workerKeepAliveTime, TimeUnit.SECONDS, new SynchronousQueue(), + new ThreadFactoryWithGarbageCleanup(threadPoolName)); + + // Thrift configs + hiveAuthFactory = new HiveAuthFactory(hiveConf); + TTransportFactory transportFactory = hiveAuthFactory.getAuthTransFactory(); + TProcessorFactory processorFactory = hiveAuthFactory.getAuthProcFactory(this); + TServerSocket serverSocket = null; + List sslVersionBlacklist = new ArrayList(); + for (String sslVersion : hiveConf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST).split(",")) { + sslVersionBlacklist.add(sslVersion); + } + if (!hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL)) { + serverSocket = HiveAuthFactory.getServerSocket(hiveHost, portNum); + } else { + String keyStorePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH).trim(); + if (keyStorePath.isEmpty()) { + throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname + + " Not configured for SSL connection"); + } + String keyStorePassword = ShimLoader.getHadoopShims().getPassword(hiveConf, + HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname); + serverSocket = HiveAuthFactory.getServerSSLSocket(hiveHost, portNum, keyStorePath, + keyStorePassword, sslVersionBlacklist); + } + + // Server args + int maxMessageSize = hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_MAX_MESSAGE_SIZE); + int requestTimeout = (int) hiveConf.getTimeVar( + HiveConf.ConfVars.HIVE_SERVER2_THRIFT_LOGIN_TIMEOUT, TimeUnit.SECONDS); + int beBackoffSlotLength = (int) hiveConf.getTimeVar( + HiveConf.ConfVars.HIVE_SERVER2_THRIFT_LOGIN_BEBACKOFF_SLOT_LENGTH, TimeUnit.MILLISECONDS); + TThreadPoolServer.Args sargs = new TThreadPoolServer.Args(serverSocket) + .processorFactory(processorFactory).transportFactory(transportFactory) + .protocolFactory(new TBinaryProtocol.Factory()) + .inputProtocolFactory(new TBinaryProtocol.Factory(true, true, maxMessageSize, maxMessageSize)) + .requestTimeout(requestTimeout).requestTimeoutUnit(TimeUnit.SECONDS) + .beBackoffSlotLength(beBackoffSlotLength).beBackoffSlotLengthUnit(TimeUnit.MILLISECONDS) + .executorService(executorService); + + // TCP Server + server = new TThreadPoolServer(sargs); + server.setServerEventHandler(serverEventHandler); + String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + + portNum + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; + LOG.info(msg); + server.serve(); + } catch (Throwable t) { + LOG.fatal( + "Error starting HiveServer2: could not start " + + ThriftBinaryCLIService.class.getSimpleName(), t); + System.exit(-1); + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java new file mode 100644 index 0000000000..5a0f1c83c7 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -0,0 +1,740 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.thrift; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import javax.security.auth.login.LoginException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hive.service.AbstractService; +import org.apache.hive.service.ServiceException; +import org.apache.hive.service.ServiceUtils; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.auth.TSetIpAddressProcessor; +import org.apache.hive.service.cli.CLIService; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.FetchType; +import org.apache.hive.service.cli.GetInfoType; +import org.apache.hive.service.cli.GetInfoValue; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationHandle; +import org.apache.hive.service.cli.OperationStatus; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.SessionHandle; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.SessionManager; +import org.apache.hive.service.server.HiveServer2; +import org.apache.thrift.TException; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.server.ServerContext; +import org.apache.thrift.server.TServer; +import org.apache.thrift.server.TServerEventHandler; +import org.apache.thrift.transport.TTransport; + +/** + * ThriftCLIService. + * + */ +public abstract class ThriftCLIService extends AbstractService implements TCLIService.Iface, Runnable { + + public static final Log LOG = LogFactory.getLog(ThriftCLIService.class.getName()); + + protected CLIService cliService; + private static final TStatus OK_STATUS = new TStatus(TStatusCode.SUCCESS_STATUS); + protected static HiveAuthFactory hiveAuthFactory; + + protected int portNum; + protected InetAddress serverIPAddress; + protected String hiveHost; + protected TServer server; + protected org.eclipse.jetty.server.Server httpServer; + + private boolean isStarted = false; + protected boolean isEmbedded = false; + + protected HiveConf hiveConf; + + protected int minWorkerThreads; + protected int maxWorkerThreads; + protected long workerKeepAliveTime; + + protected TServerEventHandler serverEventHandler; + protected ThreadLocal currentServerContext; + + static class ThriftCLIServerContext implements ServerContext { + private SessionHandle sessionHandle = null; + + public void setSessionHandle(SessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public SessionHandle getSessionHandle() { + return sessionHandle; + } + } + + public ThriftCLIService(CLIService service, String serviceName) { + super(serviceName); + this.cliService = service; + currentServerContext = new ThreadLocal(); + serverEventHandler = new TServerEventHandler() { + @Override + public ServerContext createContext( + TProtocol input, TProtocol output) { + return new ThriftCLIServerContext(); + } + + @Override + public void deleteContext(ServerContext serverContext, + TProtocol input, TProtocol output) { + ThriftCLIServerContext context = (ThriftCLIServerContext)serverContext; + SessionHandle sessionHandle = context.getSessionHandle(); + if (sessionHandle != null) { + LOG.info("Session disconnected without closing properly, close it now"); + try { + cliService.closeSession(sessionHandle); + } catch (HiveSQLException e) { + LOG.warn("Failed to close session: " + e, e); + } + } + } + + @Override + public void preServe() { + } + + @Override + public void processContext(ServerContext serverContext, + TTransport input, TTransport output) { + currentServerContext.set(serverContext); + } + }; + } + + @Override + public synchronized void init(HiveConf hiveConf) { + this.hiveConf = hiveConf; + // Initialize common server configs needed in both binary & http modes + String portString; + hiveHost = System.getenv("HIVE_SERVER2_THRIFT_BIND_HOST"); + if (hiveHost == null) { + hiveHost = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST); + } + try { + if (hiveHost != null && !hiveHost.isEmpty()) { + serverIPAddress = InetAddress.getByName(hiveHost); + } else { + serverIPAddress = InetAddress.getLocalHost(); + } + } catch (UnknownHostException e) { + throw new ServiceException(e); + } + // HTTP mode + if (HiveServer2.isHTTPTransportMode(hiveConf)) { + workerKeepAliveTime = + hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_WORKER_KEEPALIVE_TIME, + TimeUnit.SECONDS); + portString = System.getenv("HIVE_SERVER2_THRIFT_HTTP_PORT"); + if (portString != null) { + portNum = Integer.valueOf(portString); + } else { + portNum = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT); + } + } + // Binary mode + else { + workerKeepAliveTime = + hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_WORKER_KEEPALIVE_TIME, TimeUnit.SECONDS); + portString = System.getenv("HIVE_SERVER2_THRIFT_PORT"); + if (portString != null) { + portNum = Integer.valueOf(portString); + } else { + portNum = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT); + } + } + minWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MIN_WORKER_THREADS); + maxWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MAX_WORKER_THREADS); + super.init(hiveConf); + } + + @Override + public synchronized void start() { + super.start(); + if (!isStarted && !isEmbedded) { + new Thread(this).start(); + isStarted = true; + } + } + + @Override + public synchronized void stop() { + if (isStarted && !isEmbedded) { + if(server != null) { + server.stop(); + LOG.info("Thrift server has stopped"); + } + if((httpServer != null) && httpServer.isStarted()) { + try { + httpServer.stop(); + LOG.info("Http server has stopped"); + } catch (Exception e) { + LOG.error("Error stopping Http server: ", e); + } + } + isStarted = false; + } + super.stop(); + } + + public int getPortNumber() { + return portNum; + } + + public InetAddress getServerIPAddress() { + return serverIPAddress; + } + + @Override + public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) + throws TException { + TGetDelegationTokenResp resp = new TGetDelegationTokenResp(); + + if (hiveAuthFactory == null) { + resp.setStatus(unsecureTokenErrorStatus()); + } else { + try { + String token = cliService.getDelegationToken( + new SessionHandle(req.getSessionHandle()), + hiveAuthFactory, req.getOwner(), req.getRenewer()); + resp.setDelegationToken(token); + resp.setStatus(OK_STATUS); + } catch (HiveSQLException e) { + LOG.error("Error obtaining delegation token", e); + TStatus tokenErrorStatus = HiveSQLException.toTStatus(e); + tokenErrorStatus.setSqlState("42000"); + resp.setStatus(tokenErrorStatus); + } + } + return resp; + } + + @Override + public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) + throws TException { + TCancelDelegationTokenResp resp = new TCancelDelegationTokenResp(); + + if (hiveAuthFactory == null) { + resp.setStatus(unsecureTokenErrorStatus()); + } else { + try { + cliService.cancelDelegationToken(new SessionHandle(req.getSessionHandle()), + hiveAuthFactory, req.getDelegationToken()); + resp.setStatus(OK_STATUS); + } catch (HiveSQLException e) { + LOG.error("Error canceling delegation token", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + } + return resp; + } + + @Override + public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) + throws TException { + TRenewDelegationTokenResp resp = new TRenewDelegationTokenResp(); + if (hiveAuthFactory == null) { + resp.setStatus(unsecureTokenErrorStatus()); + } else { + try { + cliService.renewDelegationToken(new SessionHandle(req.getSessionHandle()), + hiveAuthFactory, req.getDelegationToken()); + resp.setStatus(OK_STATUS); + } catch (HiveSQLException e) { + LOG.error("Error obtaining renewing token", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + } + return resp; + } + + private TStatus unsecureTokenErrorStatus() { + TStatus errorStatus = new TStatus(TStatusCode.ERROR_STATUS); + errorStatus.setErrorMessage("Delegation token only supported over remote " + + "client with kerberos authentication"); + return errorStatus; + } + + @Override + public TOpenSessionResp OpenSession(TOpenSessionReq req) throws TException { + LOG.info("Client protocol version: " + req.getClient_protocol()); + TOpenSessionResp resp = new TOpenSessionResp(); + try { + SessionHandle sessionHandle = getSessionHandle(req, resp); + resp.setSessionHandle(sessionHandle.toTSessionHandle()); + // TODO: set real configuration map + resp.setConfiguration(new HashMap()); + resp.setStatus(OK_STATUS); + ThriftCLIServerContext context = + (ThriftCLIServerContext)currentServerContext.get(); + if (context != null) { + context.setSessionHandle(sessionHandle); + } + } catch (Exception e) { + LOG.warn("Error opening session: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + private String getIpAddress() { + String clientIpAddress; + // Http transport mode. + // We set the thread local ip address, in ThriftHttpServlet. + if (cliService.getHiveConf().getVar( + ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) { + clientIpAddress = SessionManager.getIpAddress(); + } + else { + // Kerberos + if (isKerberosAuthMode()) { + clientIpAddress = hiveAuthFactory.getIpAddress(); + } + // Except kerberos, NOSASL + else { + clientIpAddress = TSetIpAddressProcessor.getUserIpAddress(); + } + } + LOG.debug("Client's IP Address: " + clientIpAddress); + return clientIpAddress; + } + + /** + * Returns the effective username. + * 1. If hive.server2.allow.user.substitution = false: the username of the connecting user + * 2. If hive.server2.allow.user.substitution = true: the username of the end user, + * that the connecting user is trying to proxy for. + * This includes a check whether the connecting user is allowed to proxy for the end user. + * @param req + * @return + * @throws HiveSQLException + */ + private String getUserName(TOpenSessionReq req) throws HiveSQLException { + String userName = null; + // Kerberos + if (isKerberosAuthMode()) { + userName = hiveAuthFactory.getRemoteUser(); + } + // Except kerberos, NOSASL + if (userName == null) { + userName = TSetIpAddressProcessor.getUserName(); + } + // Http transport mode. + // We set the thread local username, in ThriftHttpServlet. + if (cliService.getHiveConf().getVar( + ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) { + userName = SessionManager.getUserName(); + } + if (userName == null) { + userName = req.getUsername(); + } + + userName = getShortName(userName); + String effectiveClientUser = getProxyUser(userName, req.getConfiguration(), getIpAddress()); + LOG.debug("Client's username: " + effectiveClientUser); + return effectiveClientUser; + } + + private String getShortName(String userName) { + String ret = null; + if (userName != null) { + int indexOfDomainMatch = ServiceUtils.indexOfDomainMatch(userName); + ret = (indexOfDomainMatch <= 0) ? userName : + userName.substring(0, indexOfDomainMatch); + } + + return ret; + } + + /** + * Create a session handle + * @param req + * @param res + * @return + * @throws HiveSQLException + * @throws LoginException + * @throws IOException + */ + SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res) + throws HiveSQLException, LoginException, IOException { + String userName = getUserName(req); + String ipAddress = getIpAddress(); + TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION, + req.getClient_protocol()); + SessionHandle sessionHandle; + if (cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && + (userName != null)) { + String delegationTokenStr = getDelegationToken(userName); + sessionHandle = cliService.openSessionWithImpersonation(protocol, userName, + req.getPassword(), ipAddress, req.getConfiguration(), delegationTokenStr); + } else { + sessionHandle = cliService.openSession(protocol, userName, req.getPassword(), + ipAddress, req.getConfiguration()); + } + res.setServerProtocolVersion(protocol); + return sessionHandle; + } + + + private String getDelegationToken(String userName) + throws HiveSQLException, LoginException, IOException { + if (userName == null || !cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION) + .equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString())) { + return null; + } + try { + return cliService.getDelegationTokenFromMetaStore(userName); + } catch (UnsupportedOperationException e) { + // The delegation token is not applicable in the given deployment mode + } + return null; + } + + private TProtocolVersion getMinVersion(TProtocolVersion... versions) { + TProtocolVersion[] values = TProtocolVersion.values(); + int current = values[values.length - 1].getValue(); + for (TProtocolVersion version : versions) { + if (current > version.getValue()) { + current = version.getValue(); + } + } + for (TProtocolVersion version : values) { + if (version.getValue() == current) { + return version; + } + } + throw new IllegalArgumentException("never"); + } + + @Override + public TCloseSessionResp CloseSession(TCloseSessionReq req) throws TException { + TCloseSessionResp resp = new TCloseSessionResp(); + try { + SessionHandle sessionHandle = new SessionHandle(req.getSessionHandle()); + cliService.closeSession(sessionHandle); + resp.setStatus(OK_STATUS); + ThriftCLIServerContext context = + (ThriftCLIServerContext)currentServerContext.get(); + if (context != null) { + context.setSessionHandle(null); + } + } catch (Exception e) { + LOG.warn("Error closing session: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetInfoResp GetInfo(TGetInfoReq req) throws TException { + TGetInfoResp resp = new TGetInfoResp(); + try { + GetInfoValue getInfoValue = + cliService.getInfo(new SessionHandle(req.getSessionHandle()), + GetInfoType.getGetInfoType(req.getInfoType())); + resp.setInfoValue(getInfoValue.toTGetInfoValue()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting info: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TExecuteStatementResp ExecuteStatement(TExecuteStatementReq req) throws TException { + TExecuteStatementResp resp = new TExecuteStatementResp(); + try { + SessionHandle sessionHandle = new SessionHandle(req.getSessionHandle()); + String statement = req.getStatement(); + Map confOverlay = req.getConfOverlay(); + Boolean runAsync = req.isRunAsync(); + OperationHandle operationHandle = runAsync ? + cliService.executeStatementAsync(sessionHandle, statement, confOverlay) + : cliService.executeStatement(sessionHandle, statement, confOverlay); + resp.setOperationHandle(operationHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error executing statement: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetTypeInfoResp GetTypeInfo(TGetTypeInfoReq req) throws TException { + TGetTypeInfoResp resp = new TGetTypeInfoResp(); + try { + OperationHandle operationHandle = cliService.getTypeInfo(new SessionHandle(req.getSessionHandle())); + resp.setOperationHandle(operationHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting type info: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetCatalogsResp GetCatalogs(TGetCatalogsReq req) throws TException { + TGetCatalogsResp resp = new TGetCatalogsResp(); + try { + OperationHandle opHandle = cliService.getCatalogs(new SessionHandle(req.getSessionHandle())); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting catalogs: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetSchemasResp GetSchemas(TGetSchemasReq req) throws TException { + TGetSchemasResp resp = new TGetSchemasResp(); + try { + OperationHandle opHandle = cliService.getSchemas( + new SessionHandle(req.getSessionHandle()), req.getCatalogName(), req.getSchemaName()); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting schemas: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetTablesResp GetTables(TGetTablesReq req) throws TException { + TGetTablesResp resp = new TGetTablesResp(); + try { + OperationHandle opHandle = cliService + .getTables(new SessionHandle(req.getSessionHandle()), req.getCatalogName(), + req.getSchemaName(), req.getTableName(), req.getTableTypes()); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting tables: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetTableTypesResp GetTableTypes(TGetTableTypesReq req) throws TException { + TGetTableTypesResp resp = new TGetTableTypesResp(); + try { + OperationHandle opHandle = cliService.getTableTypes(new SessionHandle(req.getSessionHandle())); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting table types: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetColumnsResp GetColumns(TGetColumnsReq req) throws TException { + TGetColumnsResp resp = new TGetColumnsResp(); + try { + OperationHandle opHandle = cliService.getColumns( + new SessionHandle(req.getSessionHandle()), + req.getCatalogName(), + req.getSchemaName(), + req.getTableName(), + req.getColumnName()); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting columns: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetFunctionsResp GetFunctions(TGetFunctionsReq req) throws TException { + TGetFunctionsResp resp = new TGetFunctionsResp(); + try { + OperationHandle opHandle = cliService.getFunctions( + new SessionHandle(req.getSessionHandle()), req.getCatalogName(), + req.getSchemaName(), req.getFunctionName()); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting functions: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) throws TException { + TGetOperationStatusResp resp = new TGetOperationStatusResp(); + try { + OperationStatus operationStatus = cliService.getOperationStatus( + new OperationHandle(req.getOperationHandle())); + resp.setOperationState(operationStatus.getState().toTOperationState()); + HiveSQLException opException = operationStatus.getOperationException(); + if (opException != null) { + resp.setSqlState(opException.getSQLState()); + resp.setErrorCode(opException.getErrorCode()); + resp.setErrorMessage(opException.getMessage()); + } + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting operation status: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TCancelOperationResp CancelOperation(TCancelOperationReq req) throws TException { + TCancelOperationResp resp = new TCancelOperationResp(); + try { + cliService.cancelOperation(new OperationHandle(req.getOperationHandle())); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error cancelling operation: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TCloseOperationResp CloseOperation(TCloseOperationReq req) throws TException { + TCloseOperationResp resp = new TCloseOperationResp(); + try { + cliService.closeOperation(new OperationHandle(req.getOperationHandle())); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error closing operation: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq req) + throws TException { + TGetResultSetMetadataResp resp = new TGetResultSetMetadataResp(); + try { + TableSchema schema = cliService.getResultSetMetadata(new OperationHandle(req.getOperationHandle())); + resp.setSchema(schema.toTTableSchema()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting result set metadata: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TFetchResultsResp FetchResults(TFetchResultsReq req) throws TException { + TFetchResultsResp resp = new TFetchResultsResp(); + try { + RowSet rowSet = cliService.fetchResults( + new OperationHandle(req.getOperationHandle()), + FetchOrientation.getFetchOrientation(req.getOrientation()), + req.getMaxRows(), + FetchType.getFetchType(req.getFetchType())); + resp.setResults(rowSet.toTRowSet()); + resp.setHasMoreRows(false); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error fetching results: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public abstract void run(); + + /** + * If the proxy user name is provided then check privileges to substitute the user. + * @param realUser + * @param sessionConf + * @param ipAddress + * @return + * @throws HiveSQLException + */ + private String getProxyUser(String realUser, Map sessionConf, + String ipAddress) throws HiveSQLException { + String proxyUser = null; + // Http transport mode. + // We set the thread local proxy username, in ThriftHttpServlet. + if (cliService.getHiveConf().getVar( + ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) { + proxyUser = SessionManager.getProxyUserName(); + LOG.debug("Proxy user from query string: " + proxyUser); + } + + if (proxyUser == null && sessionConf != null && sessionConf.containsKey(HiveAuthFactory.HS2_PROXY_USER)) { + String proxyUserFromThriftBody = sessionConf.get(HiveAuthFactory.HS2_PROXY_USER); + LOG.debug("Proxy user from thrift body: " + proxyUserFromThriftBody); + proxyUser = proxyUserFromThriftBody; + } + + if (proxyUser == null) { + return realUser; + } + + // check whether substitution is allowed + if (!hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ALLOW_USER_SUBSTITUTION)) { + throw new HiveSQLException("Proxy user substitution is not allowed"); + } + + // If there's no authentication, then directly substitute the user + if (HiveAuthFactory.AuthTypes.NONE.toString(). + equalsIgnoreCase(hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION))) { + return proxyUser; + } + + // Verify proxy user privilege of the realUser for the proxyUser + HiveAuthFactory.verifyProxyAccess(realUser, proxyUser, ipAddress, hiveConf); + LOG.debug("Verified proxy user: " + proxyUser); + return proxyUser; + } + + private boolean isKerberosAuthMode() { + return cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION) + .equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString()); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java new file mode 100644 index 0000000000..1af45398b8 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java @@ -0,0 +1,440 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.thrift; + +import java.util.List; +import java.util.Map; + +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.*; +import org.apache.thrift.TException; + +/** + * ThriftCLIServiceClient. + * + */ +public class ThriftCLIServiceClient extends CLIServiceClient { + private final TCLIService.Iface cliService; + + public ThriftCLIServiceClient(TCLIService.Iface cliService) { + this.cliService = cliService; + } + + public void checkStatus(TStatus status) throws HiveSQLException { + if (TStatusCode.ERROR_STATUS.equals(status.getStatusCode())) { + throw new HiveSQLException(status); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map) + */ + @Override + public SessionHandle openSession(String username, String password, + Map configuration) + throws HiveSQLException { + try { + TOpenSessionReq req = new TOpenSessionReq(); + req.setUsername(username); + req.setPassword(password); + req.setConfiguration(configuration); + TOpenSessionResp resp = cliService.OpenSession(req); + checkStatus(resp.getStatus()); + return new SessionHandle(resp.getSessionHandle(), resp.getServerProtocolVersion()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public SessionHandle openSessionWithImpersonation(String username, String password, + Map configuration, String delegationToken) throws HiveSQLException { + throw new HiveSQLException("open with impersonation operation is not supported in the client"); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public void closeSession(SessionHandle sessionHandle) throws HiveSQLException { + try { + TCloseSessionReq req = new TCloseSessionReq(sessionHandle.toTSessionHandle()); + TCloseSessionResp resp = cliService.CloseSession(req); + checkStatus(resp.getStatus()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List) + */ + @Override + public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType infoType) + throws HiveSQLException { + try { + // FIXME extract the right info type + TGetInfoReq req = new TGetInfoReq(sessionHandle.toTSessionHandle(), infoType.toTGetInfoType()); + TGetInfoResp resp = cliService.GetInfo(req); + checkStatus(resp.getStatus()); + return new GetInfoValue(resp.getInfoValue()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#executeStatement(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.util.Map) + */ + @Override + public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, + Map confOverlay) + throws HiveSQLException { + return executeStatementInternal(sessionHandle, statement, confOverlay, false); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#executeStatementAsync(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.util.Map) + */ + @Override + public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, + Map confOverlay) + throws HiveSQLException { + return executeStatementInternal(sessionHandle, statement, confOverlay, true); + } + + private OperationHandle executeStatementInternal(SessionHandle sessionHandle, String statement, + Map confOverlay, boolean isAsync) + throws HiveSQLException { + try { + TExecuteStatementReq req = + new TExecuteStatementReq(sessionHandle.toTSessionHandle(), statement); + req.setConfOverlay(confOverlay); + req.setRunAsync(isAsync); + TExecuteStatementResp resp = cliService.ExecuteStatement(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTypeInfo(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTypeInfo(SessionHandle sessionHandle) throws HiveSQLException { + try { + TGetTypeInfoReq req = new TGetTypeInfoReq(sessionHandle.toTSessionHandle()); + TGetTypeInfoResp resp = cliService.GetTypeInfo(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getCatalogs(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getCatalogs(SessionHandle sessionHandle) throws HiveSQLException { + try { + TGetCatalogsReq req = new TGetCatalogsReq(sessionHandle.toTSessionHandle()); + TGetCatalogsResp resp = cliService.GetCatalogs(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String) + */ + @Override + public OperationHandle getSchemas(SessionHandle sessionHandle, String catalogName, + String schemaName) + throws HiveSQLException { + try { + TGetSchemasReq req = new TGetSchemasReq(sessionHandle.toTSessionHandle()); + req.setCatalogName(catalogName); + req.setSchemaName(schemaName); + TGetSchemasResp resp = cliService.GetSchemas(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List) + */ + @Override + public OperationHandle getTables(SessionHandle sessionHandle, String catalogName, + String schemaName, String tableName, List tableTypes) + throws HiveSQLException { + try { + TGetTablesReq req = new TGetTablesReq(sessionHandle.toTSessionHandle()); + req.setTableName(tableName); + req.setTableTypes(tableTypes); + req.setSchemaName(schemaName); + TGetTablesResp resp = cliService.GetTables(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTableTypes(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTableTypes(SessionHandle sessionHandle) throws HiveSQLException { + try { + TGetTableTypesReq req = new TGetTableTypesReq(sessionHandle.toTSessionHandle()); + TGetTableTypesResp resp = cliService.GetTableTypes(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getColumns(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getColumns(SessionHandle sessionHandle, + String catalogName, String schemaName, String tableName, String columnName) + throws HiveSQLException { + try { + TGetColumnsReq req = new TGetColumnsReq(); + req.setSessionHandle(sessionHandle.toTSessionHandle()); + req.setCatalogName(catalogName); + req.setSchemaName(schemaName); + req.setTableName(tableName); + req.setColumnName(columnName); + TGetColumnsResp resp = cliService.GetColumns(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getFunctions(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getFunctions(SessionHandle sessionHandle, + String catalogName, String schemaName, String functionName) throws HiveSQLException { + try { + TGetFunctionsReq req = new TGetFunctionsReq(sessionHandle.toTSessionHandle(), functionName); + req.setCatalogName(catalogName); + req.setSchemaName(schemaName); + TGetFunctionsResp resp = cliService.GetFunctions(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getOperationStatus(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public OperationStatus getOperationStatus(OperationHandle opHandle) throws HiveSQLException { + try { + TGetOperationStatusReq req = new TGetOperationStatusReq(opHandle.toTOperationHandle()); + TGetOperationStatusResp resp = cliService.GetOperationStatus(req); + // Checks the status of the RPC call, throws an exception in case of error + checkStatus(resp.getStatus()); + OperationState opState = OperationState.getOperationState(resp.getOperationState()); + HiveSQLException opException = null; + if (opState == OperationState.ERROR) { + opException = new HiveSQLException(resp.getErrorMessage(), resp.getSqlState(), resp.getErrorCode()); + } + return new OperationStatus(opState, opException); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#cancelOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { + try { + TCancelOperationReq req = new TCancelOperationReq(opHandle.toTOperationHandle()); + TCancelOperationResp resp = cliService.CancelOperation(req); + checkStatus(resp.getStatus()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#closeOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void closeOperation(OperationHandle opHandle) + throws HiveSQLException { + try { + TCloseOperationReq req = new TCloseOperationReq(opHandle.toTOperationHandle()); + TCloseOperationResp resp = cliService.CloseOperation(req); + checkStatus(resp.getStatus()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public TableSchema getResultSetMetadata(OperationHandle opHandle) + throws HiveSQLException { + try { + TGetResultSetMetadataReq req = new TGetResultSetMetadataReq(opHandle.toTOperationHandle()); + TGetResultSetMetadataResp resp = cliService.GetResultSetMetadata(req); + checkStatus(resp.getStatus()); + return new TableSchema(resp.getSchema()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + @Override + public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, long maxRows, + FetchType fetchType) throws HiveSQLException { + try { + TFetchResultsReq req = new TFetchResultsReq(); + req.setOperationHandle(opHandle.toTOperationHandle()); + req.setOrientation(orientation.toTFetchOrientation()); + req.setMaxRows(maxRows); + req.setFetchType(fetchType.toTFetchType()); + TFetchResultsResp resp = cliService.FetchResults(req); + checkStatus(resp.getStatus()); + return RowSetFactory.create(resp.getResults(), opHandle.getProtocolVersion()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#fetchResults(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public RowSet fetchResults(OperationHandle opHandle) throws HiveSQLException { + // TODO: set the correct default fetch size + return fetchResults(opHandle, FetchOrientation.FETCH_NEXT, 10000, FetchType.QUERY_OUTPUT); + } + + @Override + public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String owner, String renewer) throws HiveSQLException { + TGetDelegationTokenReq req = new TGetDelegationTokenReq( + sessionHandle.toTSessionHandle(), owner, renewer); + try { + TGetDelegationTokenResp tokenResp = cliService.GetDelegationToken(req); + checkStatus(tokenResp.getStatus()); + return tokenResp.getDelegationToken(); + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + @Override + public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + TCancelDelegationTokenReq cancelReq = new TCancelDelegationTokenReq( + sessionHandle.toTSessionHandle(), tokenStr); + try { + TCancelDelegationTokenResp cancelResp = + cliService.CancelDelegationToken(cancelReq); + checkStatus(cancelResp.getStatus()); + return; + } catch (TException e) { + throw new HiveSQLException(e); + } + } + + @Override + public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + TRenewDelegationTokenReq cancelReq = new TRenewDelegationTokenReq( + sessionHandle.toTSessionHandle(), tokenStr); + try { + TRenewDelegationTokenResp renewResp = + cliService.RenewDelegationToken(cancelReq); + checkStatus(renewResp.getStatus()); + return; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java new file mode 100644 index 0000000000..3b57efa38b --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -0,0 +1,167 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.thrift; + +import java.util.Arrays; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.Shell; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.CLIService; +import org.apache.hive.service.cli.thrift.TCLIService.Iface; +import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; +import org.apache.thrift.TProcessor; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TProtocolFactory; +import org.apache.thrift.server.TServlet; +import org.eclipse.jetty.server.nio.SelectChannelConnector; +import org.eclipse.jetty.server.ssl.SslSelectChannelConnector; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.util.ssl.SslContextFactory; +import org.eclipse.jetty.util.thread.ExecutorThreadPool; + + +public class ThriftHttpCLIService extends ThriftCLIService { + + public ThriftHttpCLIService(CLIService cliService) { + super(cliService, ThriftHttpCLIService.class.getSimpleName()); + } + + /** + * Configure Jetty to serve http requests. Example of a client connection URL: + * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, + * e.g. http://gateway:port/hive2/servlets/thrifths2/ + */ + @Override + public void run() { + try { + // HTTP Server + httpServer = new org.eclipse.jetty.server.Server(); + + // Server thread pool + // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests + String threadPoolName = "HiveServer2-HttpHandler-Pool"; + ExecutorService executorService = new ThreadPoolExecutor(minWorkerThreads, maxWorkerThreads, + workerKeepAliveTime, TimeUnit.SECONDS, new SynchronousQueue(), + new ThreadFactoryWithGarbageCleanup(threadPoolName)); + ExecutorThreadPool threadPool = new ExecutorThreadPool(executorService); + httpServer.setThreadPool(threadPool); + + // Connector configs + SelectChannelConnector connector = new SelectChannelConnector(); + boolean useSsl = hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL); + String schemeName = useSsl ? "https" : "http"; + // Change connector if SSL is used + if (useSsl) { + String keyStorePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH).trim(); + String keyStorePassword = ShimLoader.getHadoopShims().getPassword(hiveConf, + HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname); + if (keyStorePath.isEmpty()) { + throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname + + " Not configured for SSL connection"); + } + SslContextFactory sslContextFactory = new SslContextFactory(); + String[] excludedProtocols = hiveConf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST).split(","); + LOG.info("HTTP Server SSL: adding excluded protocols: " + Arrays.toString(excludedProtocols)); + sslContextFactory.addExcludeProtocols(excludedProtocols); + LOG.info("HTTP Server SSL: SslContextFactory.getExcludeProtocols = " + + Arrays.toString(sslContextFactory.getExcludeProtocols())); + sslContextFactory.setKeyStorePath(keyStorePath); + sslContextFactory.setKeyStorePassword(keyStorePassword); + connector = new SslSelectChannelConnector(sslContextFactory); + } + connector.setPort(portNum); + // Linux:yes, Windows:no + connector.setReuseAddress(!Shell.WINDOWS); + int maxIdleTime = (int) hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME, + TimeUnit.MILLISECONDS); + connector.setMaxIdleTime(maxIdleTime); + + httpServer.addConnector(connector); + + // Thrift configs + hiveAuthFactory = new HiveAuthFactory(hiveConf); + TProcessor processor = new TCLIService.Processor(this); + TProtocolFactory protocolFactory = new TBinaryProtocol.Factory(); + // Set during the init phase of HiveServer2 if auth mode is kerberos + // UGI for the hive/_HOST (kerberos) principal + UserGroupInformation serviceUGI = cliService.getServiceUGI(); + // UGI for the http/_HOST (SPNego) principal + UserGroupInformation httpUGI = cliService.getHttpUGI(); + String authType = hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION); + TServlet thriftHttpServlet = new ThriftHttpServlet(processor, protocolFactory, authType, + serviceUGI, httpUGI); + + // Context handler + final ServletContextHandler context = new ServletContextHandler( + ServletContextHandler.SESSIONS); + context.setContextPath("/"); + String httpPath = getHttpPath(hiveConf + .getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH)); + httpServer.setHandler(context); + context.addServlet(new ServletHolder(thriftHttpServlet), httpPath); + + // TODO: check defaults: maxTimeout, keepalive, maxBodySize, bodyRecieveDuration, etc. + // Finally, start the server + httpServer.start(); + String msg = "Started " + ThriftHttpCLIService.class.getSimpleName() + " in " + schemeName + + " mode on port " + portNum + " path=" + httpPath + " with " + minWorkerThreads + "..." + + maxWorkerThreads + " worker threads"; + LOG.info(msg); + httpServer.join(); + } catch (Throwable t) { + LOG.fatal( + "Error starting HiveServer2: could not start " + + ThriftHttpCLIService.class.getSimpleName(), t); + System.exit(-1); + } + } + + /** + * The config parameter can be like "path", "/path", "/path/", "path/*", "/path1/path2/*" and so on. + * httpPath should end up as "/*", "/path/*" or "/path1/../pathN/*" + * @param httpPath + * @return + */ + private String getHttpPath(String httpPath) { + if(httpPath == null || httpPath.equals("")) { + httpPath = "/*"; + } + else { + if(!httpPath.startsWith("/")) { + httpPath = "/" + httpPath; + } + if(httpPath.endsWith("/")) { + httpPath = httpPath + "*"; + } + if(!httpPath.endsWith("/*")) { + httpPath = httpPath + "/*"; + } + } + return httpPath; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java new file mode 100644 index 0000000000..56c8cb6e54 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java @@ -0,0 +1,546 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.thrift; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.security.PrivilegedExceptionAction; +import java.util.Arrays; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import javax.servlet.ServletException; +import javax.servlet.http.Cookie; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.core.NewCookie; + +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.codec.binary.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.shims.HadoopShims.KerberosNameShim; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.service.auth.AuthenticationProviderFactory; +import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.auth.HttpAuthUtils; +import org.apache.hive.service.auth.HttpAuthenticationException; +import org.apache.hive.service.auth.PasswdAuthenticationProvider; +import org.apache.hive.service.cli.session.SessionManager; +import org.apache.hive.service.CookieSigner; +import org.apache.thrift.TProcessor; +import org.apache.thrift.protocol.TProtocolFactory; +import org.apache.thrift.server.TServlet; +import org.ietf.jgss.GSSContext; +import org.ietf.jgss.GSSCredential; +import org.ietf.jgss.GSSException; +import org.ietf.jgss.GSSManager; +import org.ietf.jgss.GSSName; +import org.ietf.jgss.Oid; + +/** + * + * ThriftHttpServlet + * + */ +public class ThriftHttpServlet extends TServlet { + + private static final long serialVersionUID = 1L; + public static final Log LOG = LogFactory.getLog(ThriftHttpServlet.class.getName()); + private final String authType; + private final UserGroupInformation serviceUGI; + private final UserGroupInformation httpUGI; + private HiveConf hiveConf = new HiveConf(); + + // Class members for cookie based authentication. + private CookieSigner signer; + public static final String AUTH_COOKIE = "hive.server2.auth"; + private static final Random RAN = new Random(); + private boolean isCookieAuthEnabled; + private String cookieDomain; + private String cookiePath; + private int cookieMaxAge; + private boolean isCookieSecure; + private boolean isHttpOnlyCookie; + + public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory, + String authType, UserGroupInformation serviceUGI, UserGroupInformation httpUGI) { + super(processor, protocolFactory); + this.authType = authType; + this.serviceUGI = serviceUGI; + this.httpUGI = httpUGI; + this.isCookieAuthEnabled = hiveConf.getBoolVar( + ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_AUTH_ENABLED); + // Initialize the cookie based authentication related variables. + if (isCookieAuthEnabled) { + // Generate the signer with secret. + String secret = Long.toString(RAN.nextLong()); + LOG.debug("Using the random number as the secret for cookie generation " + secret); + this.signer = new CookieSigner(secret.getBytes()); + this.cookieMaxAge = (int) hiveConf.getTimeVar( + ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_MAX_AGE, TimeUnit.SECONDS); + this.cookieDomain = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_DOMAIN); + this.cookiePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_PATH); + this.isCookieSecure = hiveConf.getBoolVar( + ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_IS_SECURE); + this.isHttpOnlyCookie = hiveConf.getBoolVar( + ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_IS_HTTPONLY); + } + } + + @Override + protected void doPost(HttpServletRequest request, HttpServletResponse response) + throws ServletException, IOException { + String clientUserName = null; + String clientIpAddress; + boolean requireNewCookie = false; + + try { + // If the cookie based authentication is already enabled, parse the + // request and validate the request cookies. + if (isCookieAuthEnabled) { + clientUserName = validateCookie(request); + requireNewCookie = (clientUserName == null); + if (requireNewCookie) { + LOG.info("Could not validate cookie sent, will try to generate a new cookie"); + } + } + // If the cookie based authentication is not enabled or the request does + // not have a valid cookie, use the kerberos or password based authentication + // depending on the server setup. + if (clientUserName == null) { + // For a kerberos setup + if (isKerberosAuthMode(authType)) { + clientUserName = doKerberosAuth(request); + } + // For password based authentication + else { + clientUserName = doPasswdAuth(request, authType); + } + } + LOG.debug("Client username: " + clientUserName); + + // Set the thread local username to be used for doAs if true + SessionManager.setUserName(clientUserName); + + // find proxy user if any from query param + String doAsQueryParam = getDoAsQueryParam(request.getQueryString()); + if (doAsQueryParam != null) { + SessionManager.setProxyUserName(doAsQueryParam); + } + + clientIpAddress = request.getRemoteAddr(); + LOG.debug("Client IP Address: " + clientIpAddress); + // Set the thread local ip address + SessionManager.setIpAddress(clientIpAddress); + // Generate new cookie and add it to the response + if (requireNewCookie && + !authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) { + String cookieToken = HttpAuthUtils.createCookieToken(clientUserName); + Cookie hs2Cookie = createCookie(signer.signCookie(cookieToken)); + + if (isHttpOnlyCookie) { + response.setHeader("SET-COOKIE", getHttpOnlyCookieHeader(hs2Cookie)); + } else { + response.addCookie(hs2Cookie); + } + LOG.info("Cookie added for clientUserName " + clientUserName); + } + super.doPost(request, response); + } + catch (HttpAuthenticationException e) { + LOG.error("Error: ", e); + // Send a 401 to the client + response.setStatus(HttpServletResponse.SC_UNAUTHORIZED); + if(isKerberosAuthMode(authType)) { + response.addHeader(HttpAuthUtils.WWW_AUTHENTICATE, HttpAuthUtils.NEGOTIATE); + } + response.getWriter().println("Authentication Error: " + e.getMessage()); + } + finally { + // Clear the thread locals + SessionManager.clearUserName(); + SessionManager.clearIpAddress(); + SessionManager.clearProxyUserName(); + } + } + + /** + * Retrieves the client name from cookieString. If the cookie does not + * correspond to a valid client, the function returns null. + * @param cookies HTTP Request cookies. + * @return Client Username if cookieString has a HS2 Generated cookie that is currently valid. + * Else, returns null. + */ + private String getClientNameFromCookie(Cookie[] cookies) { + // Current Cookie Name, Current Cookie Value + String currName, currValue; + + // Following is the main loop which iterates through all the cookies send by the client. + // The HS2 generated cookies are of the format hive.server2.auth= + // A cookie which is identified as a hiveserver2 generated cookie is validated + // by calling signer.verifyAndExtract(). If the validation passes, send the + // username for which the cookie is validated to the caller. If no client side + // cookie passes the validation, return null to the caller. + for (Cookie currCookie : cookies) { + // Get the cookie name + currName = currCookie.getName(); + if (!currName.equals(AUTH_COOKIE)) { + // Not a HS2 generated cookie, continue. + continue; + } + // If we reached here, we have match for HS2 generated cookie + currValue = currCookie.getValue(); + // Validate the value. + currValue = signer.verifyAndExtract(currValue); + // Retrieve the user name, do the final validation step. + if (currValue != null) { + String userName = HttpAuthUtils.getUserNameFromCookieToken(currValue); + + if (userName == null) { + LOG.warn("Invalid cookie token " + currValue); + continue; + } + //We have found a valid cookie in the client request. + if (LOG.isDebugEnabled()) { + LOG.debug("Validated the cookie for user " + userName); + } + return userName; + } + } + // No valid HS2 generated cookies found, return null + return null; + } + + /** + * Convert cookie array to human readable cookie string + * @param cookies Cookie Array + * @return String containing all the cookies separated by a newline character. + * Each cookie is of the format [key]=[value] + */ + private String toCookieStr(Cookie[] cookies) { + String cookieStr = ""; + + for (Cookie c : cookies) { + cookieStr += c.getName() + "=" + c.getValue() + " ;\n"; + } + return cookieStr; + } + + /** + * Validate the request cookie. This function iterates over the request cookie headers + * and finds a cookie that represents a valid client/server session. If it finds one, it + * returns the client name associated with the session. Else, it returns null. + * @param request The HTTP Servlet Request send by the client + * @return Client Username if the request has valid HS2 cookie, else returns null + * @throws UnsupportedEncodingException + */ + private String validateCookie(HttpServletRequest request) throws UnsupportedEncodingException { + // Find all the valid cookies associated with the request. + Cookie[] cookies = request.getCookies(); + + if (cookies == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("No valid cookies associated with the request " + request); + } + return null; + } + if (LOG.isDebugEnabled()) { + LOG.debug("Received cookies: " + toCookieStr(cookies)); + } + return getClientNameFromCookie(cookies); + } + + /** + * Generate a server side cookie given the cookie value as the input. + * @param str Input string token. + * @return The generated cookie. + * @throws UnsupportedEncodingException + */ + private Cookie createCookie(String str) throws UnsupportedEncodingException { + if (LOG.isDebugEnabled()) { + LOG.debug("Cookie name = " + AUTH_COOKIE + " value = " + str); + } + Cookie cookie = new Cookie(AUTH_COOKIE, str); + + cookie.setMaxAge(cookieMaxAge); + if (cookieDomain != null) { + cookie.setDomain(cookieDomain); + } + if (cookiePath != null) { + cookie.setPath(cookiePath); + } + cookie.setSecure(isCookieSecure); + return cookie; + } + + /** + * Generate httponly cookie from HS2 cookie + * @param cookie HS2 generated cookie + * @return The httponly cookie + */ + private static String getHttpOnlyCookieHeader(Cookie cookie) { + NewCookie newCookie = new NewCookie(cookie.getName(), cookie.getValue(), + cookie.getPath(), cookie.getDomain(), cookie.getVersion(), + cookie.getComment(), cookie.getMaxAge(), cookie.getSecure()); + return newCookie + "; HttpOnly"; + } + + /** + * Do the LDAP/PAM authentication + * @param request + * @param authType + * @throws HttpAuthenticationException + */ + private String doPasswdAuth(HttpServletRequest request, String authType) + throws HttpAuthenticationException { + String userName = getUsername(request, authType); + // No-op when authType is NOSASL + if (!authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) { + try { + AuthMethods authMethod = AuthMethods.getValidAuthMethod(authType); + PasswdAuthenticationProvider provider = + AuthenticationProviderFactory.getAuthenticationProvider(authMethod); + provider.Authenticate(userName, getPassword(request, authType)); + + } catch (Exception e) { + throw new HttpAuthenticationException(e); + } + } + return userName; + } + + /** + * Do the GSS-API kerberos authentication. + * We already have a logged in subject in the form of serviceUGI, + * which GSS-API will extract information from. + * In case of a SPNego request we use the httpUGI, + * for the authenticating service tickets. + * @param request + * @return + * @throws HttpAuthenticationException + */ + private String doKerberosAuth(HttpServletRequest request) + throws HttpAuthenticationException { + // Try authenticating with the http/_HOST principal + if (httpUGI != null) { + try { + return httpUGI.doAs(new HttpKerberosServerAction(request, httpUGI)); + } catch (Exception e) { + LOG.info("Failed to authenticate with http/_HOST kerberos principal, " + + "trying with hive/_HOST kerberos principal"); + } + } + // Now try with hive/_HOST principal + try { + return serviceUGI.doAs(new HttpKerberosServerAction(request, serviceUGI)); + } catch (Exception e) { + LOG.error("Failed to authenticate with hive/_HOST kerberos principal"); + throw new HttpAuthenticationException(e); + } + + } + + class HttpKerberosServerAction implements PrivilegedExceptionAction { + HttpServletRequest request; + UserGroupInformation serviceUGI; + + HttpKerberosServerAction(HttpServletRequest request, + UserGroupInformation serviceUGI) { + this.request = request; + this.serviceUGI = serviceUGI; + } + + @Override + public String run() throws HttpAuthenticationException { + // Get own Kerberos credentials for accepting connection + GSSManager manager = GSSManager.getInstance(); + GSSContext gssContext = null; + String serverPrincipal = getPrincipalWithoutRealm( + serviceUGI.getUserName()); + try { + // This Oid for Kerberos GSS-API mechanism. + Oid kerberosMechOid = new Oid("1.2.840.113554.1.2.2"); + // Oid for SPNego GSS-API mechanism. + Oid spnegoMechOid = new Oid("1.3.6.1.5.5.2"); + // Oid for kerberos principal name + Oid krb5PrincipalOid = new Oid("1.2.840.113554.1.2.2.1"); + + // GSS name for server + GSSName serverName = manager.createName(serverPrincipal, krb5PrincipalOid); + + // GSS credentials for server + GSSCredential serverCreds = manager.createCredential(serverName, + GSSCredential.DEFAULT_LIFETIME, + new Oid[]{kerberosMechOid, spnegoMechOid}, + GSSCredential.ACCEPT_ONLY); + + // Create a GSS context + gssContext = manager.createContext(serverCreds); + // Get service ticket from the authorization header + String serviceTicketBase64 = getAuthHeader(request, authType); + byte[] inToken = Base64.decodeBase64(serviceTicketBase64.getBytes()); + gssContext.acceptSecContext(inToken, 0, inToken.length); + // Authenticate or deny based on its context completion + if (!gssContext.isEstablished()) { + throw new HttpAuthenticationException("Kerberos authentication failed: " + + "unable to establish context with the service ticket " + + "provided by the client."); + } + else { + return getPrincipalWithoutRealmAndHost(gssContext.getSrcName().toString()); + } + } + catch (GSSException e) { + throw new HttpAuthenticationException("Kerberos authentication failed: ", e); + } + finally { + if (gssContext != null) { + try { + gssContext.dispose(); + } catch (GSSException e) { + // No-op + } + } + } + } + + private String getPrincipalWithoutRealm(String fullPrincipal) + throws HttpAuthenticationException { + KerberosNameShim fullKerberosName; + try { + fullKerberosName = ShimLoader.getHadoopShims().getKerberosNameShim(fullPrincipal); + } catch (IOException e) { + throw new HttpAuthenticationException(e); + } + String serviceName = fullKerberosName.getServiceName(); + String hostName = fullKerberosName.getHostName(); + String principalWithoutRealm = serviceName; + if (hostName != null) { + principalWithoutRealm = serviceName + "/" + hostName; + } + return principalWithoutRealm; + } + + private String getPrincipalWithoutRealmAndHost(String fullPrincipal) + throws HttpAuthenticationException { + KerberosNameShim fullKerberosName; + try { + fullKerberosName = ShimLoader.getHadoopShims().getKerberosNameShim(fullPrincipal); + return fullKerberosName.getShortName(); + } catch (IOException e) { + throw new HttpAuthenticationException(e); + } + } + } + + private String getUsername(HttpServletRequest request, String authType) + throws HttpAuthenticationException { + String creds[] = getAuthHeaderTokens(request, authType); + // Username must be present + if (creds[0] == null || creds[0].isEmpty()) { + throw new HttpAuthenticationException("Authorization header received " + + "from the client does not contain username."); + } + return creds[0]; + } + + private String getPassword(HttpServletRequest request, String authType) + throws HttpAuthenticationException { + String creds[] = getAuthHeaderTokens(request, authType); + // Password must be present + if (creds[1] == null || creds[1].isEmpty()) { + throw new HttpAuthenticationException("Authorization header received " + + "from the client does not contain username."); + } + return creds[1]; + } + + private String[] getAuthHeaderTokens(HttpServletRequest request, + String authType) throws HttpAuthenticationException { + String authHeaderBase64 = getAuthHeader(request, authType); + String authHeaderString = StringUtils.newStringUtf8( + Base64.decodeBase64(authHeaderBase64.getBytes())); + String[] creds = authHeaderString.split(":"); + return creds; + } + + /** + * Returns the base64 encoded auth header payload + * @param request + * @param authType + * @return + * @throws HttpAuthenticationException + */ + private String getAuthHeader(HttpServletRequest request, String authType) + throws HttpAuthenticationException { + String authHeader = request.getHeader(HttpAuthUtils.AUTHORIZATION); + // Each http request must have an Authorization header + if (authHeader == null || authHeader.isEmpty()) { + throw new HttpAuthenticationException("Authorization header received " + + "from the client is empty."); + } + + String authHeaderBase64String; + int beginIndex; + if (isKerberosAuthMode(authType)) { + beginIndex = (HttpAuthUtils.NEGOTIATE + " ").length(); + } + else { + beginIndex = (HttpAuthUtils.BASIC + " ").length(); + } + authHeaderBase64String = authHeader.substring(beginIndex); + // Authorization header must have a payload + if (authHeaderBase64String == null || authHeaderBase64String.isEmpty()) { + throw new HttpAuthenticationException("Authorization header received " + + "from the client does not contain any data."); + } + return authHeaderBase64String; + } + + private boolean isKerberosAuthMode(String authType) { + return authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString()); + } + + private static String getDoAsQueryParam(String queryString) { + if (LOG.isDebugEnabled()) { + LOG.debug("URL query string:" + queryString); + } + if (queryString == null) { + return null; + } + Map params = javax.servlet.http.HttpUtils.parseQueryString( queryString ); + Set keySet = params.keySet(); + for (String key: keySet) { + if (key.equalsIgnoreCase("doAs")) { + return params.get(key)[0]; + } + } + return null; + } + +} + + diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java new file mode 100644 index 0000000000..58e8e49bd0 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java @@ -0,0 +1,621 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.server; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.cli.GnuParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.OptionBuilder; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.api.ACLProvider; +import org.apache.curator.framework.api.BackgroundCallback; +import org.apache.curator.framework.api.CuratorEvent; +import org.apache.curator.framework.api.CuratorEventType; +import org.apache.curator.framework.recipes.nodes.PersistentEphemeralNode; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.hadoop.hive.common.LogUtils; +import org.apache.hadoop.hive.common.LogUtils.LogInitializationException; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManagerImpl; +import org.apache.hadoop.hive.ql.exec.tez.TezSessionPoolManager; +import org.apache.hadoop.hive.ql.util.ZooKeeperHiveHelper; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.hive.shims.Utils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.common.util.HiveStringUtils; +import org.apache.hive.common.util.HiveVersionInfo; +import org.apache.hive.service.CompositeService; +import org.apache.hive.service.cli.CLIService; +import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService; +import org.apache.hive.service.cli.thrift.ThriftCLIService; +import org.apache.hive.service.cli.thrift.ThriftHttpCLIService; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooDefs.Ids; +import org.apache.zookeeper.ZooDefs.Perms; +import org.apache.zookeeper.data.ACL; + +/** + * HiveServer2. + * + */ +public class HiveServer2 extends CompositeService { + private static final Log LOG = LogFactory.getLog(HiveServer2.class); + private static CountDownLatch deleteSignal; + + private CLIService cliService; + private ThriftCLIService thriftCLIService; + private PersistentEphemeralNode znode; + private String znodePath; + private CuratorFramework zooKeeperClient; + private boolean registeredWithZooKeeper = false; + + public HiveServer2() { + super(HiveServer2.class.getSimpleName()); + HiveConf.setLoadHiveServer2Config(true); + } + + @Override + public synchronized void init(HiveConf hiveConf) { + cliService = new CLIService(this); + addService(cliService); + if (isHTTPTransportMode(hiveConf)) { + thriftCLIService = new ThriftHttpCLIService(cliService); + } else { + thriftCLIService = new ThriftBinaryCLIService(cliService); + } + addService(thriftCLIService); + super.init(hiveConf); + + // Add a shutdown hook for catching SIGTERM & SIGINT + final HiveServer2 hiveServer2 = this; + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + hiveServer2.stop(); + } + }); + } + + public static boolean isHTTPTransportMode(HiveConf hiveConf) { + String transportMode = System.getenv("HIVE_SERVER2_TRANSPORT_MODE"); + if (transportMode == null) { + transportMode = hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE); + } + if (transportMode != null && (transportMode.equalsIgnoreCase("http"))) { + return true; + } + return false; + } + + /** + * ACLProvider for providing appropriate ACLs to CuratorFrameworkFactory + */ + private final ACLProvider zooKeeperAclProvider = new ACLProvider() { + List nodeAcls = new ArrayList(); + + @Override + public List getDefaultAcl() { + if (UserGroupInformation.isSecurityEnabled()) { + // Read all to the world + nodeAcls.addAll(Ids.READ_ACL_UNSAFE); + // Create/Delete/Write/Admin to the authenticated user + nodeAcls.add(new ACL(Perms.ALL, Ids.AUTH_IDS)); + } else { + // ACLs for znodes on a non-kerberized cluster + // Create/Read/Delete/Write/Admin to the world + nodeAcls.addAll(Ids.OPEN_ACL_UNSAFE); + } + return nodeAcls; + } + + @Override + public List getAclForPath(String path) { + return getDefaultAcl(); + } + }; + + /** + * Adds a server instance to ZooKeeper as a znode. + * + * @param hiveConf + * @throws Exception + */ + private void addServerInstanceToZooKeeper(HiveConf hiveConf) throws Exception { + String zooKeeperEnsemble = ZooKeeperHiveHelper.getQuorumServers(hiveConf); + String rootNamespace = hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_ZOOKEEPER_NAMESPACE); + String instanceURI = getServerInstanceURI(hiveConf); + byte[] znodeDataUTF8 = instanceURI.getBytes(Charset.forName("UTF-8")); + setUpZooKeeperAuth(hiveConf); + int sessionTimeout = + (int) hiveConf.getTimeVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_SESSION_TIMEOUT, + TimeUnit.MILLISECONDS); + int baseSleepTime = + (int) hiveConf.getTimeVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_CONNECTION_BASESLEEPTIME, + TimeUnit.MILLISECONDS); + int maxRetries = hiveConf.getIntVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_CONNECTION_MAX_RETRIES); + // Create a CuratorFramework instance to be used as the ZooKeeper client + // Use the zooKeeperAclProvider to create appropriate ACLs + zooKeeperClient = + CuratorFrameworkFactory.builder().connectString(zooKeeperEnsemble) + .sessionTimeoutMs(sessionTimeout).aclProvider(zooKeeperAclProvider) + .retryPolicy(new ExponentialBackoffRetry(baseSleepTime, maxRetries)).build(); + zooKeeperClient.start(); + // Create the parent znodes recursively; ignore if the parent already exists. + try { + zooKeeperClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(ZooKeeperHiveHelper.ZOOKEEPER_PATH_SEPARATOR + rootNamespace); + LOG.info("Created the root name space: " + rootNamespace + " on ZooKeeper for HiveServer2"); + } catch (KeeperException e) { + if (e.code() != KeeperException.Code.NODEEXISTS) { + LOG.fatal("Unable to create HiveServer2 namespace: " + rootNamespace + " on ZooKeeper", e); + throw e; + } + } + // Create a znode under the rootNamespace parent for this instance of the server + // Znode name: serverUri=host:port;version=versionInfo;sequence=sequenceNumber + try { + String pathPrefix = + ZooKeeperHiveHelper.ZOOKEEPER_PATH_SEPARATOR + rootNamespace + + ZooKeeperHiveHelper.ZOOKEEPER_PATH_SEPARATOR + "serverUri=" + instanceURI + ";" + + "version=" + HiveVersionInfo.getVersion() + ";" + "sequence="; + znode = + new PersistentEphemeralNode(zooKeeperClient, + PersistentEphemeralNode.Mode.EPHEMERAL_SEQUENTIAL, pathPrefix, znodeDataUTF8); + znode.start(); + // We'll wait for 120s for node creation + long znodeCreationTimeout = 120; + if (!znode.waitForInitialCreate(znodeCreationTimeout, TimeUnit.SECONDS)) { + throw new Exception("Max znode creation wait time: " + znodeCreationTimeout + "s exhausted"); + } + setRegisteredWithZooKeeper(true); + znodePath = znode.getActualPath(); + // Set a watch on the znode + if (zooKeeperClient.checkExists().usingWatcher(new DeRegisterWatcher()).forPath(znodePath) == null) { + // No node exists, throw exception + throw new Exception("Unable to create znode for this HiveServer2 instance on ZooKeeper."); + } + LOG.info("Created a znode on ZooKeeper for HiveServer2 uri: " + instanceURI); + } catch (Exception e) { + LOG.fatal("Unable to create a znode for this server instance", e); + if (znode != null) { + znode.close(); + } + throw (e); + } + } + + /** + * For a kerberized cluster, we dynamically set up the client's JAAS conf. + * + * @param hiveConf + * @return + * @throws Exception + */ + private void setUpZooKeeperAuth(HiveConf hiveConf) throws Exception { + if (UserGroupInformation.isSecurityEnabled()) { + String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL); + if (principal.isEmpty()) { + throw new IOException("HiveServer2 Kerberos principal is empty"); + } + String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB); + if (keyTabFile.isEmpty()) { + throw new IOException("HiveServer2 Kerberos keytab is empty"); + } + // Install the JAAS Configuration for the runtime + Utils.setZookeeperClientKerberosJaasConfig(principal, keyTabFile); + } + } + + /** + * The watcher class which sets the de-register flag when the znode corresponding to this server + * instance is deleted. Additionally, it shuts down the server if there are no more active client + * sessions at the time of receiving a 'NodeDeleted' notification from ZooKeeper. + */ + private class DeRegisterWatcher implements Watcher { + @Override + public void process(WatchedEvent event) { + if (event.getType().equals(Watcher.Event.EventType.NodeDeleted)) { + if (znode != null) { + try { + znode.close(); + LOG.warn("This HiveServer2 instance is now de-registered from ZooKeeper. " + + "The server will be shut down after the last client sesssion completes."); + } catch (IOException e) { + LOG.error("Failed to close the persistent ephemeral znode", e); + } finally { + HiveServer2.this.setRegisteredWithZooKeeper(false); + // If there are no more active client sessions, stop the server + if (cliService.getSessionManager().getOpenSessionCount() == 0) { + LOG.warn("This instance of HiveServer2 has been removed from the list of server " + + "instances available for dynamic service discovery. " + + "The last client session has ended - will shutdown now."); + HiveServer2.this.stop(); + } + } + } + } + } + } + + private void removeServerInstanceFromZooKeeper() throws Exception { + setRegisteredWithZooKeeper(false); + if (znode != null) { + znode.close(); + } + zooKeeperClient.close(); + LOG.info("Server instance removed from ZooKeeper."); + } + + public boolean isRegisteredWithZooKeeper() { + return registeredWithZooKeeper; + } + + private void setRegisteredWithZooKeeper(boolean registeredWithZooKeeper) { + this.registeredWithZooKeeper = registeredWithZooKeeper; + } + + private String getServerInstanceURI(HiveConf hiveConf) throws Exception { + if ((thriftCLIService == null) || (thriftCLIService.getServerIPAddress() == null)) { + throw new Exception("Unable to get the server address; it hasn't been initialized yet."); + } + return thriftCLIService.getServerIPAddress().getHostName() + ":" + + thriftCLIService.getPortNumber(); + } + + @Override + public synchronized void start() { + super.start(); + } + + @Override + public synchronized void stop() { + LOG.info("Shutting down HiveServer2"); + HiveConf hiveConf = this.getHiveConf(); + super.stop(); + // Remove this server instance from ZooKeeper if dynamic service discovery is set + if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_SUPPORT_DYNAMIC_SERVICE_DISCOVERY)) { + try { + removeServerInstanceFromZooKeeper(); + } catch (Exception e) { + LOG.error("Error removing znode for this HiveServer2 instance from ZooKeeper.", e); + } + } + // There should already be an instance of the session pool manager. + // If not, ignoring is fine while stopping HiveServer2. + if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_TEZ_INITIALIZE_DEFAULT_SESSIONS)) { + try { + TezSessionPoolManager.getInstance().stop(); + } catch (Exception e) { + LOG.error("Tez session pool manager stop had an error during stop of HiveServer2. " + + "Shutting down HiveServer2 anyway.", e); + } + } + + if (hiveConf.getVar(ConfVars.HIVE_EXECUTION_ENGINE).equals("spark")) { + try { + SparkSessionManagerImpl.getInstance().shutdown(); + } catch(Exception ex) { + LOG.error("Spark session pool manager failed to stop during HiveServer2 shutdown.", ex); + } + } + } + + private static void startHiveServer2() throws Throwable { + long attempts = 0, maxAttempts = 1; + while (true) { + LOG.info("Starting HiveServer2"); + HiveConf hiveConf = new HiveConf(); + maxAttempts = hiveConf.getLongVar(HiveConf.ConfVars.HIVE_SERVER2_MAX_START_ATTEMPTS); + HiveServer2 server = null; + try { + server = new HiveServer2(); + server.init(hiveConf); + server.start(); + ShimLoader.getHadoopShims().startPauseMonitor(hiveConf); + // If we're supporting dynamic service discovery, we'll add the service uri for this + // HiveServer2 instance to Zookeeper as a znode. + if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_SUPPORT_DYNAMIC_SERVICE_DISCOVERY)) { + server.addServerInstanceToZooKeeper(hiveConf); + } + if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_TEZ_INITIALIZE_DEFAULT_SESSIONS)) { + TezSessionPoolManager sessionPool = TezSessionPoolManager.getInstance(); + sessionPool.setupPool(hiveConf); + sessionPool.startPool(); + } + + if (hiveConf.getVar(ConfVars.HIVE_EXECUTION_ENGINE).equals("spark")) { + SparkSessionManagerImpl.getInstance().setup(hiveConf); + } + break; + } catch (Throwable throwable) { + if (server != null) { + try { + server.stop(); + } catch (Throwable t) { + LOG.info("Exception caught when calling stop of HiveServer2 before retrying start", t); + } finally { + server = null; + } + } + if (++attempts >= maxAttempts) { + throw new Error("Max start attempts " + maxAttempts + " exhausted", throwable); + } else { + LOG.warn("Error starting HiveServer2 on attempt " + attempts + + ", will retry in 60 seconds", throwable); + try { + Thread.sleep(60L * 1000L); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } + } + } + + /** + * Remove all znodes corresponding to the given version number from ZooKeeper + * + * @param versionNumber + * @throws Exception + */ + static void deleteServerInstancesFromZooKeeper(String versionNumber) throws Exception { + HiveConf hiveConf = new HiveConf(); + String zooKeeperEnsemble = ZooKeeperHiveHelper.getQuorumServers(hiveConf); + String rootNamespace = hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_ZOOKEEPER_NAMESPACE); + int baseSleepTime = (int) hiveConf.getTimeVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_CONNECTION_BASESLEEPTIME, TimeUnit.MILLISECONDS); + int maxRetries = hiveConf.getIntVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_CONNECTION_MAX_RETRIES); + CuratorFramework zooKeeperClient = + CuratorFrameworkFactory.builder().connectString(zooKeeperEnsemble) + .retryPolicy(new ExponentialBackoffRetry(baseSleepTime, maxRetries)).build(); + zooKeeperClient.start(); + List znodePaths = + zooKeeperClient.getChildren().forPath( + ZooKeeperHiveHelper.ZOOKEEPER_PATH_SEPARATOR + rootNamespace); + List znodePathsUpdated; + // Now for each path that is for the given versionNumber, delete the znode from ZooKeeper + for (int i = 0; i < znodePaths.size(); i++) { + String znodePath = znodePaths.get(i); + deleteSignal = new CountDownLatch(1); + if (znodePath.contains("version=" + versionNumber + ";")) { + String fullZnodePath = + ZooKeeperHiveHelper.ZOOKEEPER_PATH_SEPARATOR + rootNamespace + + ZooKeeperHiveHelper.ZOOKEEPER_PATH_SEPARATOR + znodePath; + LOG.warn("Will attempt to remove the znode: " + fullZnodePath + " from ZooKeeper"); + System.out.println("Will attempt to remove the znode: " + fullZnodePath + " from ZooKeeper"); + zooKeeperClient.delete().guaranteed().inBackground(new DeleteCallBack()) + .forPath(fullZnodePath); + // Wait for the delete to complete + deleteSignal.await(); + // Get the updated path list + znodePathsUpdated = + zooKeeperClient.getChildren().forPath( + ZooKeeperHiveHelper.ZOOKEEPER_PATH_SEPARATOR + rootNamespace); + // Gives a list of any new paths that may have been created to maintain the persistent ephemeral node + znodePathsUpdated.removeAll(znodePaths); + // Add the new paths to the znodes list. We'll try for their removal as well. + znodePaths.addAll(znodePathsUpdated); + } + } + zooKeeperClient.close(); + } + + private static class DeleteCallBack implements BackgroundCallback { + @Override + public void processResult(CuratorFramework zooKeeperClient, CuratorEvent event) + throws Exception { + if (event.getType() == CuratorEventType.DELETE) { + deleteSignal.countDown(); + } + } + } + + public static void main(String[] args) { + HiveConf.setLoadHiveServer2Config(true); + try { + ServerOptionsProcessor oproc = new ServerOptionsProcessor("hiveserver2"); + ServerOptionsProcessorResponse oprocResponse = oproc.parse(args); + + // NOTE: It is critical to do this here so that log4j is reinitialized + // before any of the other core hive classes are loaded + String initLog4jMessage = LogUtils.initHiveLog4j(); + LOG.debug(initLog4jMessage); + HiveStringUtils.startupShutdownMessage(HiveServer2.class, args, LOG); + + // Log debug message from "oproc" after log4j initialize properly + LOG.debug(oproc.getDebugMessage().toString()); + + // Call the executor which will execute the appropriate command based on the parsed options + oprocResponse.getServerOptionsExecutor().execute(); + } catch (LogInitializationException e) { + LOG.error("Error initializing log: " + e.getMessage(), e); + System.exit(-1); + } + } + + /** + * ServerOptionsProcessor. + * Process arguments given to HiveServer2 (-hiveconf property=value) + * Set properties in System properties + * Create an appropriate response object, + * which has executor to execute the appropriate command based on the parsed options. + */ + static class ServerOptionsProcessor { + private final Options options = new Options(); + private org.apache.commons.cli.CommandLine commandLine; + private final String serverName; + private final StringBuilder debugMessage = new StringBuilder(); + + @SuppressWarnings("static-access") + ServerOptionsProcessor(String serverName) { + this.serverName = serverName; + // -hiveconf x=y + options.addOption(OptionBuilder + .withValueSeparator() + .hasArgs(2) + .withArgName("property=value") + .withLongOpt("hiveconf") + .withDescription("Use value for given property") + .create()); + // -deregister + options.addOption(OptionBuilder + .hasArgs(1) + .withArgName("versionNumber") + .withLongOpt("deregister") + .withDescription("Deregister all instances of given version from dynamic service discovery") + .create()); + options.addOption(new Option("H", "help", false, "Print help information")); + } + + ServerOptionsProcessorResponse parse(String[] argv) { + try { + commandLine = new GnuParser().parse(options, argv); + // Process --hiveconf + // Get hiveconf param values and set the System property values + Properties confProps = commandLine.getOptionProperties("hiveconf"); + for (String propKey : confProps.stringPropertyNames()) { + // save logging message for log4j output latter after log4j initialize properly + debugMessage.append("Setting " + propKey + "=" + confProps.getProperty(propKey) + ";\n"); + System.setProperty(propKey, confProps.getProperty(propKey)); + } + + // Process --help + if (commandLine.hasOption('H')) { + return new ServerOptionsProcessorResponse(new HelpOptionExecutor(serverName, options)); + } + + // Process --deregister + if (commandLine.hasOption("deregister")) { + return new ServerOptionsProcessorResponse(new DeregisterOptionExecutor( + commandLine.getOptionValue("deregister"))); + } + } catch (ParseException e) { + // Error out & exit - we were not able to parse the args successfully + System.err.println("Error starting HiveServer2 with given arguments: "); + System.err.println(e.getMessage()); + System.exit(-1); + } + // Default executor, when no option is specified + return new ServerOptionsProcessorResponse(new StartOptionExecutor()); + } + + StringBuilder getDebugMessage() { + return debugMessage; + } + } + + /** + * The response sent back from {@link ServerOptionsProcessor#parse(String[])} + */ + static class ServerOptionsProcessorResponse { + private final ServerOptionsExecutor serverOptionsExecutor; + + ServerOptionsProcessorResponse(ServerOptionsExecutor serverOptionsExecutor) { + this.serverOptionsExecutor = serverOptionsExecutor; + } + + ServerOptionsExecutor getServerOptionsExecutor() { + return serverOptionsExecutor; + } + } + + /** + * The executor interface for running the appropriate HiveServer2 command based on parsed options + */ + static interface ServerOptionsExecutor { + public void execute(); + } + + /** + * HelpOptionExecutor: executes the --help option by printing out the usage + */ + static class HelpOptionExecutor implements ServerOptionsExecutor { + private final Options options; + private final String serverName; + + HelpOptionExecutor(String serverName, Options options) { + this.options = options; + this.serverName = serverName; + } + + @Override + public void execute() { + new HelpFormatter().printHelp(serverName, options); + System.exit(0); + } + } + + /** + * StartOptionExecutor: starts HiveServer2. + * This is the default executor, when no option is specified. + */ + static class StartOptionExecutor implements ServerOptionsExecutor { + @Override + public void execute() { + try { + startHiveServer2(); + } catch (Throwable t) { + LOG.fatal("Error starting HiveServer2", t); + System.exit(-1); + } + } + } + + /** + * DeregisterOptionExecutor: executes the --deregister option by deregistering all HiveServer2 + * instances from ZooKeeper of a specific version. + */ + static class DeregisterOptionExecutor implements ServerOptionsExecutor { + private final String versionNumber; + + DeregisterOptionExecutor(String versionNumber) { + this.versionNumber = versionNumber; + } + + @Override + public void execute() { + try { + deleteServerInstancesFromZooKeeper(versionNumber); + } catch (Exception e) { + LOG.fatal("Error deregistering HiveServer2 instances for version: " + versionNumber + + " from ZooKeeper", e); + System.out.println("Error deregistering HiveServer2 instances for version: " + versionNumber + + " from ZooKeeper." + e); + System.exit(-1); + } + System.exit(0); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadFactoryWithGarbageCleanup.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadFactoryWithGarbageCleanup.java new file mode 100644 index 0000000000..fb8141a905 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadFactoryWithGarbageCleanup.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.hive.service.server; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ThreadFactory; + +import org.apache.hadoop.hive.metastore.RawStore; + +/** + * A ThreadFactory for constructing new HiveServer2 threads that lets you plug + * in custom cleanup code to be called before this thread is GC-ed. + * Currently cleans up the following: + * 1. ThreadLocal RawStore object: + * In case of an embedded metastore, HiveServer2 threads (foreground & background) + * end up caching a ThreadLocal RawStore object. The ThreadLocal RawStore object has + * an instance of PersistenceManagerFactory & PersistenceManager. + * The PersistenceManagerFactory keeps a cache of PersistenceManager objects, + * which are only removed when PersistenceManager#close method is called. + * HiveServer2 uses ExecutorService for managing thread pools for foreground & background threads. + * ExecutorService unfortunately does not provide any hooks to be called, + * when a thread from the pool is terminated. + * As a solution, we're using this ThreadFactory to keep a cache of RawStore objects per thread. + * And we are doing clean shutdown in the finalizer for each thread. + */ +public class ThreadFactoryWithGarbageCleanup implements ThreadFactory { + + private static Map threadRawStoreMap = new ConcurrentHashMap(); + + private final String namePrefix; + + public ThreadFactoryWithGarbageCleanup(String threadPoolName) { + namePrefix = threadPoolName; + } + + @Override + public Thread newThread(Runnable runnable) { + Thread newThread = new ThreadWithGarbageCleanup(runnable); + newThread.setName(namePrefix + ": Thread-" + newThread.getId()); + return newThread; + } + + public static Map getThreadRawStoreMap() { + return threadRawStoreMap; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java new file mode 100644 index 0000000000..8ee98103f7 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.hive.service.server; + +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.metastore.HiveMetaStore; +import org.apache.hadoop.hive.metastore.RawStore; + +/** + * A HiveServer2 thread used to construct new server threads. + * In particular, this thread ensures an orderly cleanup, + * when killed by its corresponding ExecutorService. + */ +public class ThreadWithGarbageCleanup extends Thread { + private static final Log LOG = LogFactory.getLog(ThreadWithGarbageCleanup.class); + + Map threadRawStoreMap = + ThreadFactoryWithGarbageCleanup.getThreadRawStoreMap(); + + public ThreadWithGarbageCleanup(Runnable runnable) { + super(runnable); + } + + /** + * Add any Thread specific garbage cleanup code here. + * Currently, it shuts down the RawStore object for this thread if it is not null. + */ + @Override + public void finalize() throws Throwable { + cleanRawStore(); + super.finalize(); + } + + private void cleanRawStore() { + Long threadId = this.getId(); + RawStore threadLocalRawStore = threadRawStoreMap.get(threadId); + if (threadLocalRawStore != null) { + LOG.debug("RawStore: " + threadLocalRawStore + ", for the thread: " + + this.getName() + " will be closed now."); + threadLocalRawStore.shutdown(); + threadRawStoreMap.remove(threadId); + } + } + + /** + * Cache the ThreadLocal RawStore object. Called from the corresponding thread. + */ + public void cacheThreadLocalRawStore() { + Long threadId = this.getId(); + RawStore threadLocalRawStore = HiveMetaStore.HMSHandler.getRawStore(); + if (threadLocalRawStore != null && !threadRawStoreMap.containsKey(threadId)) { + LOG.debug("Adding RawStore: " + threadLocalRawStore + ", for the thread: " + + this.getName() + " to threadRawStoreMap for future cleanup."); + threadRawStoreMap.put(threadId, threadLocalRawStore); + } + } +} -- cgit v1.2.3