-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-27748][SS] Kafka consumer/producer password/token redaction. #24627
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 2 commits
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 |
|---|---|---|
| @@ -0,0 +1,53 @@ | ||
| /* | ||
| * 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.kafka010 | ||
|
|
||
| import org.apache.kafka.common.config.SaslConfigs | ||
|
|
||
| import org.apache.spark.SparkEnv | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.internal.config.SECRET_REDACTION_PATTERN | ||
| import org.apache.spark.util.Utils.{redact, REDACTION_REPLACEMENT_TEXT} | ||
|
|
||
| private[spark] object KafkaRedactionUtil extends Logging { | ||
| private[spark] def redactParams(params: Seq[(String, Object)]): Seq[(String, Object)] = { | ||
| val redactionPattern = Some(SparkEnv.get.conf.get(SECRET_REDACTION_PATTERN)) | ||
| params.map { case (key, value) => | ||
| if (key.equalsIgnoreCase(SaslConfigs.SASL_JAAS_CONFIG)) { | ||
| (key, redactJaasParam(value.asInstanceOf[String])) | ||
| } else { | ||
| value match { | ||
| case s: String => | ||
| val (_, newValue) = redact(redactionPattern, Seq((key, s))).head | ||
| (key, newValue) | ||
|
|
||
| case _ => | ||
| (key, value) | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private[kafka010] def redactJaasParam(param: String): String = { | ||
| if (param != null && !param.isEmpty) { | ||
| param.replaceAll("password=\".*\"", s"""password="$REDACTION_REPLACEMENT_TEXT"""") | ||
| } else { | ||
| param | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -41,6 +41,7 @@ import org.apache.spark.deploy.SparkHadoopUtil | |
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.internal.config._ | ||
| import org.apache.spark.util.Utils | ||
| import org.apache.spark.util.Utils.REDACTION_REPLACEMENT_TEXT | ||
|
|
||
| private[spark] object KafkaTokenUtil extends Logging { | ||
| val TOKEN_KIND = new Text("KAFKA_DELEGATION_TOKEN") | ||
|
|
@@ -193,7 +194,7 @@ private[spark] object KafkaTokenUtil extends Logging { | |
| | debug=${isGlobalKrbDebugEnabled()} | ||
| | useTicketCache=true | ||
| | serviceName="${clusterConf.kerberosServiceName}"; | ||
| """.stripMargin.replace("\n", "") | ||
| """.stripMargin.replace("\n", "").trim | ||
|
Contributor
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. This is unrelated fix. There were additional spaces at the end of the generated string. This change removes it. Worth to mention the code works both way but thought it's just ugly. |
||
| logDebug(s"Krb ticket cache JAAS params: $params") | ||
| params | ||
| } | ||
|
|
@@ -226,7 +227,8 @@ private[spark] object KafkaTokenUtil extends Logging { | |
| 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( | ||
| logDebug("%-15s %-15s %-15s %-25s %-15s %-15s %-15s".format( | ||
| REDACTION_REPLACEMENT_TEXT, | ||
| tokenInfo.tokenId, | ||
| tokenInfo.owner, | ||
| tokenInfo.renewersAsString, | ||
|
|
@@ -268,8 +270,8 @@ private[spark] object KafkaTokenUtil extends Logging { | |
| | serviceName="${clusterConf.kerberosServiceName}" | ||
| | username="$username" | ||
| | password="$password"; | ||
| """.stripMargin.replace("\n", "") | ||
| logDebug(s"Scram JAAS params: ${params.replaceAll("password=\".*\"", "password=\"[hidden]\"")}") | ||
| """.stripMargin.replace("\n", "").trim | ||
|
Contributor
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. This is unrelated fix. There were additional spaces at the end of the generated string. This change removes it. Worth to mention the code works both way but thought it's just ugly. |
||
| logDebug(s"Scram JAAS params: ${KafkaRedactionUtil.redactJaasParam(params)}") | ||
|
|
||
| params | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -40,6 +40,21 @@ trait KafkaDelegationTokenTest extends BeforeAndAfterEach { | |
| protected val tokenId = "tokenId" + ju.UUID.randomUUID().toString | ||
| protected val tokenPassword = "tokenPassword" + ju.UUID.randomUUID().toString | ||
|
|
||
| protected val identifier1 = "cluster1" | ||
| protected val identifier2 = "cluster2" | ||
| protected val tokenService1 = KafkaTokenUtil.getTokenService(identifier1) | ||
| protected val tokenService2 = KafkaTokenUtil.getTokenService(identifier2) | ||
| protected val bootStrapServers = "127.0.0.1:0" | ||
| protected val matchingTargetServersRegex = "127.0.0.*:0" | ||
| protected val nonMatchingTargetServersRegex = "127.0.intentionally_non_matching.*:0" | ||
| protected val trustStoreLocation = "/path/to/trustStore" | ||
| protected val trustStorePassword = "trustStoreSecret" | ||
| protected val keyStoreLocation = "/path/to/keyStore" | ||
| protected val keyStorePassword = "keyStoreSecret" | ||
| protected val keyPassword = "keySecret" | ||
| protected val keytab = "/path/to/keytab" | ||
| protected val principal = "[email protected]" | ||
|
|
||
| private class KafkaJaasConfiguration extends Configuration { | ||
| val entry = | ||
| new AppConfigurationEntry( | ||
|
|
@@ -89,4 +104,21 @@ trait KafkaDelegationTokenTest extends BeforeAndAfterEach { | |
| doReturn(conf).when(env).conf | ||
| SparkEnv.set(env) | ||
| } | ||
|
|
||
| protected def createClusterConf( | ||
| identifier: String, | ||
| securityProtocol: String): KafkaTokenClusterConf = { | ||
| KafkaTokenClusterConf( | ||
| identifier, | ||
| bootStrapServers, | ||
| KafkaTokenSparkConf.DEFAULT_TARGET_SERVERS_REGEX, | ||
| securityProtocol, | ||
| KafkaTokenSparkConf.DEFAULT_SASL_KERBEROS_SERVICE_NAME, | ||
| Some(trustStoreLocation), | ||
| Some(trustStorePassword), | ||
| Some(keyStoreLocation), | ||
| Some(keyStorePassword), | ||
| Some(keyPassword), | ||
| KafkaTokenSparkConf.DEFAULT_SASL_TOKEN_MECHANISM) | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,106 @@ | ||
| /* | ||
| * 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.kafka010 | ||
|
|
||
| import java.{util => ju} | ||
|
|
||
| import org.apache.kafka.clients.consumer.ConsumerConfig | ||
| import org.apache.kafka.common.config.{SaslConfigs, SslConfigs} | ||
| import org.apache.kafka.common.security.auth.SecurityProtocol.SASL_SSL | ||
| import org.apache.kafka.common.serialization.StringDeserializer | ||
|
|
||
| import org.apache.spark.SparkFunSuite | ||
| import org.apache.spark.util.Utils.REDACTION_REPLACEMENT_TEXT | ||
|
|
||
| class KafkaRedactionUtilSuite extends SparkFunSuite with KafkaDelegationTokenTest { | ||
| test("redactParams should give back empty parameters") { | ||
| setSparkEnv(Map()) | ||
| assert(KafkaRedactionUtil.redactParams(Seq()) === Seq()) | ||
| } | ||
|
|
||
| test("redactParams should give back non String parameters") { | ||
| setSparkEnv(Map()) | ||
| val kafkaParams = Seq( | ||
| ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> classOf[StringDeserializer] | ||
| ) | ||
|
|
||
| assert(KafkaRedactionUtil.redactParams(kafkaParams) === kafkaParams) | ||
| } | ||
|
|
||
| test("redactParams should redact token password from parameters") { | ||
| setSparkEnv(Map()) | ||
| val groupId = "id-" + ju.UUID.randomUUID().toString | ||
| addTokenToUGI(tokenService1) | ||
| val clusterConf = createClusterConf(identifier1, SASL_SSL.name) | ||
| val jaasParams = KafkaTokenUtil.getTokenJaasParams(clusterConf) | ||
| val kafkaParams = Seq( | ||
| ConsumerConfig.GROUP_ID_CONFIG -> groupId, | ||
| SaslConfigs.SASL_JAAS_CONFIG -> jaasParams | ||
| ) | ||
|
|
||
| val redactedParams = KafkaRedactionUtil.redactParams(kafkaParams).toMap | ||
|
|
||
| assert(redactedParams.get(ConsumerConfig.GROUP_ID_CONFIG).get.asInstanceOf[String] | ||
| === groupId) | ||
| val redactedJaasParams = redactedParams.get(SaslConfigs.SASL_JAAS_CONFIG).get | ||
| .asInstanceOf[String] | ||
| assert(redactedJaasParams.contains(tokenId)) | ||
| assert(!redactedJaasParams.contains(tokenPassword)) | ||
| } | ||
|
|
||
| test("redactParams should redact passwords from parameters") { | ||
| setSparkEnv(Map()) | ||
| val groupId = "id-" + ju.UUID.randomUUID().toString | ||
| val kafkaParams = Seq( | ||
| ConsumerConfig.GROUP_ID_CONFIG -> groupId, | ||
| SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG -> trustStorePassword, | ||
| SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG -> keyStorePassword, | ||
| SslConfigs.SSL_KEY_PASSWORD_CONFIG -> keyPassword | ||
| ) | ||
|
|
||
| val redactedParams = KafkaRedactionUtil.redactParams(kafkaParams).toMap | ||
|
|
||
| assert(redactedParams(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] | ||
| === groupId) | ||
| assert(redactedParams(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).asInstanceOf[String] | ||
| === REDACTION_REPLACEMENT_TEXT) | ||
| assert(redactedParams(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).asInstanceOf[String] | ||
| === REDACTION_REPLACEMENT_TEXT) | ||
| assert(redactedParams(SslConfigs.SSL_KEY_PASSWORD_CONFIG).asInstanceOf[String] | ||
| === REDACTION_REPLACEMENT_TEXT) | ||
| } | ||
|
|
||
| test("redactJaasParam should give back null") { | ||
| assert(KafkaRedactionUtil.redactJaasParam(null) === null) | ||
| } | ||
|
|
||
| test("redactJaasParam should give back empty string") { | ||
| assert(KafkaRedactionUtil.redactJaasParam("") === "") | ||
| } | ||
|
|
||
| test("redactJaasParam should redact token password") { | ||
| addTokenToUGI(tokenService1) | ||
| val clusterConf = createClusterConf(identifier1, SASL_SSL.name) | ||
| val jaasParams = KafkaTokenUtil.getTokenJaasParams(clusterConf) | ||
|
|
||
| val redactedJaasParams = KafkaRedactionUtil.redactJaasParam(jaasParams) | ||
|
|
||
| assert(redactedJaasParams.contains(tokenId)) | ||
| assert(!redactedJaasParams.contains(tokenPassword)) | ||
| } | ||
| } |
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.
Hi, @gaborgsomogyi .
Is this the only reason why we cannot use
Utils.redactdirectly?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.
@dongjoon-hyun
Not sure what you mean only reason. The short answer is yes.
A little but more detailed
SaslConfigs.SASL_JAAS_CONFIGhas different format than any other property. A normal property looks like the following:Key=ssl.truststore.password, Value=secret.SaslConfigs.SASL_JAAS_CONFIGhowever have the following syntax:Key=sasl.jaas.config, Value=org.apache.kafka.common.security.scram.ScramLoginModule required tokenauth=true serviceName="kafka" username="admin" password="admin-secret";Utils.redactmakes a malformed and unreadable string out of it.