-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-16742] Mesos Kerberos Support #17665
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
Changes from all commits
ce63a9b
75d849a
35002f2
13981c8
af4a3e4
5cc66dc
a47c9c0
c8ec049
954eeff
2d76928
d8a968d
b8093c8
25d5088
4c387eb
e32afee
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -30,6 +30,7 @@ import scala.util.Properties | |
| import org.apache.commons.lang3.StringUtils | ||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.security.UserGroupInformation | ||
| import org.apache.hadoop.yarn.conf.YarnConfiguration | ||
|
Contributor
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. Hmm... you're adding a YARN dependency in
Author
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. Yea it looks like this should require FYI, we originally talked about placing the code below into the Mesos scheduler, but that seems to be too late. The
Contributor
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. I don't think you need the explicit dependency (otherwise this would not be compiling). This is probably being brought by hadoop-client already.
Author
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. Better to explicitly declare a dependency rather than rely on transitivity, right? |
||
| import org.apache.ivy.Ivy | ||
| import org.apache.ivy.core.LogOptions | ||
| import org.apache.ivy.core.module.descriptor._ | ||
|
|
@@ -45,6 +46,7 @@ import org.apache.ivy.plugins.resolver.{ChainResolver, FileSystemResolver, IBibl | |
| import org.apache.spark._ | ||
| import org.apache.spark.api.r.RUtils | ||
| import org.apache.spark.deploy.rest._ | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.launcher.SparkLauncher | ||
| import org.apache.spark.util._ | ||
|
|
||
|
|
@@ -63,7 +65,7 @@ private[deploy] object SparkSubmitAction extends Enumeration { | |
| * This program handles setting up the classpath with relevant Spark dependencies and provides | ||
| * a layer over the different cluster managers and deploy modes that Spark supports. | ||
| */ | ||
| object SparkSubmit extends CommandLineUtils { | ||
| object SparkSubmit extends CommandLineUtils with Logging { | ||
|
|
||
| // Cluster managers | ||
| private val YARN = 1 | ||
|
|
@@ -564,12 +566,22 @@ object SparkSubmit extends CommandLineUtils { | |
| // properties and then loaded by SparkConf | ||
| sysProps.put("spark.yarn.keytab", args.keytab) | ||
| sysProps.put("spark.yarn.principal", args.principal) | ||
|
|
||
| UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab) | ||
| } | ||
| } | ||
| } | ||
|
|
||
|
|
||
| // [SPARK-20328]. HadoopRDD calls into a Hadoop library that fetches delegation tokens with | ||
| // renewer set to the YARN ResourceManager. Since YARN isn't configured in Mesos mode, we | ||
| // must trick it into thinking we're YARN. | ||
| if (clusterManager == MESOS && UserGroupInformation.isSecurityEnabled) { | ||
| val shortUserName = UserGroupInformation.getCurrentUser.getShortUserName | ||
| val key = s"spark.hadoop.${YarnConfiguration.RM_PRINCIPAL}" | ||
|
Contributor
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. Does it work in user impersonation scenario? Here
Author
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 it? It looks like Regardless, the renewer specified here actually has no effect, since we aren't renewing yet. Once I add renewal, I'll need to revisit this to make sure it's consistent with the renewal we're going to do in
Contributor
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. The renewer for delegation tokens should be the user that created them (if the service is not doing it for the user, like in the YARN case); only the logged in user can create tokens, so this has to be the current user (not the proxy user, which happens later). The tokens should be created in the proxy user's name (so user = "proxy" renewer = "real user"), when you care to support that.
So do you mean that currently this only works until the delegation tokens need renewal (which is 1 day by default - a lot shorter than the max life time)?
Author
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 |
||
|
|
||
| logDebug(s"Setting ${key} to ${shortUserName}.") | ||
| sysProps.put(key, shortUserName) | ||
| } | ||
|
|
||
| // In yarn-cluster mode, use yarn.Client as a wrapper around the user class | ||
| if (isYarnCluster) { | ||
| childMainClass = "org.apache.spark.deploy.yarn.Client" | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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.spark.deploy.security | ||
|
|
||
| import java.io.{ByteArrayOutputStream, DataOutputStream} | ||
|
|
||
| import org.apache.hadoop.security.Credentials | ||
|
|
||
| class CredentialsSerializer { | ||
| def serializeTokens(creds: Credentials): Array[Byte] = { | ||
| val byteStream = new ByteArrayOutputStream | ||
| val dataStream = new DataOutputStream(byteStream) | ||
| creds.writeTokenStorageToStream(dataStream) | ||
| byteStream.toByteArray | ||
| } | ||
|
|
||
| def deserializeTokens(tokenBytes: Array[Byte]): Credentials = { | ||
| val tokensBuf = new java.io.ByteArrayInputStream(tokenBytes) | ||
|
|
||
| val creds = new Credentials() | ||
| creds.readTokenStorageStream(new java.io.DataInputStream(tokensBuf)) | ||
| creds | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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.spark.deploy.security | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.{FileSystem, Path} | ||
| import org.apache.hadoop.security.{Credentials, UserGroupInformation} | ||
| import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier | ||
|
|
||
| import org.apache.spark.SparkConf | ||
| import org.apache.spark.internal.Logging | ||
|
|
||
| private[deploy] class HadoopFSCredentialProvider | ||
| extends ServiceCredentialProvider with Logging { | ||
| // Token renewal interval, this value will be set in the first call, | ||
| // if None means no token renewer specified or no token can be renewed, | ||
| // so cannot get token renewal interval. | ||
| private var tokenRenewalInterval: Option[Long] = null | ||
|
|
||
| override val serviceName: String = "hadoopfs" | ||
|
|
||
| override def obtainCredentials( | ||
| hadoopConf: Configuration, | ||
| sparkConf: SparkConf, | ||
| creds: Credentials): Option[Long] = { | ||
| // NameNode to access, used to get tokens from different FileSystems | ||
| val tmpCreds = new Credentials() | ||
| val tokenRenewer = getTokenRenewer(hadoopConf) | ||
| hadoopFSsToAccess(hadoopConf, sparkConf).foreach { dst => | ||
| val dstFs = dst.getFileSystem(hadoopConf) | ||
| logInfo("getting token for: " + dst) | ||
| dstFs.addDelegationTokens(tokenRenewer, tmpCreds) | ||
| } | ||
|
|
||
| // Get the token renewal interval if it is not set. It will only be called once. | ||
| if (tokenRenewalInterval == null) { | ||
| tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf) | ||
| } | ||
|
|
||
| // Get the time of next renewal. | ||
| val nextRenewalDate = tokenRenewalInterval.flatMap { interval => | ||
| val nextRenewalDates = tmpCreds.getAllTokens.asScala | ||
| .filter(_.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier]) | ||
| .map { t => | ||
| val identifier = t.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] | ||
| identifier.getIssueDate + interval | ||
| } | ||
| if (nextRenewalDates.isEmpty) None else Some(nextRenewalDates.min) | ||
| } | ||
|
|
||
| creds.addAll(tmpCreds) | ||
| nextRenewalDate | ||
| } | ||
|
|
||
| protected def getTokenRenewalInterval( | ||
| hadoopConf: Configuration, | ||
| sparkConf: SparkConf): Option[Long] = None | ||
|
|
||
| protected def getTokenRenewer(hadoopConf: Configuration): String = { | ||
| UserGroupInformation.getCurrentUser.getShortUserName | ||
| } | ||
|
|
||
| protected def hadoopFSsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = { | ||
| Set(FileSystem.get(hadoopConf).getHomeDirectory) | ||
| } | ||
| } |
Uh oh!
There was an error while loading. Please reload this page.
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 have a problem here that I could use some help on.
hive-exechas a dependency onpentahothat doesn't exist in maven central anymore. This is why we have an entry for it inspark-parent'sdependencyManagementsection that excludespentahoviacalcite-core.But the
scopeinspark-parent'sdependencyManagemententry forhive-execiscompile, so if I make ittesthere, which is what it was in YARN, then it fails to match that entry anymore, and thus doesn't inherit the excludes. This shouldn't be a problem, because dependencies in thetestscope aren't transitive, so we should never be trying to fetch the missingpentahodependency. Butsbtbuilds don't seem to recognize this transitivity, so if I add thetestscope, I get this error:What I can't understand is how this wasn't a problem when these dependencies were in YARN. For some reason
sbt dependencyTreeonmastershows thatcalcite-coreis excluded from thehive-execdependency tree, but the same command on this PR tries to resolvecalcite-core.Let me know if you have any ideas here. My best idea right now is to just try to duplicate the excludes in
spark-parent.