|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.deploy.security |
| 19 | + |
| 20 | +import java.{ util => ju } |
| 21 | +import java.text.SimpleDateFormat |
| 22 | + |
| 23 | +import scala.util.control.NonFatal |
| 24 | + |
| 25 | +import org.apache.hadoop.io.Text |
| 26 | +import org.apache.hadoop.security.token.{Token, TokenIdentifier} |
| 27 | +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier |
| 28 | +import org.apache.kafka.clients.CommonClientConfigs |
| 29 | +import org.apache.kafka.clients.admin.{AdminClient, CreateDelegationTokenOptions} |
| 30 | +import org.apache.kafka.common.config.SaslConfigs |
| 31 | +import org.apache.kafka.common.security.JaasContext |
| 32 | +import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, SASL_SSL, SSL} |
| 33 | +import org.apache.kafka.common.security.token.delegation.DelegationToken |
| 34 | + |
| 35 | +import org.apache.spark.SparkConf |
| 36 | +import org.apache.spark.internal.Logging |
| 37 | +import org.apache.spark.internal.config._ |
| 38 | + |
| 39 | +private[spark] object KafkaTokenUtil extends Logging { |
| 40 | + val TOKEN_KIND = new Text("KAFKA_DELEGATION_TOKEN") |
| 41 | + val TOKEN_SERVICE = new Text("kafka.server.delegation.token") |
| 42 | + |
| 43 | + private[spark] class KafkaDelegationTokenIdentifier extends AbstractDelegationTokenIdentifier { |
| 44 | + override def getKind: Text = TOKEN_KIND |
| 45 | + } |
| 46 | + |
| 47 | + private[security] def obtainToken(sparkConf: SparkConf): (Token[_ <: TokenIdentifier], Long) = { |
| 48 | + val adminClient = AdminClient.create(createAdminClientProperties(sparkConf)) |
| 49 | + val createDelegationTokenOptions = new CreateDelegationTokenOptions() |
| 50 | + val createResult = adminClient.createDelegationToken(createDelegationTokenOptions) |
| 51 | + val token = createResult.delegationToken().get() |
| 52 | + printToken(token) |
| 53 | + |
| 54 | + (new Token[KafkaDelegationTokenIdentifier]( |
| 55 | + token.tokenInfo.tokenId.getBytes, |
| 56 | + token.hmacAsBase64String.getBytes, |
| 57 | + TOKEN_KIND, |
| 58 | + TOKEN_SERVICE |
| 59 | + ), token.tokenInfo.expiryTimestamp) |
| 60 | + } |
| 61 | + |
| 62 | + private[security] def createAdminClientProperties(sparkConf: SparkConf): ju.Properties = { |
| 63 | + val adminClientProperties = new ju.Properties |
| 64 | + |
| 65 | + val bootstrapServers = sparkConf.get(Kafka.BOOTSTRAP_SERVERS) |
| 66 | + require(bootstrapServers.nonEmpty, s"Tried to obtain kafka delegation token but bootstrap " + |
| 67 | + "servers not configured.") |
| 68 | + adminClientProperties.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers.get) |
| 69 | + |
| 70 | + val protocol = sparkConf.get(Kafka.SECURITY_PROTOCOL) |
| 71 | + adminClientProperties.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, protocol) |
| 72 | + protocol match { |
| 73 | + case SASL_SSL.name => |
| 74 | + setTrustStoreProperties(sparkConf, adminClientProperties) |
| 75 | + |
| 76 | + case SSL.name => |
| 77 | + setTrustStoreProperties(sparkConf, adminClientProperties) |
| 78 | + setKeyStoreProperties(sparkConf, adminClientProperties) |
| 79 | + logWarning("Obtaining kafka delegation token with SSL protocol. Please " + |
| 80 | + "configure 2-way authentication on the broker side.") |
| 81 | + |
| 82 | + case SASL_PLAINTEXT.name => |
| 83 | + logWarning("Obtaining kafka delegation token through plain communication channel. Please " + |
| 84 | + "consider the security impact.") |
| 85 | + } |
| 86 | + |
| 87 | + // There are multiple possibilities to log in and applied in the following order: |
| 88 | + // - JVM global security provided -> try to log in with JVM global security configuration |
| 89 | + // which can be configured for example with 'java.security.auth.login.config'. |
| 90 | + // For this no additional parameter needed. |
| 91 | + // - Keytab is provided -> try to log in with kerberos module and keytab using kafka's dynamic |
| 92 | + // JAAS configuration. |
| 93 | + // - Keytab not provided -> try to log in with kerberos module and ticket cache using kafka's |
| 94 | + // dynamic JAAS configuration. |
| 95 | + // Kafka client is unable to use subject from JVM which already logged in |
| 96 | + // to kdc (see KAFKA-7677) |
| 97 | + if (isGlobalJaasConfigurationProvided) { |
| 98 | + logDebug("JVM global security configuration detected, using it for login.") |
| 99 | + } else { |
| 100 | + adminClientProperties.put(SaslConfigs.SASL_MECHANISM, SaslConfigs.GSSAPI_MECHANISM) |
| 101 | + if (sparkConf.contains(KEYTAB)) { |
| 102 | + logDebug("Keytab detected, using it for login.") |
| 103 | + val jaasParams = getKeytabJaasParams(sparkConf) |
| 104 | + adminClientProperties.put(SaslConfigs.SASL_JAAS_CONFIG, jaasParams) |
| 105 | + } else { |
| 106 | + logDebug("Using ticket cache for login.") |
| 107 | + val jaasParams = getTicketCacheJaasParams(sparkConf) |
| 108 | + adminClientProperties.put(SaslConfigs.SASL_JAAS_CONFIG, jaasParams) |
| 109 | + } |
| 110 | + } |
| 111 | + |
| 112 | + adminClientProperties |
| 113 | + } |
| 114 | + |
| 115 | + def isGlobalJaasConfigurationProvided: Boolean = { |
| 116 | + try { |
| 117 | + JaasContext.loadClientContext(ju.Collections.emptyMap[String, Object]()) |
| 118 | + true |
| 119 | + } catch { |
| 120 | + case NonFatal(_) => false |
| 121 | + } |
| 122 | + } |
| 123 | + |
| 124 | + private def setTrustStoreProperties(sparkConf: SparkConf, properties: ju.Properties): Unit = { |
| 125 | + sparkConf.get(Kafka.TRUSTSTORE_LOCATION).foreach { truststoreLocation => |
| 126 | + properties.put("ssl.truststore.location", truststoreLocation) |
| 127 | + } |
| 128 | + sparkConf.get(Kafka.TRUSTSTORE_PASSWORD).foreach { truststorePassword => |
| 129 | + properties.put("ssl.truststore.password", truststorePassword) |
| 130 | + } |
| 131 | + } |
| 132 | + |
| 133 | + private def setKeyStoreProperties(sparkConf: SparkConf, properties: ju.Properties): Unit = { |
| 134 | + sparkConf.get(Kafka.KEYSTORE_LOCATION).foreach { keystoreLocation => |
| 135 | + properties.put("ssl.keystore.location", keystoreLocation) |
| 136 | + } |
| 137 | + sparkConf.get(Kafka.KEYSTORE_PASSWORD).foreach { keystorePassword => |
| 138 | + properties.put("ssl.keystore.password", keystorePassword) |
| 139 | + } |
| 140 | + sparkConf.get(Kafka.KEY_PASSWORD).foreach { keyPassword => |
| 141 | + properties.put("ssl.key.password", keyPassword) |
| 142 | + } |
| 143 | + } |
| 144 | + |
| 145 | + private[security] def getKeytabJaasParams(sparkConf: SparkConf): String = { |
| 146 | + val serviceName = sparkConf.get(Kafka.KERBEROS_SERVICE_NAME) |
| 147 | + require(serviceName.nonEmpty, "Kerberos service name must be defined") |
| 148 | + |
| 149 | + val params = |
| 150 | + s""" |
| 151 | + |${getKrb5LoginModuleName} required |
| 152 | + | useKeyTab=true |
| 153 | + | serviceName="${serviceName.get}" |
| 154 | + | keyTab="${sparkConf.get(KEYTAB).get}" |
| 155 | + | principal="${sparkConf.get(PRINCIPAL).get}"; |
| 156 | + """.stripMargin.replace("\n", "") |
| 157 | + logDebug(s"Krb keytab JAAS params: $params") |
| 158 | + params |
| 159 | + } |
| 160 | + |
| 161 | + def getTicketCacheJaasParams(sparkConf: SparkConf): String = { |
| 162 | + val serviceName = sparkConf.get(Kafka.KERBEROS_SERVICE_NAME) |
| 163 | + require(serviceName.nonEmpty, "Kerberos service name must be defined") |
| 164 | + |
| 165 | + val params = |
| 166 | + s""" |
| 167 | + |${getKrb5LoginModuleName} required |
| 168 | + | useTicketCache=true |
| 169 | + | serviceName="${serviceName.get}"; |
| 170 | + """.stripMargin.replace("\n", "") |
| 171 | + logDebug(s"Krb ticket cache JAAS params: $params") |
| 172 | + params |
| 173 | + } |
| 174 | + |
| 175 | + /** |
| 176 | + * Krb5LoginModule package vary in different JVMs. |
| 177 | + * Please see Hadoop UserGroupInformation for further details. |
| 178 | + */ |
| 179 | + private def getKrb5LoginModuleName(): String = { |
| 180 | + if (System.getProperty("java.vendor").contains("IBM")) { |
| 181 | + "com.ibm.security.auth.module.Krb5LoginModule" |
| 182 | + } else { |
| 183 | + "com.sun.security.auth.module.Krb5LoginModule" |
| 184 | + } |
| 185 | + } |
| 186 | + |
| 187 | + private def printToken(token: DelegationToken): Unit = { |
| 188 | + if (log.isDebugEnabled) { |
| 189 | + val dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm") |
| 190 | + logDebug("%-15s %-30s %-15s %-25s %-15s %-15s %-15s".format( |
| 191 | + "TOKENID", "HMAC", "OWNER", "RENEWERS", "ISSUEDATE", "EXPIRYDATE", "MAXDATE")) |
| 192 | + val tokenInfo = token.tokenInfo |
| 193 | + logDebug("%-15s [hidden] %-15s %-25s %-15s %-15s %-15s".format( |
| 194 | + tokenInfo.tokenId, |
| 195 | + tokenInfo.owner, |
| 196 | + tokenInfo.renewersAsString, |
| 197 | + dateFormat.format(tokenInfo.issueTimestamp), |
| 198 | + dateFormat.format(tokenInfo.expiryTimestamp), |
| 199 | + dateFormat.format(tokenInfo.maxTimestamp))) |
| 200 | + } |
| 201 | + } |
| 202 | +} |
0 commit comments