-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
[SPARK-25501][SS] Add kafka delegation token support. #22598
Closed
Closed
Changes from all commits
Commits
Show all changes
25 commits
Select commit
Hold shift + click to select a range
f9b4685
[SPARK-25501][SS] Add kafka delegation token support.
gaborgsomogyi 2a79d59
Review fixes:
gaborgsomogyi a4ab4f5
Review fixes:
gaborgsomogyi 8c860fe
Small cosmetic change
gaborgsomogyi a2c4397
Leftover from last review fix commit
gaborgsomogyi fffd139
Review fixes:
gaborgsomogyi 7c22433
Review fixes:
gaborgsomogyi d7af8ef
Compile fix
gaborgsomogyi 1badd33
Merge branch 'master' into SPARK-25501
gaborgsomogyi caf7f1f
Providing renewal date
gaborgsomogyi deea82f
Compile fix
gaborgsomogyi d938594
Move TokenUtil to core with provided kafka dependency to make DStream…
gaborgsomogyi 17d25f2
Compile fix
gaborgsomogyi 47d79ef
Compile fix
gaborgsomogyi b4541cd
Review fixes
gaborgsomogyi 2a0cdb7
Review fixes:
gaborgsomogyi 396c80a
Merge branch 'master' into SPARK-25501
gaborgsomogyi 5741917
Update kafka version to 2.1.0 because SPARK-25954 just resolved
gaborgsomogyi 30df8f1
Add SSL protocol support with two-way authentication
gaborgsomogyi 36d05d2
Review fixes:
gaborgsomogyi cbff31c
Add ticket cache dymanic configuration
gaborgsomogyi 1eceaa9
Merge branch 'master' into SPARK-25501
gaborgsomogyi 2a0b1c3
Remove token cache usage without delegation token
gaborgsomogyi c0519f8
Review fixes:
gaborgsomogyi a122865
Remove unnecessary modifiers
gaborgsomogyi File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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
61 changes: 61 additions & 0 deletions
61
core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.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,61 @@ | ||
/* | ||
* 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.language.existentials | ||
import scala.util.control.NonFatal | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.security.Credentials | ||
import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, SASL_SSL, SSL} | ||
|
||
import org.apache.spark.SparkConf | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.internal.config._ | ||
|
||
private[security] class KafkaDelegationTokenProvider | ||
gaborgsomogyi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
extends HadoopDelegationTokenProvider with Logging { | ||
|
||
override def serviceName: String = "kafka" | ||
|
||
override def obtainDelegationTokens( | ||
gaborgsomogyi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
hadoopConf: Configuration, | ||
sparkConf: SparkConf, | ||
creds: Credentials): Option[Long] = { | ||
try { | ||
logDebug("Attempting to fetch Kafka security token.") | ||
val (token, nextRenewalDate) = KafkaTokenUtil.obtainToken(sparkConf) | ||
creds.addToken(token.getService, token) | ||
return Some(nextRenewalDate) | ||
} catch { | ||
case NonFatal(e) => | ||
logInfo(s"Failed to get token from service $serviceName", e) | ||
} | ||
None | ||
gaborgsomogyi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
|
||
override def delegationTokensRequired( | ||
gaborgsomogyi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
sparkConf: SparkConf, | ||
hadoopConf: Configuration): Boolean = { | ||
val protocol = sparkConf.get(Kafka.SECURITY_PROTOCOL) | ||
sparkConf.contains(Kafka.BOOTSTRAP_SERVERS) && | ||
(protocol == SASL_SSL.name || | ||
protocol == SSL.name || | ||
protocol == SASL_PLAINTEXT.name) | ||
} | ||
} |
202 changes: 202 additions & 0 deletions
202
core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.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,202 @@ | ||
/* | ||
* 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.{ util => ju } | ||
import java.text.SimpleDateFormat | ||
|
||
import scala.util.control.NonFatal | ||
|
||
import org.apache.hadoop.io.Text | ||
import org.apache.hadoop.security.token.{Token, TokenIdentifier} | ||
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier | ||
import org.apache.kafka.clients.CommonClientConfigs | ||
import org.apache.kafka.clients.admin.{AdminClient, CreateDelegationTokenOptions} | ||
import org.apache.kafka.common.config.SaslConfigs | ||
import org.apache.kafka.common.security.JaasContext | ||
import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, SASL_SSL, SSL} | ||
import org.apache.kafka.common.security.token.delegation.DelegationToken | ||
|
||
import org.apache.spark.SparkConf | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.internal.config._ | ||
|
||
private[spark] object KafkaTokenUtil extends Logging { | ||
val TOKEN_KIND = new Text("KAFKA_DELEGATION_TOKEN") | ||
val TOKEN_SERVICE = new Text("kafka.server.delegation.token") | ||
|
||
private[spark] class KafkaDelegationTokenIdentifier extends AbstractDelegationTokenIdentifier { | ||
override def getKind: Text = TOKEN_KIND | ||
} | ||
|
||
private[security] def obtainToken(sparkConf: SparkConf): (Token[_ <: TokenIdentifier], Long) = { | ||
val adminClient = AdminClient.create(createAdminClientProperties(sparkConf)) | ||
val createDelegationTokenOptions = new CreateDelegationTokenOptions() | ||
val createResult = adminClient.createDelegationToken(createDelegationTokenOptions) | ||
val token = createResult.delegationToken().get() | ||
printToken(token) | ||
|
||
(new Token[KafkaDelegationTokenIdentifier]( | ||
token.tokenInfo.tokenId.getBytes, | ||
token.hmacAsBase64String.getBytes, | ||
TOKEN_KIND, | ||
TOKEN_SERVICE | ||
), token.tokenInfo.expiryTimestamp) | ||
} | ||
|
||
private[security] def createAdminClientProperties(sparkConf: SparkConf): ju.Properties = { | ||
val adminClientProperties = new ju.Properties | ||
|
||
val bootstrapServers = sparkConf.get(Kafka.BOOTSTRAP_SERVERS) | ||
require(bootstrapServers.nonEmpty, s"Tried to obtain kafka delegation token but bootstrap " + | ||
"servers not configured.") | ||
adminClientProperties.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers.get) | ||
|
||
val protocol = sparkConf.get(Kafka.SECURITY_PROTOCOL) | ||
adminClientProperties.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, protocol) | ||
protocol match { | ||
case SASL_SSL.name => | ||
setTrustStoreProperties(sparkConf, adminClientProperties) | ||
|
||
case SSL.name => | ||
setTrustStoreProperties(sparkConf, adminClientProperties) | ||
setKeyStoreProperties(sparkConf, adminClientProperties) | ||
logWarning("Obtaining kafka delegation token with SSL protocol. Please " + | ||
"configure 2-way authentication on the broker side.") | ||
|
||
case SASL_PLAINTEXT.name => | ||
logWarning("Obtaining kafka delegation token through plain communication channel. Please " + | ||
"consider the security impact.") | ||
} | ||
|
||
// There are multiple possibilities to log in and applied in the following order: | ||
gaborgsomogyi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
// - JVM global security provided -> try to log in with JVM global security configuration | ||
// which can be configured for example with 'java.security.auth.login.config'. | ||
// For this no additional parameter needed. | ||
// - Keytab is provided -> try to log in with kerberos module and keytab using kafka's dynamic | ||
// JAAS configuration. | ||
// - Keytab not provided -> try to log in with kerberos module and ticket cache using kafka's | ||
// dynamic JAAS configuration. | ||
// Kafka client is unable to use subject from JVM which already logged in | ||
// to kdc (see KAFKA-7677) | ||
if (isGlobalJaasConfigurationProvided) { | ||
logDebug("JVM global security configuration detected, using it for login.") | ||
} else { | ||
adminClientProperties.put(SaslConfigs.SASL_MECHANISM, SaslConfigs.GSSAPI_MECHANISM) | ||
if (sparkConf.contains(KEYTAB)) { | ||
logDebug("Keytab detected, using it for login.") | ||
val jaasParams = getKeytabJaasParams(sparkConf) | ||
adminClientProperties.put(SaslConfigs.SASL_JAAS_CONFIG, jaasParams) | ||
} else { | ||
logDebug("Using ticket cache for login.") | ||
val jaasParams = getTicketCacheJaasParams(sparkConf) | ||
adminClientProperties.put(SaslConfigs.SASL_JAAS_CONFIG, jaasParams) | ||
} | ||
} | ||
|
||
adminClientProperties | ||
} | ||
|
||
def isGlobalJaasConfigurationProvided: Boolean = { | ||
try { | ||
JaasContext.loadClientContext(ju.Collections.emptyMap[String, Object]()) | ||
true | ||
} catch { | ||
case NonFatal(_) => false | ||
} | ||
} | ||
|
||
private def setTrustStoreProperties(sparkConf: SparkConf, properties: ju.Properties): Unit = { | ||
sparkConf.get(Kafka.TRUSTSTORE_LOCATION).foreach { truststoreLocation => | ||
properties.put("ssl.truststore.location", truststoreLocation) | ||
} | ||
sparkConf.get(Kafka.TRUSTSTORE_PASSWORD).foreach { truststorePassword => | ||
properties.put("ssl.truststore.password", truststorePassword) | ||
} | ||
} | ||
|
||
private def setKeyStoreProperties(sparkConf: SparkConf, properties: ju.Properties): Unit = { | ||
sparkConf.get(Kafka.KEYSTORE_LOCATION).foreach { keystoreLocation => | ||
properties.put("ssl.keystore.location", keystoreLocation) | ||
} | ||
sparkConf.get(Kafka.KEYSTORE_PASSWORD).foreach { keystorePassword => | ||
properties.put("ssl.keystore.password", keystorePassword) | ||
} | ||
sparkConf.get(Kafka.KEY_PASSWORD).foreach { keyPassword => | ||
properties.put("ssl.key.password", keyPassword) | ||
} | ||
} | ||
|
||
private[security] def getKeytabJaasParams(sparkConf: SparkConf): String = { | ||
val serviceName = sparkConf.get(Kafka.KERBEROS_SERVICE_NAME) | ||
require(serviceName.nonEmpty, "Kerberos service name must be defined") | ||
|
||
val params = | ||
s""" | ||
|${getKrb5LoginModuleName} required | ||
| useKeyTab=true | ||
| serviceName="${serviceName.get}" | ||
| keyTab="${sparkConf.get(KEYTAB).get}" | ||
| principal="${sparkConf.get(PRINCIPAL).get}"; | ||
""".stripMargin.replace("\n", "") | ||
logDebug(s"Krb keytab JAAS params: $params") | ||
params | ||
} | ||
|
||
def getTicketCacheJaasParams(sparkConf: SparkConf): String = { | ||
val serviceName = sparkConf.get(Kafka.KERBEROS_SERVICE_NAME) | ||
require(serviceName.nonEmpty, "Kerberos service name must be defined") | ||
|
||
val params = | ||
s""" | ||
|${getKrb5LoginModuleName} required | ||
| useTicketCache=true | ||
| serviceName="${serviceName.get}"; | ||
""".stripMargin.replace("\n", "") | ||
logDebug(s"Krb ticket cache JAAS params: $params") | ||
params | ||
} | ||
|
||
/** | ||
* Krb5LoginModule package vary in different JVMs. | ||
* Please see Hadoop UserGroupInformation for further details. | ||
*/ | ||
private def getKrb5LoginModuleName(): String = { | ||
if (System.getProperty("java.vendor").contains("IBM")) { | ||
"com.ibm.security.auth.module.Krb5LoginModule" | ||
} else { | ||
"com.sun.security.auth.module.Krb5LoginModule" | ||
} | ||
} | ||
|
||
private def printToken(token: DelegationToken): Unit = { | ||
if (log.isDebugEnabled) { | ||
val dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm") | ||
logDebug("%-15s %-30s %-15s %-25s %-15s %-15s %-15s".format( | ||
"TOKENID", "HMAC", "OWNER", "RENEWERS", "ISSUEDATE", "EXPIRYDATE", "MAXDATE")) | ||
val tokenInfo = token.tokenInfo | ||
logDebug("%-15s [hidden] %-15s %-25s %-15s %-15s %-15s".format( | ||
tokenInfo.tokenId, | ||
tokenInfo.owner, | ||
tokenInfo.renewersAsString, | ||
dateFormat.format(tokenInfo.issueTimestamp), | ||
dateFormat.format(tokenInfo.expiryTimestamp), | ||
dateFormat.format(tokenInfo.maxTimestamp))) | ||
} | ||
} | ||
} |
82 changes: 82 additions & 0 deletions
82
core/src/main/scala/org/apache/spark/internal/config/Kafka.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,82 @@ | ||
/* | ||
* 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.internal.config | ||
|
||
private[spark] object Kafka { | ||
|
||
val BOOTSTRAP_SERVERS = | ||
ConfigBuilder("spark.kafka.bootstrap.servers") | ||
.doc("A list of coma separated host/port pairs to use for establishing the initial " + | ||
"connection to the Kafka cluster. For further details please see kafka documentation. " + | ||
"Only used to obtain delegation token.") | ||
.stringConf | ||
.createOptional | ||
|
||
val SECURITY_PROTOCOL = | ||
ConfigBuilder("spark.kafka.security.protocol") | ||
.doc("Protocol used to communicate with brokers. For further details please see kafka " + | ||
"documentation. Only used to obtain delegation token.") | ||
.stringConf | ||
.createWithDefault("SASL_SSL") | ||
|
||
val KERBEROS_SERVICE_NAME = | ||
ConfigBuilder("spark.kafka.sasl.kerberos.service.name") | ||
.doc("The Kerberos principal name that Kafka runs as. This can be defined either in " + | ||
"Kafka's JAAS config or in Kafka's config. For further details please see kafka " + | ||
"documentation. Only used to obtain delegation token.") | ||
.stringConf | ||
.createOptional | ||
|
||
val TRUSTSTORE_LOCATION = | ||
ConfigBuilder("spark.kafka.ssl.truststore.location") | ||
.doc("The location of the trust store file. For further details please see kafka " + | ||
"documentation. Only used to obtain delegation token.") | ||
.stringConf | ||
.createOptional | ||
|
||
val TRUSTSTORE_PASSWORD = | ||
ConfigBuilder("spark.kafka.ssl.truststore.password") | ||
.doc("The store password for the trust store file. This is optional for client and only " + | ||
"needed if ssl.truststore.location is configured. For further details please see kafka " + | ||
"documentation. Only used to obtain delegation token.") | ||
.stringConf | ||
.createOptional | ||
|
||
val KEYSTORE_LOCATION = | ||
ConfigBuilder("spark.kafka.ssl.keystore.location") | ||
.doc("The location of the key store file. This is optional for client and can be used for " + | ||
"two-way authentication for client. For further details please see kafka documentation. " + | ||
"Only used to obtain delegation token.") | ||
.stringConf | ||
.createOptional | ||
|
||
val KEYSTORE_PASSWORD = | ||
ConfigBuilder("spark.kafka.ssl.keystore.password") | ||
.doc("The store password for the key store file. This is optional for client and only " + | ||
"needed if ssl.keystore.location is configured. For further details please see kafka " + | ||
"documentation. Only used to obtain delegation token.") | ||
.stringConf | ||
.createOptional | ||
|
||
val KEY_PASSWORD = | ||
ConfigBuilder("spark.kafka.ssl.key.password") | ||
.doc("The password of the private key in the key store file. This is optional for client. " + | ||
"For further details please see kafka documentation. Only used to obtain delegation token.") | ||
.stringConf | ||
.createOptional | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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 starting to think we should pull all this kerberos stuff into a separate module... then we wouldn't need this kind of hack in core.
That's obviously for a separate discussion, though.
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.
Yeah, fully agree. Neither copy-paste nor having provided dependency in core is a clean solution.
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.
+1 for it in one place, easier to maintain what is a security-critical piece of code. -1 if ends up something like UGI which everyone is scared to go near