[GitHub] spark pull request #22598: [SPARK-25501][SS] Add kafka delegation token supp...

2018-11-21 Thread soenkeliebau
Github user soenkeliebau commented on a diff in the pull request:

https://github.com/apache/spark/pull/22598#discussion_r235395152
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala
 ---
@@ -0,0 +1,65 @@
+/*
+ * 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.reflect.runtime.universe
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.Credentials
+import org.apache.hadoop.security.token.{Token, TokenIdentifier}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.{KAFKA_BOOTSTRAP_SERVERS, 
KAFKA_SECURITY_PROTOCOL}
+import org.apache.spark.util.Utils
+
+private[security] class KafkaDelegationTokenProvider
+  extends HadoopDelegationTokenProvider with Logging {
+
+  override def serviceName: String = "kafka"
+
+  override def obtainDelegationTokens(
+  hadoopConf: Configuration,
+  sparkConf: SparkConf,
+  creds: Credentials): Option[Long] = {
+try {
+  val mirror = 
universe.runtimeMirror(Utils.getContextOrSparkClassLoader)
+  val obtainToken = mirror.classLoader.
+loadClass("org.apache.spark.sql.kafka010.TokenUtil").
+getMethod("obtainToken", classOf[SparkConf])
+
+  logDebug("Attempting to fetch Kafka security token.")
+  val token = obtainToken.invoke(null, sparkConf)
+.asInstanceOf[Token[_ <: TokenIdentifier]]
+  creds.addToken(token.getService, token)
+} catch {
+  case NonFatal(e) =>
+logInfo(s"Failed to get token from service $serviceName", e)
+}
+
+None
+  }
+
+  override def delegationTokensRequired(
+  sparkConf: SparkConf,
+  hadoopConf: Configuration): Boolean = {
+sparkConf.get(KAFKA_BOOTSTRAP_SERVERS).isDefined &&
+  sparkConf.get(KAFKA_SECURITY_PROTOCOL).startsWith("SASL")
--- End diff --

Looking at the possible protocol 
[values](https://github.com/apache/kafka/blob/068ab9cefae301f3187ea885d645c425955e77d2/clients/src/main/java/org/apache/kafka/common/security/auth/SecurityProtocol.java#L26)
 this condition misses the value "SSL" unless I am mistaken. If the user 
presents a client certificate during the SSL handshake for a SSL connection he 
is allowed to obtain delegation tokens as well. If a client cert was not 
provided then Kafka will reject the token request, but that can should be 
covered in error handling I guess.
Would it maybe be an option to move this check into the KafkaTokenUtil 
class and compare with the actual values of the SecurityProtocol enum instead 
of relying on hard coded Strings here?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #22598: [SPARK-25501][SS] Add kafka delegation token supp...

2018-10-30 Thread soenkeliebau
Github user soenkeliebau commented on a diff in the pull request:

https://github.com/apache/spark/pull/22598#discussion_r229237022
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala
 ---
@@ -0,0 +1,65 @@
+/*
+ * 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.reflect.runtime.universe
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.Credentials
+import org.apache.hadoop.security.token.{Token, TokenIdentifier}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.{KAFKA_BOOTSTRAP_SERVERS, 
KAFKA_SECURITY_PROTOCOL}
+import org.apache.spark.util.Utils
+
+private[security] class KafkaDelegationTokenProvider
+  extends HadoopDelegationTokenProvider with Logging {
+
+  override def serviceName: String = "kafka"
+
+  override def obtainDelegationTokens(
+  hadoopConf: Configuration,
+  sparkConf: SparkConf,
+  creds: Credentials): Option[Long] = {
+try {
+  val mirror = 
universe.runtimeMirror(Utils.getContextOrSparkClassLoader)
+  val obtainToken = mirror.classLoader.
+loadClass("org.apache.spark.sql.kafka010.TokenUtil").
+getMethod("obtainToken", classOf[SparkConf])
--- End diff --

You are right, missed that. Sorry


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #22598: [SPARK-25501][SS] Add kafka delegation token supp...

2018-10-26 Thread soenkeliebau
Github user soenkeliebau commented on a diff in the pull request:

https://github.com/apache/spark/pull/22598#discussion_r228576259
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala
 ---
@@ -0,0 +1,65 @@
+/*
+ * 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.reflect.runtime.universe
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.Credentials
+import org.apache.hadoop.security.token.{Token, TokenIdentifier}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.{KAFKA_BOOTSTRAP_SERVERS, 
KAFKA_SECURITY_PROTOCOL}
+import org.apache.spark.util.Utils
+
+private[security] class KafkaDelegationTokenProvider
+  extends HadoopDelegationTokenProvider with Logging {
+
+  override def serviceName: String = "kafka"
+
+  override def obtainDelegationTokens(
+  hadoopConf: Configuration,
+  sparkConf: SparkConf,
+  creds: Credentials): Option[Long] = {
+try {
+  val mirror = 
universe.runtimeMirror(Utils.getContextOrSparkClassLoader)
+  val obtainToken = mirror.classLoader.
+loadClass("org.apache.spark.sql.kafka010.TokenUtil").
+getMethod("obtainToken", classOf[SparkConf])
--- End diff --

Is there a specific reason to dynamically load this class? This code looks 
quite similar to 
[HBaseDelegationTokenProvider](https://github.com/apache/spark/blob/5264164a67df498b73facae207eda12ee133be7d/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala#L41),
 but I think there it was done to avoid creating a hard dependency on the HBase 
libs of which TokenUtil is a part. In this case the (Kafka)TokenUtil is within 
the Spark project, so we could just use it directly, could we not?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #22598: [SPARK-25501][SS] Add kafka delegation token supp...

2018-10-25 Thread soenkeliebau
Github user soenkeliebau commented on a diff in the pull request:

https://github.com/apache/spark/pull/22598#discussion_r228131806
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala
 ---
@@ -0,0 +1,65 @@
+/*
+ * 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.reflect.runtime.universe
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.Credentials
+import org.apache.hadoop.security.token.{Token, TokenIdentifier}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.{KAFKA_BOOTSTRAP_SERVERS, 
KAFKA_SECURITY_PROTOCOL}
+import org.apache.spark.util.Utils
+
+private[security] class KafkaDelegationTokenProvider
+  extends HadoopDelegationTokenProvider with Logging {
+
+  override def serviceName: String = "kafka"
+
+  override def obtainDelegationTokens(
+  hadoopConf: Configuration,
+  sparkConf: SparkConf,
+  creds: Credentials): Option[Long] = {
+try {
+  val mirror = 
universe.runtimeMirror(Utils.getContextOrSparkClassLoader)
+  val obtainToken = mirror.classLoader.
+loadClass("org.apache.spark.sql.kafka010.TokenUtil").
+getMethod("obtainToken", classOf[SparkConf])
+
+  logDebug("Attempting to fetch Kafka security token.")
+  val token = obtainToken.invoke(null, sparkConf)
+.asInstanceOf[Token[_ <: TokenIdentifier]]
+  creds.addToken(token.getService, token)
+} catch {
+  case NonFatal(e) =>
+logInfo(s"Failed to get token from service $serviceName", e)
+}
+
+None
+  }
+
+  override def delegationTokensRequired(
+  sparkConf: SparkConf,
+  hadoopConf: Configuration): Boolean = {
+sparkConf.get(KAFKA_BOOTSTRAP_SERVERS).isDefined &&
+  sparkConf.get(KAFKA_SECURITY_PROTOCOL).startsWith("SASL")
--- End diff --

Was this a conscious decision to not use delegation tokens when using ssl 
authentication? That might also be useful to avoid spreading keys all over the 
cluster, even if load on the kdc is not an issue in that case.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org