ulysses-you commented on a change in pull request #1053:
URL: https://github.com/apache/incubator-kyuubi/pull/1053#discussion_r704193761



##########
File path: 
kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/HiveDelegationTokenProvider.scala
##########
@@ -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.kyuubi.credentials
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.hive.conf.HiveConf
+import org.apache.hadoop.hive.metastore.{IMetaStoreClient, 
RetryingMetaStoreClient}
+import org.apache.hadoop.hive.thrift.DelegationTokenIdentifier
+import org.apache.hadoop.io.Text
+import org.apache.hadoop.security.{Credentials, SecurityUtil}
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod
+import org.apache.hadoop.security.token.Token
+
+import org.apache.kyuubi.Logging
+import org.apache.kyuubi.config.KyuubiConf
+
+class HiveDelegationTokenProvider extends HadoopDelegationTokenProvider with 
Logging {
+
+  private var client: Option[IMetaStoreClient] = None
+  private var principal: String = _
+
+  override def serviceName: String = "hive"
+
+  override def initialize(hadoopConf: Configuration, kyuubiConf: KyuubiConf): 
Unit = {
+    val conf = new HiveConf(hadoopConf, classOf[HiveConf])
+    val metastoreUris = conf.getTrimmed("hive.metastore.uris", "")
+
+    if (SecurityUtil.getAuthenticationMethod(hadoopConf) != 
AuthenticationMethod.SIMPLE
+      && metastoreUris.nonEmpty
+      && conf.getBoolean("hive.metastore.sasl.enabled", false)) {
+
+      val principalKey = "hive.metastore.kerberos.principal"
+      principal = conf.getTrimmed(principalKey, "")
+      require(principal.nonEmpty, s"Hive principal $principalKey undefined")
+
+      client = Some(RetryingMetaStoreClient.getProxy(conf, false))

Review comment:
       shall we close the client in some place ?

##########
File path: kyuubi-server/pom.xml
##########
@@ -63,7 +63,33 @@
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-client-runtime</artifactId>
-            <scope>runtime</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-metastore</artifactId>

Review comment:
       cc @pan3793 add some new hive dependency since we need connect to hive 
metastore

##########
File path: 
kyuubi-server/src/test/scala/org/apache/kyuubi/credentials/HiveDelegationTokenProviderSuite.scala
##########
@@ -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.kyuubi.credentials
+
+import java.io.{File, FileOutputStream}
+import java.net.URLClassLoader
+import java.util.concurrent.atomic.AtomicBoolean
+import java.util.concurrent.locks.ReentrantLock
+
+import scala.collection.JavaConverters._
+import scala.concurrent.ExecutionContext.Implicits.global
+import scala.concurrent.Future
+
+import org.apache.commons.io.FileUtils
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.hive.conf.HiveConf
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars._
+import org.apache.hadoop.hive.metastore.{HiveMetaException, HiveMetaStore}
+import org.apache.hadoop.hive.thrift.{DelegationTokenIdentifier, 
HadoopThriftAuthBridge, HadoopThriftAuthBridge23}
+import org.apache.hadoop.io.Text
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+import org.apache.thrift.TProcessor
+import org.apache.thrift.protocol.TProtocol
+import org.scalatest.concurrent.Eventually._
+import org.scalatest.time.SpanSugar.convertIntToGrainOfTime
+
+import org.apache.kyuubi.{KerberizedTestHelper, Logging, Utils}
+import org.apache.kyuubi.config.KyuubiConf
+import org.apache.kyuubi.credentials.LocalMetaServer.defaultHiveConf
+
+class HiveDelegationTokenProviderSuite extends KerberizedTestHelper {
+
+  private val hadoopConfDir: File = Utils.createTempDir().toFile
+  private var hiveConf: HiveConf = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    tryWithSecurityEnabled {
+      UserGroupInformation.loginUserFromKeytab(testPrincipal, testKeytab)
+
+      // HiveMetaStore creates a new hadoop `Configuration` object for each 
request to verify
+      // whether user can impersonate others.
+      // So we create a URLClassLoader with core-site.xml and set it as the 
thread's context
+      // classloader when creating `Configuration` object.
+      val conf = new Configuration(false)
+      conf.set("hadoop.security.authentication", "kerberos")
+      val realUser = UserGroupInformation.getCurrentUser.getShortUserName
+      conf.set(s"hadoop.proxyuser.$realUser.groups", "*")
+      conf.set(s"hadoop.proxyuser.$realUser.hosts", "*")
+
+      val xml = new File(hadoopConfDir, "core-site.xml")
+      val os = new FileOutputStream(xml)
+      try {
+        conf.writeXml(os)
+      } finally {
+        os.close()
+      }
+
+      val classloader =
+        new URLClassLoader(
+          Array(hadoopConfDir.toURI.toURL),
+          classOf[Configuration].getClassLoader)
+
+      hiveConf = LocalMetaServer.defaultHiveConf()
+      hiveConf.addResource(conf)
+      hiveConf.setVar(METASTORE_USE_THRIFT_SASL, "true")
+      hiveConf.setVar(METASTORE_KERBEROS_PRINCIPAL, testPrincipal)
+      hiveConf.setVar(METASTORE_KERBEROS_KEYTAB_FILE, testKeytab)
+      val metaServer = new LocalMetaServer(hiveConf, classloader)
+      metaServer.start()

Review comment:
       shall we stop the metastore server ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to