@@ -25,25 +25,43 @@ import org.apache.kafka.common.security.auth.SecurityProtocol.SASL_SSL
2525import org .apache .kafka .common .serialization .StringDeserializer
2626
2727import org .apache .spark .SparkFunSuite
28+ import org .apache .spark .internal .config .SECRET_REDACTION_PATTERN
2829import org .apache .spark .util .Utils .REDACTION_REPLACEMENT_TEXT
2930
3031class KafkaRedactionUtilSuite extends SparkFunSuite with KafkaDelegationTokenTest {
3132 test(" redactParams should give back empty parameters" ) {
32- setSparkEnv(Map () )
33+ setSparkEnv(Map .empty )
3334 assert(KafkaRedactionUtil .redactParams(Seq ()) === Seq ())
3435 }
3536
36- test(" redactParams should give back non String parameters " ) {
37- setSparkEnv(Map () )
37+ test(" redactParams should give back null value " ) {
38+ setSparkEnv(Map .empty )
3839 val kafkaParams = Seq (
39- ConsumerConfig .KEY_DESERIALIZER_CLASS_CONFIG -> classOf [ StringDeserializer ]
40+ ConsumerConfig .KEY_DESERIALIZER_CLASS_CONFIG -> null
4041 )
4142
4243 assert(KafkaRedactionUtil .redactParams(kafkaParams) === kafkaParams)
4344 }
4445
46+ test(" redactParams should redact non String parameters" ) {
47+ setSparkEnv(
48+ Map (
49+ SECRET_REDACTION_PATTERN .key -> ConsumerConfig .KEY_DESERIALIZER_CLASS_CONFIG
50+ )
51+ )
52+ val kafkaParams = Seq (
53+ ConsumerConfig .KEY_DESERIALIZER_CLASS_CONFIG -> classOf [StringDeserializer ]
54+ )
55+
56+ val redactedParams = KafkaRedactionUtil .redactParams(kafkaParams).toMap
57+
58+ assert(redactedParams.size === 1 )
59+ assert(redactedParams.get(ConsumerConfig .KEY_DESERIALIZER_CLASS_CONFIG ).get
60+ === REDACTION_REPLACEMENT_TEXT )
61+ }
62+
4563 test(" redactParams should redact token password from parameters" ) {
46- setSparkEnv(Map () )
64+ setSparkEnv(Map .empty )
4765 val groupId = " id-" + ju.UUID .randomUUID().toString
4866 addTokenToUGI(tokenService1)
4967 val clusterConf = createClusterConf(identifier1, SASL_SSL .name)
@@ -55,16 +73,15 @@ class KafkaRedactionUtilSuite extends SparkFunSuite with KafkaDelegationTokenTes
5573
5674 val redactedParams = KafkaRedactionUtil .redactParams(kafkaParams).toMap
5775
58- assert(redactedParams.get( ConsumerConfig . GROUP_ID_CONFIG ).get. asInstanceOf [ String ]
59- === groupId)
76+ assert(redactedParams.size === 2 )
77+ assert(redactedParams.get( ConsumerConfig . GROUP_ID_CONFIG ).get === groupId)
6078 val redactedJaasParams = redactedParams.get(SaslConfigs .SASL_JAAS_CONFIG ).get
61- .asInstanceOf [String ]
6279 assert(redactedJaasParams.contains(tokenId))
6380 assert(! redactedJaasParams.contains(tokenPassword))
6481 }
6582
6683 test(" redactParams should redact passwords from parameters" ) {
67- setSparkEnv(Map () )
84+ setSparkEnv(Map .empty )
6885 val groupId = " id-" + ju.UUID .randomUUID().toString
6986 val kafkaParams = Seq (
7087 ConsumerConfig .GROUP_ID_CONFIG -> groupId,
@@ -75,14 +92,11 @@ class KafkaRedactionUtilSuite extends SparkFunSuite with KafkaDelegationTokenTes
7592
7693 val redactedParams = KafkaRedactionUtil .redactParams(kafkaParams).toMap
7794
78- assert(redactedParams(ConsumerConfig .GROUP_ID_CONFIG ).asInstanceOf [String ]
79- === groupId)
80- assert(redactedParams(SslConfigs .SSL_TRUSTSTORE_PASSWORD_CONFIG ).asInstanceOf [String ]
81- === REDACTION_REPLACEMENT_TEXT )
82- assert(redactedParams(SslConfigs .SSL_KEYSTORE_PASSWORD_CONFIG ).asInstanceOf [String ]
83- === REDACTION_REPLACEMENT_TEXT )
84- assert(redactedParams(SslConfigs .SSL_KEY_PASSWORD_CONFIG ).asInstanceOf [String ]
85- === REDACTION_REPLACEMENT_TEXT )
95+ assert(redactedParams.size === 4 )
96+ assert(redactedParams(ConsumerConfig .GROUP_ID_CONFIG ) === groupId)
97+ assert(redactedParams(SslConfigs .SSL_TRUSTSTORE_PASSWORD_CONFIG ) === REDACTION_REPLACEMENT_TEXT )
98+ assert(redactedParams(SslConfigs .SSL_KEYSTORE_PASSWORD_CONFIG ) === REDACTION_REPLACEMENT_TEXT )
99+ assert(redactedParams(SslConfigs .SSL_KEY_PASSWORD_CONFIG ) === REDACTION_REPLACEMENT_TEXT )
86100 }
87101
88102 test(" redactJaasParam should give back null" ) {
0 commit comments