-
Notifications
You must be signed in to change notification settings - Fork 930
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Support to run HiveSQLEngine on kerberized YARN #6199
Changes from all commits
4d262c8
f84c7bc
e706936
061223d
85ce9bb
5733dfd
5d3013a
5ae0a3e
9e2d862
41fed0c
118afe2
458493a
383d1cd
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Large diffs are not rendered by default.
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,10 +17,12 @@ | |
package org.apache.kyuubi.engine.deploy.yarn | ||
|
||
import java.io._ | ||
import java.nio.ByteBuffer | ||
import java.nio.charset.StandardCharsets | ||
import java.nio.file.Files | ||
import java.security.PrivilegedExceptionAction | ||
import java.util | ||
import java.util.{Locale, Properties} | ||
import java.util.{Locale, Properties, UUID} | ||
import java.util.zip.{ZipEntry, ZipOutputStream} | ||
|
||
import scala.collection.JavaConverters._ | ||
|
@@ -30,7 +32,8 @@ import scala.collection.mutable.ListBuffer | |
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.{FileSystem, Path} | ||
import org.apache.hadoop.fs.permission.FsPermission | ||
import org.apache.hadoop.security.UserGroupInformation | ||
import org.apache.hadoop.mapred.Master | ||
import org.apache.hadoop.security.{Credentials, UserGroupInformation} | ||
import org.apache.hadoop.yarn.api.ApplicationConstants | ||
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment | ||
import org.apache.hadoop.yarn.api.records._ | ||
|
@@ -40,6 +43,7 @@ import org.apache.hadoop.yarn.util.Records | |
import org.apache.kyuubi.{KyuubiException, Logging, Utils} | ||
import org.apache.kyuubi.config.KyuubiConf | ||
import org.apache.kyuubi.config.KyuubiConf._ | ||
import org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_SESSION_USER_KEY | ||
import org.apache.kyuubi.engine.deploy.yarn.EngineYarnModeSubmitter._ | ||
import org.apache.kyuubi.util.KyuubiHadoopUtils | ||
|
||
|
@@ -81,6 +85,9 @@ abstract class EngineYarnModeSubmitter extends Logging { | |
|
||
var yarnConf: Configuration = _ | ||
var hadoopConf: Configuration = _ | ||
var appUser: String = _ | ||
var keytab: String = _ | ||
var amKeytabFileName: Option[String] = _ | ||
|
||
var engineType: String | ||
|
||
|
@@ -91,9 +98,35 @@ abstract class EngineYarnModeSubmitter extends Logging { | |
*/ | ||
def engineExtraJars(): Seq[File] = Seq.empty | ||
|
||
protected def submitApplication(): Unit = { | ||
def run(): Unit = { | ||
yarnConf = KyuubiHadoopUtils.newYarnConfiguration(kyuubiConf) | ||
hadoopConf = KyuubiHadoopUtils.newHadoopConf(kyuubiConf) | ||
appUser = kyuubiConf.getOption(KYUUBI_SESSION_USER_KEY).orNull | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. is this correct for GROUP and SERVER share level? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It's weird. I remember we use this key to distinguish the session user and app user in some cases ... There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It is confusing because we currently use There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. SGTM |
||
require(appUser != null, s"$KYUUBI_SESSION_USER_KEY is not set") | ||
keytab = kyuubiConf.get(ENGINE_KEYTAB).orNull | ||
val principal = kyuubiConf.get(ENGINE_PRINCIPAL).orNull | ||
amKeytabFileName = | ||
if (UserGroupInformation.isSecurityEnabled && principal != null && keytab != null) { | ||
info(s"Kerberos credentials: principal = $principal, keytab = $keytab") | ||
UserGroupInformation.loginUserFromKeytab(principal, keytab) | ||
// Generate a file name that can be used for the keytab file, that does not conflict | ||
// with any user file. | ||
Some(new File(keytab).getName() + "-" + UUID.randomUUID().toString) | ||
} else { | ||
None | ||
} | ||
|
||
val ugi = if (UserGroupInformation.getCurrentUser.getShortUserName == appUser) { | ||
UserGroupInformation.getCurrentUser | ||
} else { | ||
UserGroupInformation.createProxyUser(appUser, UserGroupInformation.getCurrentUser) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This also supports proxy users, right? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This should only happen when using TGT cache with YARN mode, though it does not work ... it's good to keep those code path in case some one modifies the Hive source code to make it workable, but we'd better add a warning message to explicitly tell users this is not likely to work. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. More precisely speaking, proxy user mechanism works fine during the submitting progress. AM can be started successfully. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes, exactly |
||
} | ||
ugi.doAs(new PrivilegedExceptionAction[Unit] { | ||
override def run(): Unit = submitApplication() | ||
}) | ||
} | ||
|
||
protected def submitApplication(): Unit = { | ||
try { | ||
yarnClient = YarnClient.createYarnClient() | ||
yarnClient.init(yarnConf) | ||
|
@@ -134,6 +167,29 @@ abstract class EngineYarnModeSubmitter extends Logging { | |
} | ||
} | ||
|
||
private def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { | ||
if (UserGroupInformation.isSecurityEnabled) { | ||
val credentials = obtainHadoopFsDelegationToken() | ||
val serializedCreds = KyuubiHadoopUtils.serializeCredentials(credentials) | ||
amContainer.setTokens(ByteBuffer.wrap(serializedCreds)) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why does the ApplicationMaster need to login even if the token is set on this block? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Without |
||
} | ||
} | ||
|
||
private def obtainHadoopFsDelegationToken(): Credentials = { | ||
val tokenRenewer = Master.getMasterPrincipal(hadoopConf) | ||
info(s"Delegation token renewer is: $tokenRenewer") | ||
|
||
if (tokenRenewer == null || tokenRenewer.isEmpty) { | ||
val errorMessage = "Can't get Master Kerberos principal for use as renewer." | ||
error(errorMessage) | ||
throw new KyuubiException(errorMessage) | ||
} | ||
|
||
val credentials = new Credentials() | ||
FileSystem.get(hadoopConf).addDelegationTokens(tokenRenewer, credentials) | ||
credentials | ||
} | ||
|
||
private def createContainerLaunchContext(): ContainerLaunchContext = { | ||
info("Setting up container launch context for engine AM") | ||
val env = setupLaunchEnv(kyuubiConf) | ||
|
@@ -171,6 +227,7 @@ abstract class EngineYarnModeSubmitter extends Logging { | |
amContainer.setCommands(printableCommands.asJava) | ||
info(s"Commands: ${printableCommands.mkString(" ")}") | ||
|
||
setupSecurityToken(amContainer) | ||
amContainer | ||
} | ||
|
||
|
@@ -187,6 +244,19 @@ abstract class EngineYarnModeSubmitter extends Logging { | |
|
||
distributeJars(localResources, env) | ||
distributeConf(localResources, env) | ||
|
||
// If we passed in a keytab, make sure we copy the keytab to the staging directory on | ||
// HDFS, and setup the relevant environment vars, so the AM can login again. | ||
amKeytabFileName.foreach { kt => | ||
info("To enable the AM to login from keytab, credentials are being copied over to the AM" + | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This log seems a bit redundant, because in the Kerberos environment, we explicitly require users to configure this configuration item. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Keytabs are security-sensitive files. I think it's better to tell user why we upload keytabs in place. |
||
" via the YARN Secure Distributed Cache.") | ||
distribute( | ||
new Path(new File(keytab).toURI), | ||
LocalResourceType.FILE, | ||
kt, | ||
localResources) | ||
} | ||
|
||
localResources | ||
} | ||
|
||
|
@@ -253,6 +323,7 @@ abstract class EngineYarnModeSubmitter extends Logging { | |
listDistinctFiles(yarnConf.get).foreach(putEntry) | ||
|
||
val properties = confToProperties(kyuubiConf) | ||
amKeytabFileName.foreach(kt => properties.put(ENGINE_KEYTAB.key, kt)) | ||
writePropertiesToArchive(properties, KYUUBI_CONF_FILE, confStream) | ||
} finally { | ||
confStream.close() | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The submitter process determine whether to login as well in kerberized environment?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm not sure what "determine whether to login as well" means. Can you elaborate it?