Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
hive conf
  • Loading branch information
turboFei committed May 13, 2023
commit 7657cbb1188feb6c093b40cc47af35bc5dac7c0e
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
package org.apache.kyuubi.engine.spark

import scala.collection.JavaConverters._
import scala.util.control.NonFatal

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.io.Text
Expand All @@ -32,7 +31,7 @@ import org.apache.kyuubi.{KyuubiSQLException, Logging}
import org.apache.kyuubi.config.KyuubiConf
import org.apache.kyuubi.config.KyuubiReservedKeys._
import org.apache.kyuubi.ha.client.{EngineServiceDiscovery, ServiceDiscovery}
import org.apache.kyuubi.reflection.{DynConstructors, DynFields}
import org.apache.kyuubi.reflection.DynConstructors
import org.apache.kyuubi.service.{Serverable, Service, TBinaryFrontendService}
import org.apache.kyuubi.service.TFrontendService._
import org.apache.kyuubi.util.KyuubiHadoopUtils
Expand Down Expand Up @@ -113,6 +112,8 @@ class SparkTBinaryFrontendService(
object SparkTBinaryFrontendService extends Logging {

val HIVE_DELEGATION_TOKEN = new Text("HIVE_DELEGATION_TOKEN")
val HIVE_CONF_CLASSNAME = "org.apache.hadoop.hive.conf.HiveConf"
@volatile private var _hiveConf: Configuration = _

private[spark] def renewDelegationToken(sc: SparkContext, delegationToken: String): Unit = {
val newCreds = KyuubiHadoopUtils.decodeCredentials(delegationToken)
Expand All @@ -131,26 +132,12 @@ object SparkTBinaryFrontendService extends Logging {
}
}

private def hiveConf(hadoopConf: Configuration): Configuration = {
try {
val hiveConfClass = "org.apache.hadoop.hive.conf.HiveConf"
DynConstructors.builder()
.impl(hiveConfClass, classOf[Configuration], classOf[Class[_]])
.build[Configuration]()
.newInstance(hadoopConf, Class.forName(hiveConfClass))
} catch {
case e: Throwable =>
warn("Fail to create Hive Configuration", e)
hadoopConf
}
}

private def addHiveToken(
sc: SparkContext,
newTokens: Map[Text, Token[_ <: TokenIdentifier]],
oldCreds: Credentials,
updateCreds: Credentials): Unit = {
val metastoreUris = sc.hadoopConfiguration.getTrimmed("hive.metastore.uris", "")
val metastoreUris = hiveConf(sc.hadoopConfiguration).getTrimmed("hive.metastore.uris", "")

// `HiveMetaStoreClient` selects the first token whose service is "" and kind is
// "HIVE_DELEGATION_TOKEN" to authenticate.
Expand Down Expand Up @@ -221,4 +208,25 @@ object SparkTBinaryFrontendService extends Logging {
1
}
}

private[kyuubi] def hiveConf(hadoopConf: Configuration): Configuration = {
if (_hiveConf == null) {
synchronized {
if (_hiveConf == null) {
_hiveConf =
try {
DynConstructors.builder()
.impl(HIVE_CONF_CLASSNAME, classOf[Configuration], classOf[Class[_]])
.build[Configuration]()
.newInstance(hadoopConf, Class.forName(HIVE_CONF_CLASSNAME))
} catch {
case e: Throwable =>
warn("Fail to create Hive Configuration", e)
hadoopConf
}
}
}
}
_hiveConf
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* 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.engine.spark

import org.apache.hadoop.conf.Configuration

import org.apache.kyuubi.KyuubiFunSuite

class SparkTBinaryFrontendServiceSuite extends KyuubiFunSuite {
test("new hive conf") {
val hiveConf = SparkTBinaryFrontendService.hiveConf(new Configuration())
assert(hiveConf.getClass().getName == SparkTBinaryFrontendService.HIVE_CONF_CLASSNAME)
}
}