-
Notifications
You must be signed in to change notification settings - Fork 916
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Merge pull request #123 from yaooqinn/KYUUBI-122
[KYUUBI-122]obtain delegation tokens from possible kerberized services
- Loading branch information
Showing
16 changed files
with
513 additions
and
22 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
68 changes: 68 additions & 0 deletions
68
kyuubi-server/src/main/scala/yaooqinn/kyuubi/session/security/HDFSTokenCollector.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,68 @@ | ||
/* | ||
* 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 yaooqinn.kyuubi.session.security | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.{FileSystem, Path} | ||
import org.apache.hadoop.mapred.Master | ||
import org.apache.hadoop.security.{Credentials, UserGroupInformation} | ||
import org.apache.spark.KyuubiSparkUtil._ | ||
import org.apache.spark.SparkConf | ||
|
||
import yaooqinn.kyuubi.Logging | ||
import yaooqinn.kyuubi.service.ServiceException | ||
|
||
/** | ||
* Token collector for secured HDFS FileSystems | ||
*/ | ||
private[security] object HDFSTokenCollector extends TokenCollector with Logging { | ||
|
||
private def hadoopFStoAccess(conf: SparkConf, hadoopConf: Configuration): Set[FileSystem] = { | ||
val fileSystems = conf.getOption(ACCESS_FSS) | ||
.orElse(conf.getOption(ACCESS_NNS)) match { | ||
case Some(nns) => nns.split(",").map(new Path(_).getFileSystem(hadoopConf)).toSet | ||
case _ => Set.empty[FileSystem] | ||
} | ||
|
||
fileSystems + | ||
conf.getOption(STAGING_DIR).map(new Path(_).getFileSystem(hadoopConf)) | ||
.getOrElse(FileSystem.get(hadoopConf)) | ||
} | ||
|
||
private def renewer(hadoopConf: Configuration): String = { | ||
val tokenRenewer = Master.getMasterPrincipal(hadoopConf) | ||
debug("Delegation token renewer is: " + tokenRenewer) | ||
|
||
if (tokenRenewer == null || tokenRenewer.length() == 0) { | ||
val errorMessage = "Can't get Master Kerberos principal for use as renewer." | ||
error(errorMessage) | ||
throw new ServiceException(errorMessage) | ||
} | ||
tokenRenewer | ||
} | ||
|
||
override def obtainTokens(conf: SparkConf): Unit = { | ||
val hadoopConf = newConfiguration(conf) | ||
val tokenRenewer = renewer(hadoopConf) | ||
val creds = new Credentials() | ||
hadoopFStoAccess(conf, hadoopConf).foreach { fs => | ||
fs.addDelegationTokens(tokenRenewer, creds) | ||
} | ||
UserGroupInformation.getCurrentUser.addCredentials(creds) | ||
} | ||
} |
67 changes: 67 additions & 0 deletions
67
kyuubi-server/src/main/scala/yaooqinn/kyuubi/session/security/HiveTokenCollector.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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 yaooqinn.kyuubi.session.security | ||
|
||
import scala.util.control.NonFatal | ||
|
||
import org.apache.commons.lang3.StringUtils | ||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier | ||
import org.apache.hadoop.hive.conf.HiveConf | ||
import org.apache.hadoop.hive.ql.metadata.Hive | ||
import org.apache.hadoop.io.Text | ||
import org.apache.hadoop.security.{Credentials, UserGroupInformation} | ||
import org.apache.hadoop.security.token.Token | ||
import org.apache.spark.SparkConf | ||
|
||
import yaooqinn.kyuubi.Logging | ||
import yaooqinn.kyuubi.utils.KyuubiHadoopUtil | ||
import yaooqinn.kyuubi.utils.KyuubiHiveUtil._ | ||
|
||
private[security] object HiveTokenCollector extends TokenCollector with Logging { | ||
|
||
override def obtainTokens(conf: SparkConf): Unit = { | ||
try { | ||
val c = hiveConf(conf) | ||
val principal = c.getTrimmed(METASTORE_PRINCIPAL) | ||
val uris = c.getTrimmed(URIS) | ||
require(StringUtils.isNotEmpty(principal), METASTORE_PRINCIPAL + " Undefined") | ||
require(StringUtils.isNotEmpty(uris), URIS + " Undefined") | ||
val currentUser = UserGroupInformation.getCurrentUser.getUserName | ||
val credentials = new Credentials() | ||
KyuubiHadoopUtil.doAsRealUser { | ||
val hive = Hive.get(c, classOf[HiveConf]) | ||
info(s"Getting token from Hive Metastore for owner $currentUser via $principal") | ||
val tokenString = hive.getDelegationToken(currentUser, principal) | ||
val token = new Token[DelegationTokenIdentifier] | ||
token.decodeFromUrlString(tokenString) | ||
info(s"Got " + DelegationTokenIdentifier.stringifyToken(token)) | ||
credentials.addToken(new Text("hive.metastore.delegation.token"), token) | ||
} | ||
UserGroupInformation.getCurrentUser.addCredentials(credentials) | ||
} catch { | ||
case NonFatal(e) => | ||
error("Failed to get token from hive metatore service", e) | ||
} finally { | ||
Hive.closeCurrent() | ||
} | ||
} | ||
|
||
override def tokensRequired(conf: SparkConf): Boolean = { | ||
UserGroupInformation.isSecurityEnabled && StringUtils.isNotBlank(hiveConf(conf).get(URIS)) | ||
} | ||
} |
54 changes: 54 additions & 0 deletions
54
kyuubi-server/src/main/scala/yaooqinn/kyuubi/session/security/TokenCollector.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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 yaooqinn.kyuubi.session.security | ||
|
||
import org.apache.hadoop.security.UserGroupInformation | ||
import org.apache.spark.SparkConf | ||
|
||
/** | ||
* An interface for secured service token collectors | ||
*/ | ||
private[security] trait TokenCollector { | ||
|
||
/** | ||
* Obtain tokens from secured services, such as Hive Metastore Server. HDFS etc. | ||
* @param conf a SparkConf | ||
*/ | ||
def obtainTokens(conf: SparkConf): Unit | ||
|
||
/** | ||
* Check whether a service need tokens to visit | ||
* @param conf a SparkConf | ||
* @return true if the service to visit requires tokens | ||
*/ | ||
def tokensRequired(conf: SparkConf): Boolean = UserGroupInformation.isSecurityEnabled | ||
|
||
} | ||
|
||
private[session] object TokenCollector { | ||
|
||
/** | ||
* Obtain tokens from all secured services if required. | ||
* @param conf a SparkConf | ||
*/ | ||
def obtainTokenIfRequired(conf: SparkConf): Unit = { | ||
Seq(HiveTokenCollector, HDFSTokenCollector).foreach { co => | ||
if (co.tokensRequired(conf)) co.obtainTokens(conf) | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
37 changes: 37 additions & 0 deletions
37
kyuubi-server/src/main/scala/yaooqinn/kyuubi/utils/KyuubiHiveUtil.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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 yaooqinn.kyuubi.utils | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.hive.conf.HiveConf | ||
import org.apache.spark.{KyuubiSparkUtil, SparkConf} | ||
|
||
object KyuubiHiveUtil { | ||
|
||
private val HIVE_PREFIX = "hive." | ||
private val METASTORE_PREFIX = "metastore." | ||
|
||
val URIS: String = HIVE_PREFIX + METASTORE_PREFIX + "uris" | ||
val METASTORE_PRINCIPAL: String = HIVE_PREFIX + METASTORE_PREFIX + "kerberos.principal" | ||
|
||
def hiveConf(conf: SparkConf): Configuration = { | ||
val hadoopConf = KyuubiSparkUtil.newConfiguration(conf) | ||
new HiveConf(hadoopConf, classOf[HiveConf]) | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.