[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223815272
  
--- Diff: 
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosHadoopDelegationTokenManager.scala
 ---
@@ -34,15 +34,9 @@ private[spark] class MesosHadoopDelegationTokenManager(
 
   private val tokenManager = new HadoopDelegationTokenManager(sparkConf, 
hadoopConf)
 
-  def start(driverEndpoint: RpcEndpointRef): Unit = {
-require(driverEndpoint != null, "DriverEndpoint is not initialized")
-setDriverRef(driverEndpoint)
-if (renewalEnabled) {
-  super.start()
-} else {
-  logInfo("Using ticket cache for Kerberos authentication, no token 
renewal.")
-  createAndUpdateTokens()
-}
+  override def start(driver: Option[RpcEndpointRef]): UserGroupInformation 
= {
+require(driver.orNull != null, "Driver endpoint is not initialized")
--- End diff --

`Some(null).isDefined` is true. And if you get initialization order wrong 
in these classes, you can get into that situation. So this is safer.


---

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



[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread attilapiros
Github user attilapiros commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223796403
  
--- Diff: 
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosHadoopDelegationTokenManager.scala
 ---
@@ -34,15 +34,9 @@ private[spark] class MesosHadoopDelegationTokenManager(
 
   private val tokenManager = new HadoopDelegationTokenManager(sparkConf, 
hadoopConf)
 
-  def start(driverEndpoint: RpcEndpointRef): Unit = {
-require(driverEndpoint != null, "DriverEndpoint is not initialized")
-setDriverRef(driverEndpoint)
-if (renewalEnabled) {
-  super.start()
-} else {
-  logInfo("Using ticket cache for Kerberos authentication, no token 
renewal.")
-  createAndUpdateTokens()
-}
+  override def start(driver: Option[RpcEndpointRef]): UserGroupInformation 
= {
+require(driver.orNull != null, "Driver endpoint is not initialized")
--- End diff --

What about `driver.isDefined` instead of `driver.orNull != null`?


---

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



[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223767588
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/AbstractCredentialRenewer.scala
 ---
@@ -0,0 +1,224 @@
+/*
+ * 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.io.File
+import java.security.PrivilegedExceptionAction
+import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.AtomicReference
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc.RpcEndpointRef
+import 
org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens
+import org.apache.spark.ui.UIUtils
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Base class for periodically updating delegation tokens needed by the 
application.
+ *
+ * When configured with a principal and a keytab, this manager will make 
sure long-running apps
+ * (such as Spark Streaming apps) can run without interruption while 
accessing secured services. It
+ * periodically logs in to the KDC with user-provided credentials, and 
contacts all the configured
+ * secure services to obtain delegation tokens to be distributed to the 
rest of the application.
+ *
+ * This class will manage the kerberos login, by renewing the TGT when 
needed. Because the UGI API
+ * does not expose the TTL of the TGT, a configuration controls how often 
to check that a relogin is
+ * necessary. This is done reasonably often since the check is a no-op 
when the relogin is not yet
+ * needed. The check period can be overridden in the configuration.
+ *
+ * New delegation tokens are created once 75% of the renewal interval of 
the original tokens has
+ * elapsed. The new tokens are sent to the Spark driver endpoint once it's 
registered with the AM.
+ * The driver is tasked with distributing the tokens to other processes 
that might need them.
+ *
+ * This class can also be used when without a principal and keytab, in 
which case token renewal will
+ * not be available. It provides a different API in that case (see 
`createAndUpdateTokens()`), which
+ * automates the distribution of tokens to the different processes in the 
Spark app.
+ */
+private[spark] abstract class AbstractCredentialRenewer(
+protected val sparkConf: SparkConf,
+protected val hadoopConf: Configuration) extends Logging {
+
+  private val principal = sparkConf.get(PRINCIPAL).orNull
+  private val keytab = sparkConf.get(KEYTAB).orNull
+
+  if (principal != null) {
+require(keytab != null, "Kerberos principal specified without a 
keytab.")
+require(new File(keytab).isFile(), s"Cannot find keytab at $keytab.")
+  }
+
+  private val renewalExecutor: ScheduledExecutorService =
+if (principal != null) {
+  ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential 
Renewal Thread")
+} else {
+  null
+}
+
+  private val driverRef = new AtomicReference[RpcEndpointRef]()
+
+  protected def setDriverRef(ref: RpcEndpointRef): Unit = {
+driverRef.set(ref)
+  }
+
+  protected def renewalEnabled: Boolean = principal != null
+
+  /**
+   * Start the token renewer. Upon start, if a principal has been 
configured, the renewer will:
+   *
+   * - log in the configured principal, and set up a task to keep that 
user's ticket renewed
+   * - obtain delegation tokens from all available providers
+   * - schedule a periodic task to update the tokens when needed.
+   *
+   * @return The newly logged in user.
+   */
+  def start(): UserGroupInformation = {
   

[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223767408
  
--- Diff: 
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala
 ---
@@ -60,8 +60,13 @@ private[spark] class MesosCoarseGrainedSchedulerBackend(
   extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv)
 with org.apache.mesos.Scheduler with MesosSchedulerUtils {
 
-  private lazy val hadoopDelegationTokenManager: 
MesosHadoopDelegationTokenManager =
-new MesosHadoopDelegationTokenManager(conf, sc.hadoopConfiguration, 
driverEndpoint)
+  private val tokenManager: MesosHadoopDelegationTokenManager = {
--- End diff --

Just adds unnecessary indirection.


---

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



[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223767268
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/AbstractCredentialRenewer.scala
 ---
@@ -0,0 +1,224 @@
+/*
+ * 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.io.File
+import java.security.PrivilegedExceptionAction
+import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.AtomicReference
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc.RpcEndpointRef
+import 
org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens
+import org.apache.spark.ui.UIUtils
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Base class for periodically updating delegation tokens needed by the 
application.
+ *
+ * When configured with a principal and a keytab, this manager will make 
sure long-running apps
+ * (such as Spark Streaming apps) can run without interruption while 
accessing secured services. It
+ * periodically logs in to the KDC with user-provided credentials, and 
contacts all the configured
+ * secure services to obtain delegation tokens to be distributed to the 
rest of the application.
+ *
+ * This class will manage the kerberos login, by renewing the TGT when 
needed. Because the UGI API
+ * does not expose the TTL of the TGT, a configuration controls how often 
to check that a relogin is
+ * necessary. This is done reasonably often since the check is a no-op 
when the relogin is not yet
+ * needed. The check period can be overridden in the configuration.
+ *
+ * New delegation tokens are created once 75% of the renewal interval of 
the original tokens has
+ * elapsed. The new tokens are sent to the Spark driver endpoint once it's 
registered with the AM.
+ * The driver is tasked with distributing the tokens to other processes 
that might need them.
+ *
+ * This class can also be used when without a principal and keytab, in 
which case token renewal will
--- End diff --

The goal is to make it simple for scheduler backends to use token 
functionality. Adding more classes that the backends need to deal with 
complicates things instead of making them simpler.


---

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



[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223767033
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/AbstractCredentialRenewer.scala
 ---
@@ -0,0 +1,224 @@
+/*
+ * 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.io.File
+import java.security.PrivilegedExceptionAction
+import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.AtomicReference
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc.RpcEndpointRef
+import 
org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens
+import org.apache.spark.ui.UIUtils
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Base class for periodically updating delegation tokens needed by the 
application.
+ *
+ * When configured with a principal and a keytab, this manager will make 
sure long-running apps
+ * (such as Spark Streaming apps) can run without interruption while 
accessing secured services. It
+ * periodically logs in to the KDC with user-provided credentials, and 
contacts all the configured
+ * secure services to obtain delegation tokens to be distributed to the 
rest of the application.
+ *
+ * This class will manage the kerberos login, by renewing the TGT when 
needed. Because the UGI API
+ * does not expose the TTL of the TGT, a configuration controls how often 
to check that a relogin is
+ * necessary. This is done reasonably often since the check is a no-op 
when the relogin is not yet
+ * needed. The check period can be overridden in the configuration.
+ *
+ * New delegation tokens are created once 75% of the renewal interval of 
the original tokens has
+ * elapsed. The new tokens are sent to the Spark driver endpoint once it's 
registered with the AM.
+ * The driver is tasked with distributing the tokens to other processes 
that might need them.
+ *
+ * This class can also be used when without a principal and keytab, in 
which case token renewal will
+ * not be available. It provides a different API in that case (see 
`createAndUpdateTokens()`), which
+ * automates the distribution of tokens to the different processes in the 
Spark app.
+ */
+private[spark] abstract class AbstractCredentialRenewer(
+protected val sparkConf: SparkConf,
+protected val hadoopConf: Configuration) extends Logging {
+
+  private val principal = sparkConf.get(PRINCIPAL).orNull
+  private val keytab = sparkConf.get(KEYTAB).orNull
+
+  if (principal != null) {
+require(keytab != null, "Kerberos principal specified without a 
keytab.")
+require(new File(keytab).isFile(), s"Cannot find keytab at $keytab.")
+  }
+
+  private val renewalExecutor: ScheduledExecutorService =
+if (principal != null) {
+  ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential 
Renewal Thread")
+} else {
+  null
+}
+
+  private val driverRef = new AtomicReference[RpcEndpointRef]()
+
+  protected def setDriverRef(ref: RpcEndpointRef): Unit = {
+driverRef.set(ref)
+  }
+
+  protected def renewalEnabled: Boolean = principal != null
+
+  /**
+   * Start the token renewer. Upon start, if a principal has been 
configured, the renewer will:
+   *
+   * - log in the configured principal, and set up a task to keep that 
user's ticket renewed
+   * - obtain delegation tokens from all available providers
+   * - schedule a periodic task to update the tokens when needed.
+   *
+   * @return The newly logged in user.
+   */
+  def start(): UserGroupInformation = {
   

[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223766731
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/AbstractCredentialRenewer.scala
 ---
@@ -0,0 +1,224 @@
+/*
+ * 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.io.File
+import java.security.PrivilegedExceptionAction
+import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.AtomicReference
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc.RpcEndpointRef
+import 
org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens
+import org.apache.spark.ui.UIUtils
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Base class for periodically updating delegation tokens needed by the 
application.
+ *
+ * When configured with a principal and a keytab, this manager will make 
sure long-running apps
+ * (such as Spark Streaming apps) can run without interruption while 
accessing secured services. It
+ * periodically logs in to the KDC with user-provided credentials, and 
contacts all the configured
+ * secure services to obtain delegation tokens to be distributed to the 
rest of the application.
+ *
+ * This class will manage the kerberos login, by renewing the TGT when 
needed. Because the UGI API
+ * does not expose the TTL of the TGT, a configuration controls how often 
to check that a relogin is
+ * necessary. This is done reasonably often since the check is a no-op 
when the relogin is not yet
+ * needed. The check period can be overridden in the configuration.
+ *
+ * New delegation tokens are created once 75% of the renewal interval of 
the original tokens has
+ * elapsed. The new tokens are sent to the Spark driver endpoint once it's 
registered with the AM.
+ * The driver is tasked with distributing the tokens to other processes 
that might need them.
+ *
+ * This class can also be used when without a principal and keytab, in 
which case token renewal will
+ * not be available. It provides a different API in that case (see 
`createAndUpdateTokens()`), which
+ * automates the distribution of tokens to the different processes in the 
Spark app.
+ */
+private[spark] abstract class AbstractCredentialRenewer(
+protected val sparkConf: SparkConf,
+protected val hadoopConf: Configuration) extends Logging {
+
+  private val principal = sparkConf.get(PRINCIPAL).orNull
+  private val keytab = sparkConf.get(KEYTAB).orNull
+
+  if (principal != null) {
+require(keytab != null, "Kerberos principal specified without a 
keytab.")
+require(new File(keytab).isFile(), s"Cannot find keytab at $keytab.")
+  }
+
+  private val renewalExecutor: ScheduledExecutorService =
--- End diff --

`Option` doesn't add anything useful here.

I want `createAndUpdateTokens` here because it will be useful for k8s 
client mode and for when the YARN code is refactored. Just like it is in Mesos 
today.


---

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



[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223766345
  
--- Diff: 
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosHadoopDelegationTokenManager.scala
 ---
@@ -14,147 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.spark.scheduler.cluster.mesos
 
-import java.security.PrivilegedExceptionAction
-import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
-
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
 
 import org.apache.spark.SparkConf
-import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.security.AbstractCredentialRenewer
 import org.apache.spark.deploy.security.HadoopDelegationTokenManager
-import org.apache.spark.internal.{config, Logging}
 import org.apache.spark.rpc.RpcEndpointRef
-import 
org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens
-import org.apache.spark.ui.UIUtils
-import org.apache.spark.util.ThreadUtils
-
 
 /**
- * The MesosHadoopDelegationTokenManager fetches and updates Hadoop 
delegation tokens on the behalf
- * of the MesosCoarseGrainedSchedulerBackend. It is modeled after the YARN 
AMCredentialRenewer,
- * and similarly will renew the Credentials when 75% of the renewal 
interval has passed.
- * The principal difference is that instead of writing the new credentials 
to HDFS and
- * incrementing the timestamp of the file, the new credentials (called 
Tokens when they are
- * serialized) are broadcast to all running executors. On the executor 
side, when new Tokens are
- * received they overwrite the current credentials.
+ * Mesos-specific implementation of AbstractCredentialRenewer.
  */
 private[spark] class MesosHadoopDelegationTokenManager(
--- End diff --

That's the existing name and I'd rather not change it.


---

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



[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223766218
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/AbstractCredentialRenewer.scala
 ---
@@ -0,0 +1,224 @@
+/*
+ * 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.io.File
+import java.security.PrivilegedExceptionAction
+import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.AtomicReference
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc.RpcEndpointRef
+import 
org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens
+import org.apache.spark.ui.UIUtils
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Base class for periodically updating delegation tokens needed by the 
application.
+ *
+ * When configured with a principal and a keytab, this manager will make 
sure long-running apps
+ * (such as Spark Streaming apps) can run without interruption while 
accessing secured services. It
+ * periodically logs in to the KDC with user-provided credentials, and 
contacts all the configured
+ * secure services to obtain delegation tokens to be distributed to the 
rest of the application.
+ *
+ * This class will manage the kerberos login, by renewing the TGT when 
needed. Because the UGI API
+ * does not expose the TTL of the TGT, a configuration controls how often 
to check that a relogin is
+ * necessary. This is done reasonably often since the check is a no-op 
when the relogin is not yet
+ * needed. The check period can be overridden in the configuration.
+ *
+ * New delegation tokens are created once 75% of the renewal interval of 
the original tokens has
+ * elapsed. The new tokens are sent to the Spark driver endpoint once it's 
registered with the AM.
+ * The driver is tasked with distributing the tokens to other processes 
that might need them.
+ *
+ * This class can also be used when without a principal and keytab, in 
which case token renewal will
+ * not be available. It provides a different API in that case (see 
`createAndUpdateTokens()`), which
+ * automates the distribution of tokens to the different processes in the 
Spark app.
+ */
+private[spark] abstract class AbstractCredentialRenewer(
+protected val sparkConf: SparkConf,
+protected val hadoopConf: Configuration) extends Logging {
+
+  private val principal = sparkConf.get(PRINCIPAL).orNull
+  private val keytab = sparkConf.get(KEYTAB).orNull
+
+  if (principal != null) {
+require(keytab != null, "Kerberos principal specified without a 
keytab.")
+require(new File(keytab).isFile(), s"Cannot find keytab at $keytab.")
+  }
+
+  private val renewalExecutor: ScheduledExecutorService =
+if (principal != null) {
+  ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential 
Renewal Thread")
+} else {
+  null
+}
+
+  private val driverRef = new AtomicReference[RpcEndpointRef]()
+
+  protected def setDriverRef(ref: RpcEndpointRef): Unit = {
+driverRef.set(ref)
+  }
+
+  protected def renewalEnabled: Boolean = principal != null
+
+  /**
+   * Start the token renewer. Upon start, if a principal has been 
configured, the renewer will:
+   *
+   * - log in the configured principal, and set up a task to keep that 
user's ticket renewed
+   * - obtain delegation tokens from all available providers
+   * - schedule a periodic task to update the tokens when needed.
+   *
+   * @return The newly logged in user.
+   */
+  def start(): UserGroupInformation = {
   

[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread attilapiros
Github user attilapiros commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223605049
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/AbstractCredentialRenewer.scala
 ---
@@ -0,0 +1,224 @@
+/*
+ * 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.io.File
+import java.security.PrivilegedExceptionAction
+import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.AtomicReference
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc.RpcEndpointRef
+import 
org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens
+import org.apache.spark.ui.UIUtils
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Base class for periodically updating delegation tokens needed by the 
application.
+ *
+ * When configured with a principal and a keytab, this manager will make 
sure long-running apps
+ * (such as Spark Streaming apps) can run without interruption while 
accessing secured services. It
+ * periodically logs in to the KDC with user-provided credentials, and 
contacts all the configured
+ * secure services to obtain delegation tokens to be distributed to the 
rest of the application.
+ *
+ * This class will manage the kerberos login, by renewing the TGT when 
needed. Because the UGI API
+ * does not expose the TTL of the TGT, a configuration controls how often 
to check that a relogin is
+ * necessary. This is done reasonably often since the check is a no-op 
when the relogin is not yet
+ * needed. The check period can be overridden in the configuration.
+ *
+ * New delegation tokens are created once 75% of the renewal interval of 
the original tokens has
+ * elapsed. The new tokens are sent to the Spark driver endpoint once it's 
registered with the AM.
+ * The driver is tasked with distributing the tokens to other processes 
that might need them.
+ *
+ * This class can also be used when without a principal and keytab, in 
which case token renewal will
+ * not be available. It provides a different API in that case (see 
`createAndUpdateTokens()`), which
+ * automates the distribution of tokens to the different processes in the 
Spark app.
+ */
+private[spark] abstract class AbstractCredentialRenewer(
+protected val sparkConf: SparkConf,
+protected val hadoopConf: Configuration) extends Logging {
+
+  private val principal = sparkConf.get(PRINCIPAL).orNull
+  private val keytab = sparkConf.get(KEYTAB).orNull
+
+  if (principal != null) {
+require(keytab != null, "Kerberos principal specified without a 
keytab.")
+require(new File(keytab).isFile(), s"Cannot find keytab at $keytab.")
+  }
+
+  private val renewalExecutor: ScheduledExecutorService =
+if (principal != null) {
+  ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential 
Renewal Thread")
+} else {
+  null
+}
+
+  private val driverRef = new AtomicReference[RpcEndpointRef]()
+
+  protected def setDriverRef(ref: RpcEndpointRef): Unit = {
+driverRef.set(ref)
+  }
+
+  protected def renewalEnabled: Boolean = principal != null
+
+  /**
+   * Start the token renewer. Upon start, if a principal has been 
configured, the renewer will:
+   *
+   * - log in the configured principal, and set up a task to keep that 
user's ticket renewed
+   * - obtain delegation tokens from all available providers
+   * - schedule a periodic task to update the tokens when needed.
+   *
+   * @return The newly logged in user.
+   */
+  def start(): UserGroupInformation = 

[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread attilapiros
Github user attilapiros commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223590938
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/AbstractCredentialRenewer.scala
 ---
@@ -0,0 +1,224 @@
+/*
+ * 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.io.File
+import java.security.PrivilegedExceptionAction
+import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.AtomicReference
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc.RpcEndpointRef
+import 
org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens
+import org.apache.spark.ui.UIUtils
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Base class for periodically updating delegation tokens needed by the 
application.
+ *
+ * When configured with a principal and a keytab, this manager will make 
sure long-running apps
+ * (such as Spark Streaming apps) can run without interruption while 
accessing secured services. It
+ * periodically logs in to the KDC with user-provided credentials, and 
contacts all the configured
+ * secure services to obtain delegation tokens to be distributed to the 
rest of the application.
+ *
+ * This class will manage the kerberos login, by renewing the TGT when 
needed. Because the UGI API
+ * does not expose the TTL of the TGT, a configuration controls how often 
to check that a relogin is
+ * necessary. This is done reasonably often since the check is a no-op 
when the relogin is not yet
+ * needed. The check period can be overridden in the configuration.
+ *
+ * New delegation tokens are created once 75% of the renewal interval of 
the original tokens has
+ * elapsed. The new tokens are sent to the Spark driver endpoint once it's 
registered with the AM.
+ * The driver is tasked with distributing the tokens to other processes 
that might need them.
+ *
+ * This class can also be used when without a principal and keytab, in 
which case token renewal will
+ * not be available. It provides a different API in that case (see 
`createAndUpdateTokens()`), which
+ * automates the distribution of tokens to the different processes in the 
Spark app.
+ */
+private[spark] abstract class AbstractCredentialRenewer(
+protected val sparkConf: SparkConf,
+protected val hadoopConf: Configuration) extends Logging {
+
+  private val principal = sparkConf.get(PRINCIPAL).orNull
+  private val keytab = sparkConf.get(KEYTAB).orNull
+
+  if (principal != null) {
+require(keytab != null, "Kerberos principal specified without a 
keytab.")
+require(new File(keytab).isFile(), s"Cannot find keytab at $keytab.")
+  }
+
+  private val renewalExecutor: ScheduledExecutorService =
--- End diff --

Why not a typesafe Option here? 
If `createAndUpdateTokens()` is extracted this would be always defined.



---

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



[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread attilapiros
Github user attilapiros commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223616283
  
--- Diff: 
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala
 ---
@@ -60,8 +60,13 @@ private[spark] class MesosCoarseGrainedSchedulerBackend(
   extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv)
 with org.apache.mesos.Scheduler with MesosSchedulerUtils {
 
-  private lazy val hadoopDelegationTokenManager: 
MesosHadoopDelegationTokenManager =
-new MesosHadoopDelegationTokenManager(conf, sc.hadoopConfiguration, 
driverEndpoint)
+  private val tokenManager: MesosHadoopDelegationTokenManager = {
--- End diff --

Option?


---

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



[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread attilapiros
Github user attilapiros commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223622626
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/AbstractCredentialRenewer.scala
 ---
@@ -0,0 +1,224 @@
+/*
+ * 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.io.File
+import java.security.PrivilegedExceptionAction
+import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.AtomicReference
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc.RpcEndpointRef
+import 
org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens
+import org.apache.spark.ui.UIUtils
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Base class for periodically updating delegation tokens needed by the 
application.
+ *
+ * When configured with a principal and a keytab, this manager will make 
sure long-running apps
+ * (such as Spark Streaming apps) can run without interruption while 
accessing secured services. It
+ * periodically logs in to the KDC with user-provided credentials, and 
contacts all the configured
+ * secure services to obtain delegation tokens to be distributed to the 
rest of the application.
+ *
+ * This class will manage the kerberos login, by renewing the TGT when 
needed. Because the UGI API
+ * does not expose the TTL of the TGT, a configuration controls how often 
to check that a relogin is
+ * necessary. This is done reasonably often since the check is a no-op 
when the relogin is not yet
+ * needed. The check period can be overridden in the configuration.
+ *
+ * New delegation tokens are created once 75% of the renewal interval of 
the original tokens has
+ * elapsed. The new tokens are sent to the Spark driver endpoint once it's 
registered with the AM.
+ * The driver is tasked with distributing the tokens to other processes 
that might need them.
+ *
+ * This class can also be used when without a principal and keytab, in 
which case token renewal will
+ * not be available. It provides a different API in that case (see 
`createAndUpdateTokens()`), which
+ * automates the distribution of tokens to the different processes in the 
Spark app.
+ */
+private[spark] abstract class AbstractCredentialRenewer(
+protected val sparkConf: SparkConf,
+protected val hadoopConf: Configuration) extends Logging {
+
+  private val principal = sparkConf.get(PRINCIPAL).orNull
+  private val keytab = sparkConf.get(KEYTAB).orNull
+
+  if (principal != null) {
+require(keytab != null, "Kerberos principal specified without a 
keytab.")
+require(new File(keytab).isFile(), s"Cannot find keytab at $keytab.")
+  }
+
+  private val renewalExecutor: ScheduledExecutorService =
+if (principal != null) {
+  ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential 
Renewal Thread")
+} else {
+  null
+}
+
+  private val driverRef = new AtomicReference[RpcEndpointRef]()
+
+  protected def setDriverRef(ref: RpcEndpointRef): Unit = {
+driverRef.set(ref)
+  }
+
+  protected def renewalEnabled: Boolean = principal != null
+
+  /**
+   * Start the token renewer. Upon start, if a principal has been 
configured, the renewer will:
+   *
+   * - log in the configured principal, and set up a task to keep that 
user's ticket renewed
+   * - obtain delegation tokens from all available providers
+   * - schedule a periodic task to update the tokens when needed.
+   *
+   * @return The newly logged in user.
+   */
+  def start(): UserGroupInformation = 

[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread attilapiros
Github user attilapiros commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223624353
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/AbstractCredentialRenewer.scala
 ---
@@ -0,0 +1,224 @@
+/*
+ * 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.io.File
+import java.security.PrivilegedExceptionAction
+import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.AtomicReference
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc.RpcEndpointRef
+import 
org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens
+import org.apache.spark.ui.UIUtils
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Base class for periodically updating delegation tokens needed by the 
application.
+ *
+ * When configured with a principal and a keytab, this manager will make 
sure long-running apps
+ * (such as Spark Streaming apps) can run without interruption while 
accessing secured services. It
+ * periodically logs in to the KDC with user-provided credentials, and 
contacts all the configured
+ * secure services to obtain delegation tokens to be distributed to the 
rest of the application.
+ *
+ * This class will manage the kerberos login, by renewing the TGT when 
needed. Because the UGI API
+ * does not expose the TTL of the TGT, a configuration controls how often 
to check that a relogin is
+ * necessary. This is done reasonably often since the check is a no-op 
when the relogin is not yet
+ * needed. The check period can be overridden in the configuration.
+ *
+ * New delegation tokens are created once 75% of the renewal interval of 
the original tokens has
+ * elapsed. The new tokens are sent to the Spark driver endpoint once it's 
registered with the AM.
+ * The driver is tasked with distributing the tokens to other processes 
that might need them.
+ *
+ * This class can also be used when without a principal and keytab, in 
which case token renewal will
+ * not be available. It provides a different API in that case (see 
`createAndUpdateTokens()`), which
+ * automates the distribution of tokens to the different processes in the 
Spark app.
+ */
+private[spark] abstract class AbstractCredentialRenewer(
+protected val sparkConf: SparkConf,
+protected val hadoopConf: Configuration) extends Logging {
+
+  private val principal = sparkConf.get(PRINCIPAL).orNull
+  private val keytab = sparkConf.get(KEYTAB).orNull
+
+  if (principal != null) {
+require(keytab != null, "Kerberos principal specified without a 
keytab.")
+require(new File(keytab).isFile(), s"Cannot find keytab at $keytab.")
+  }
+
+  private val renewalExecutor: ScheduledExecutorService =
+if (principal != null) {
+  ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential 
Renewal Thread")
+} else {
+  null
+}
+
+  private val driverRef = new AtomicReference[RpcEndpointRef]()
+
+  protected def setDriverRef(ref: RpcEndpointRef): Unit = {
+driverRef.set(ref)
+  }
+
+  protected def renewalEnabled: Boolean = principal != null
+
+  /**
+   * Start the token renewer. Upon start, if a principal has been 
configured, the renewer will:
+   *
+   * - log in the configured principal, and set up a task to keep that 
user's ticket renewed
+   * - obtain delegation tokens from all available providers
+   * - schedule a periodic task to update the tokens when needed.
+   *
+   * @return The newly logged in user.
+   */
+  def start(): UserGroupInformation = 

[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread attilapiros
Github user attilapiros commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223398529
  
--- Diff: 
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosHadoopDelegationTokenManager.scala
 ---
@@ -14,147 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.spark.scheduler.cluster.mesos
 
-import java.security.PrivilegedExceptionAction
-import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
-
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
 
 import org.apache.spark.SparkConf
-import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.security.AbstractCredentialRenewer
 import org.apache.spark.deploy.security.HadoopDelegationTokenManager
-import org.apache.spark.internal.{config, Logging}
 import org.apache.spark.rpc.RpcEndpointRef
-import 
org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens
-import org.apache.spark.ui.UIUtils
-import org.apache.spark.util.ThreadUtils
-
 
 /**
- * The MesosHadoopDelegationTokenManager fetches and updates Hadoop 
delegation tokens on the behalf
- * of the MesosCoarseGrainedSchedulerBackend. It is modeled after the YARN 
AMCredentialRenewer,
- * and similarly will renew the Credentials when 75% of the renewal 
interval has passed.
- * The principal difference is that instead of writing the new credentials 
to HDFS and
- * incrementing the timestamp of the file, the new credentials (called 
Tokens when they are
- * serialized) are broadcast to all running executors. On the executor 
side, when new Tokens are
- * received they overwrite the current credentials.
+ * Mesos-specific implementation of AbstractCredentialRenewer.
  */
 private[spark] class MesosHadoopDelegationTokenManager(
--- End diff --

What about calling it MesosCredentialRenewer (following its base class 
naming and AMCredentialRenewer)? 
And as it already has a HadoopDelegationTokenManager member with different 
responsibility.


---

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



[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-09 Thread attilapiros
Github user attilapiros commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223607997
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/AbstractCredentialRenewer.scala
 ---
@@ -0,0 +1,224 @@
+/*
+ * 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.io.File
+import java.security.PrivilegedExceptionAction
+import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.AtomicReference
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc.RpcEndpointRef
+import 
org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens
+import org.apache.spark.ui.UIUtils
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Base class for periodically updating delegation tokens needed by the 
application.
+ *
+ * When configured with a principal and a keytab, this manager will make 
sure long-running apps
+ * (such as Spark Streaming apps) can run without interruption while 
accessing secured services. It
+ * periodically logs in to the KDC with user-provided credentials, and 
contacts all the configured
+ * secure services to obtain delegation tokens to be distributed to the 
rest of the application.
+ *
+ * This class will manage the kerberos login, by renewing the TGT when 
needed. Because the UGI API
+ * does not expose the TTL of the TGT, a configuration controls how often 
to check that a relogin is
+ * necessary. This is done reasonably often since the check is a no-op 
when the relogin is not yet
+ * needed. The check period can be overridden in the configuration.
+ *
+ * New delegation tokens are created once 75% of the renewal interval of 
the original tokens has
+ * elapsed. The new tokens are sent to the Spark driver endpoint once it's 
registered with the AM.
+ * The driver is tasked with distributing the tokens to other processes 
that might need them.
+ *
+ * This class can also be used when without a principal and keytab, in 
which case token renewal will
--- End diff --

Why not extracting the `createAndUpdateTokens()` (along with 
`obtainDelegationTokens`) functionality into a separate class which would be 
used by this `Renewer` as well. So `Renewer` would be non-abstract class and 
the `renewalEnabled` and all the conditions using it would go away. Even giving 
a principal would be required so could be checked at the member val. 


---

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



[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-08 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223493773
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/AbstractCredentialRenewer.scala
 ---
@@ -0,0 +1,224 @@
+/*
+ * 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.io.File
+import java.security.PrivilegedExceptionAction
+import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.AtomicReference
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc.RpcEndpointRef
+import 
org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens
+import org.apache.spark.ui.UIUtils
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Base class for periodically updating delegation tokens needed by the 
application.
+ *
+ * When configured with a principal and a keytab, this manager will make 
sure long-running apps
+ * (such as Spark Streaming apps) can run without interruption while 
accessing secured services. It
+ * periodically logs in to the KDC with user-provided credentials, and 
contacts all the configured
+ * secure services to obtain delegation tokens to be distributed to the 
rest of the application.
+ *
+ * This class will manage the kerberos login, by renewing the TGT when 
needed. Because the UGI API
+ * does not expose the TTL of the TGT, a configuration controls how often 
to check that a relogin is
+ * necessary. This is done reasonably often since the check is a no-op 
when the relogin is not yet
+ * needed. The check period can be overridden in the configuration.
+ *
+ * New delegation tokens are created once 75% of the renewal interval of 
the original tokens has
+ * elapsed. The new tokens are sent to the Spark driver endpoint once it's 
registered with the AM.
+ * The driver is tasked with distributing the tokens to other processes 
that might need them.
+ *
+ * This class can also be used when without a principal and keytab, in 
which case token renewal will
+ * not be available. It provides a different API in that case (see 
`createAndUpdateTokens()`), which
+ * automates the distribution of tokens to the different processes in the 
Spark app.
+ */
+private[spark] abstract class AbstractCredentialRenewer(
+protected val sparkConf: SparkConf,
+protected val hadoopConf: Configuration) extends Logging {
+
+  private val principal = sparkConf.get(PRINCIPAL).orNull
+  private val keytab = sparkConf.get(KEYTAB).orNull
+
+  if (principal != null) {
+require(keytab != null, "Kerberos principal specified without a 
keytab.")
+require(new File(keytab).isFile(), s"Cannot find keytab at $keytab.")
+  }
+
+  private val renewalExecutor: ScheduledExecutorService =
+if (principal != null) {
+  ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential 
Renewal Thread")
+} else {
+  null
+}
+
+  private val driverRef = new AtomicReference[RpcEndpointRef]()
+
+  protected def setDriverRef(ref: RpcEndpointRef): Unit = {
+driverRef.set(ref)
+  }
+
+  protected def renewalEnabled: Boolean = principal != null
+
+  /**
+   * Start the token renewer. Upon start, if a principal has been 
configured, the renewer will:
+   *
+   * - log in the configured principal, and set up a task to keep that 
user's ticket renewed
+   * - obtain delegation tokens from all available providers
+   * - schedule a periodic task to update the tokens when needed.
+   *
+   * @return The newly logged in user.
+   */
+  def start(): UserGroupInformation = {
   

[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-08 Thread ifilonenko
Github user ifilonenko commented on a diff in the pull request:

https://github.com/apache/spark/pull/22624#discussion_r223472701
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/security/AbstractCredentialRenewer.scala
 ---
@@ -0,0 +1,224 @@
+/*
+ * 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.io.File
+import java.security.PrivilegedExceptionAction
+import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.AtomicReference
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc.RpcEndpointRef
+import 
org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens
+import org.apache.spark.ui.UIUtils
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Base class for periodically updating delegation tokens needed by the 
application.
+ *
+ * When configured with a principal and a keytab, this manager will make 
sure long-running apps
+ * (such as Spark Streaming apps) can run without interruption while 
accessing secured services. It
+ * periodically logs in to the KDC with user-provided credentials, and 
contacts all the configured
+ * secure services to obtain delegation tokens to be distributed to the 
rest of the application.
+ *
+ * This class will manage the kerberos login, by renewing the TGT when 
needed. Because the UGI API
+ * does not expose the TTL of the TGT, a configuration controls how often 
to check that a relogin is
+ * necessary. This is done reasonably often since the check is a no-op 
when the relogin is not yet
+ * needed. The check period can be overridden in the configuration.
+ *
+ * New delegation tokens are created once 75% of the renewal interval of 
the original tokens has
+ * elapsed. The new tokens are sent to the Spark driver endpoint once it's 
registered with the AM.
+ * The driver is tasked with distributing the tokens to other processes 
that might need them.
+ *
+ * This class can also be used when without a principal and keytab, in 
which case token renewal will
+ * not be available. It provides a different API in that case (see 
`createAndUpdateTokens()`), which
+ * automates the distribution of tokens to the different processes in the 
Spark app.
+ */
+private[spark] abstract class AbstractCredentialRenewer(
+protected val sparkConf: SparkConf,
+protected val hadoopConf: Configuration) extends Logging {
+
+  private val principal = sparkConf.get(PRINCIPAL).orNull
+  private val keytab = sparkConf.get(KEYTAB).orNull
+
+  if (principal != null) {
+require(keytab != null, "Kerberos principal specified without a 
keytab.")
+require(new File(keytab).isFile(), s"Cannot find keytab at $keytab.")
+  }
+
+  private val renewalExecutor: ScheduledExecutorService =
+if (principal != null) {
+  ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential 
Renewal Thread")
+} else {
+  null
+}
+
+  private val driverRef = new AtomicReference[RpcEndpointRef]()
+
+  protected def setDriverRef(ref: RpcEndpointRef): Unit = {
+driverRef.set(ref)
+  }
+
+  protected def renewalEnabled: Boolean = principal != null
+
+  /**
+   * Start the token renewer. Upon start, if a principal has been 
configured, the renewer will:
+   *
+   * - log in the configured principal, and set up a task to keep that 
user's ticket renewed
+   * - obtain delegation tokens from all available providers
+   * - schedule a periodic task to update the tokens when needed.
+   *
+   * @return The newly logged in user.
+   */
+  def start(): UserGroupInformation = 

[GitHub] spark pull request #22624: [SPARK-23781][CORE] Add base class for token rene...

2018-10-03 Thread vanzin
GitHub user vanzin opened a pull request:

https://github.com/apache/spark/pull/22624

[SPARK-23781][CORE] Add base class for token renewal functionality.

The base class is based on YARN's token renewer, with some new
functionality to support the current usage in Mesos. It supports
both renewal (which you get when you provide a principal and keytab
to Spark) and just creating an arbitrary set of tokens, which is
used by Mesos in the non-renewal case.

The internal API is a little sub-optimal because YARN needs to defer
some of the initialization of the renewer; so the code needs to handle
the case where the renewer doesn't yet have a reference to the driver
endpoint, for example.

YARN was tested with custom app to stress the token renewal code in
a real cluster. Tested the driver behavior with a hacked up Mesos
cluster (access to Hive and HDFS worked both with TGT and keytab,
delegation tokens were obtained, but I couldn't get executors to run).

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/vanzin/spark SPARK-23781

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/22624.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #22624


commit 01e39466d622665b63576e90b8fd0da7cae7e43f
Author: Marcelo Vanzin 
Date:   2018-10-02T20:24:21Z

[SPARK-23781][CORE] Add base class for token renewal functionality.

The base class is based on YARN's token renewer, with some new
functionality to support the current usage in Mesos. It supports
both renewal (which you get when you provide a principal and keytab
to Spark) and just creating an arbitrary set of tokens, which is
used by Mesos in the non-renewal case.

The internal API is a little sub-optimal because YARN needs to defer
some of the initialization of the renewer; so the code needs to handle
the case where the renewer doesn't yet have a reference to the driver
endpoint, for example.

YARN was tested with custom app to stress the token renewal code in
a real cluster. Tested the driver behavior with a hacked up Mesos
clusters (access to Hive and HDFS worked both with TGT and keytab,
delegation tokens were obtained, but I couldn't get executors to run).




---

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