This is an automated email from the ASF dual-hosted git repository.

vanzin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 28ced38  [SPARK-26772][YARN] Delete ServiceCredentialProvider and make 
HadoopDelegationTokenProvider a developer API
28ced38 is described below

commit 28ced387b9ef0b4c9d3b72913b839786fa0bfa38
Author: Gabor Somogyi <gabor.g.somo...@gmail.com>
AuthorDate: Fri Feb 15 14:43:13 2019 -0800

    [SPARK-26772][YARN] Delete ServiceCredentialProvider and make 
HadoopDelegationTokenProvider a developer API
    
    ## What changes were proposed in this pull request?
    
    `HadoopDelegationTokenProvider` has basically the same functionality just 
like `ServiceCredentialProvider` so the interfaces can be merged.
    
    `YARNHadoopDelegationTokenManager` now loads `ServiceCredentialProvider`s 
in one step. The drawback of this if one provider fails all others are not 
loaded. `HadoopDelegationTokenManager` loads `HadoopDelegationTokenProvider`s 
independently so it provides more robust behaviour.
    
    In this PR I've I've made the following changes:
    * Deleted `YARNHadoopDelegationTokenManager` and `ServiceCredentialProvider`
    * Made `HadoopDelegationTokenProvider` a `DeveloperApi`
    
    ## How was this patch tested?
    
    Existing unit tests.
    
    Closes #23686 from gaborgsomogyi/SPARK-26772.
    
    Authored-by: Gabor Somogyi <gabor.g.somo...@gmail.com>
    Signed-off-by: Marcelo Vanzin <van...@cloudera.com>
---
 ...e.spark.security.HadoopDelegationTokenProvider} |  0
 .../security/HBaseDelegationTokenProvider.scala    |  2 +-
 .../security/HadoopDelegationTokenManager.scala    |  2 +-
 .../security/HadoopFSDelegationTokenProvider.scala |  1 +
 .../security/HadoopDelegationTokenProvider.scala   |  8 ++-
 ...e.spark.security.HadoopDelegationTokenProvider} |  0
 .../HadoopDelegationTokenManagerSuite.scala        |  1 +
 docs/running-on-yarn.md                            |  7 --
 docs/security.md                                   |  5 ++
 ...e.spark.security.HadoopDelegationTokenProvider} |  0
 .../kafka010/KafkaDelegationTokenProvider.scala    |  2 +-
 .../org/apache/spark/deploy/yarn/Client.scala      |  9 ++-
 .../yarn/security/ServiceCredentialProvider.scala  | 58 -----------------
 .../YARNHadoopDelegationTokenManager.scala         | 75 ----------------------
 .../scheduler/cluster/YarnSchedulerBackend.scala   |  4 +-
 ....deploy.yarn.security.ServiceCredentialProvider |  1 -
 .../YARNHadoopDelegationTokenManagerSuite.scala    | 51 ---------------
 ...e.spark.security.HadoopDelegationTokenProvider} |  0
 .../security/HiveDelegationTokenProvider.scala     |  3 +-
 19 files changed, 21 insertions(+), 208 deletions(-)

diff --git 
a/core/src/main/resources/META-INF/services/org.apache.spark.deploy.security.HadoopDelegationTokenProvider
 
b/core/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider
similarity index 100%
rename from 
core/src/main/resources/META-INF/services/org.apache.spark.deploy.security.HadoopDelegationTokenProvider
rename to 
core/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala
 
b/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala
index e56d034..2e21ada 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala
@@ -23,12 +23,12 @@ import scala.reflect.runtime.universe
 import scala.util.control.NonFatal
 
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.FileSystem
 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.security.HadoopDelegationTokenProvider
 import org.apache.spark.util.Utils
 
 private[security] class HBaseDelegationTokenProvider
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
 
b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
index 6a18a8d..4db86ba 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
@@ -26,7 +26,6 @@ import java.util.concurrent.{ScheduledExecutorService, 
TimeUnit}
 import scala.collection.mutable
 
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.FileSystem
 import org.apache.hadoop.security.{Credentials, UserGroupInformation}
 
 import org.apache.spark.SparkConf
@@ -35,6 +34,7 @@ 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.security.HadoopDelegationTokenProvider
 import org.apache.spark.ui.UIUtils
 import org.apache.spark.util.{ThreadUtils, Utils}
 
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala
 
b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala
index 725eefb..ac432e7 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala
@@ -30,6 +30,7 @@ import 
org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti
 import org.apache.spark.{SparkConf, SparkException}
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
+import org.apache.spark.security.HadoopDelegationTokenProvider
 
 private[deploy] class HadoopFSDelegationTokenProvider
     extends HadoopDelegationTokenProvider with Logging {
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala
 
b/core/src/main/scala/org/apache/spark/security/HadoopDelegationTokenProvider.scala
similarity index 92%
rename from 
core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala
rename to 
core/src/main/scala/org/apache/spark/security/HadoopDelegationTokenProvider.scala
index 3dc952d..cff8d81 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala
+++ 
b/core/src/main/scala/org/apache/spark/security/HadoopDelegationTokenProvider.scala
@@ -15,18 +15,20 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy.security
+package org.apache.spark.security
 
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.FileSystem
 import org.apache.hadoop.security.Credentials
 
 import org.apache.spark.SparkConf
+import org.apache.spark.annotation.DeveloperApi
 
 /**
+ * ::DeveloperApi::
  * Hadoop delegation token provider.
  */
-private[spark] trait HadoopDelegationTokenProvider {
+@DeveloperApi
+trait HadoopDelegationTokenProvider {
 
   /**
    * Name of the service to provide delegation tokens. This name should be 
unique.  Spark will
diff --git 
a/core/src/test/resources/META-INF/services/org.apache.spark.deploy.security.HadoopDelegationTokenProvider
 
b/core/src/test/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider
similarity index 100%
rename from 
core/src/test/resources/META-INF/services/org.apache.spark.deploy.security.HadoopDelegationTokenProvider
rename to 
core/src/test/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
 
b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
index 2f36dba..70174f7 100644
--- 
a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.security.Credentials
 
 import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.security.HadoopDelegationTokenProvider
 
 private class ExceptionThrowingDelegationTokenProvider extends 
HadoopDelegationTokenProvider {
   ExceptionThrowingDelegationTokenProvider.constructed = true
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 8f1a127..6ee4b3d 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -538,13 +538,6 @@ for:
   filesystem if `spark.yarn.stagingDir` is not set);
 - if Hadoop federation is enabled, all the federated filesystems in the 
configuration.
 
-The YARN integration also supports custom delegation token providers using the 
Java Services
-mechanism (see `java.util.ServiceLoader`). Implementations of
-`org.apache.spark.deploy.yarn.security.ServiceCredentialProvider` can be made 
available to Spark
-by listing their names in the corresponding file in the jar's 
`META-INF/services` directory. These
-providers can be disabled individually by setting 
`spark.security.credentials.{service}.enabled` to
-`false`, where `{service}` is the name of the credential provider.
-
 ## YARN-specific Kerberos Configuration
 
 <table class="table">
diff --git a/docs/security.md b/docs/security.md
index d2cff41..20492d8 100644
--- a/docs/security.md
+++ b/docs/security.md
@@ -756,6 +756,11 @@ If an application needs to interact with other secure 
Hadoop filesystems, their
 explicitly provided to Spark at launch time. This is done by listing them in 
the
 `spark.kerberos.access.hadoopFileSystems` property, described in the 
configuration section below.
 
+Spark also supports custom delegation token providers using the Java Services
+mechanism (see `java.util.ServiceLoader`). Implementations of
+`org.apache.spark.security.HadoopDelegationTokenProvider` can be made 
available to Spark
+by listing their names in the corresponding file in the jar's 
`META-INF/services` directory.
+
 Delegation token support is currently only supported in YARN and Mesos modes. 
Consult the
 deployment-specific page for more information.
 
diff --git 
a/external/kafka-0-10-token-provider/src/main/resources/META-INF/services/org.apache.spark.deploy.security.HadoopDelegationTokenProvider
 
b/external/kafka-0-10-token-provider/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider
similarity index 100%
rename from 
external/kafka-0-10-token-provider/src/main/resources/META-INF/services/org.apache.spark.deploy.security.HadoopDelegationTokenProvider
rename to 
external/kafka-0-10-token-provider/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider
diff --git 
a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala
 
b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala
index c69e8a3..cba4b40 100644
--- 
a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala
+++ 
b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala
@@ -25,9 +25,9 @@ import org.apache.hadoop.security.Credentials
 import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, 
SASL_SSL, SSL}
 
 import org.apache.spark.SparkConf
-import org.apache.spark.deploy.security.HadoopDelegationTokenProvider
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config.Kafka
+import org.apache.spark.security.HadoopDelegationTokenProvider
 
 private[spark] class KafkaDelegationTokenProvider
   extends HadoopDelegationTokenProvider with Logging {
diff --git 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 7523e3c..6ca81fb 100644
--- 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -21,7 +21,6 @@ import java.io.{FileSystem => _, _}
 import java.net.{InetAddress, UnknownHostException, URI}
 import java.nio.ByteBuffer
 import java.nio.charset.StandardCharsets
-import java.security.PrivilegedExceptionAction
 import java.util.{Locale, Properties, UUID}
 import java.util.zip.{ZipEntry, ZipOutputStream}
 
@@ -34,9 +33,9 @@ import com.google.common.io.Files
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs._
 import org.apache.hadoop.fs.permission.FsPermission
-import org.apache.hadoop.io.{DataOutputBuffer, Text}
+import org.apache.hadoop.io.Text
 import org.apache.hadoop.mapreduce.MRJobConfig
-import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.util.StringUtils
 import org.apache.hadoop.yarn.api._
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
@@ -50,8 +49,8 @@ import org.apache.hadoop.yarn.util.Records
 
 import org.apache.spark.{SecurityManager, SparkConf, SparkException}
 import org.apache.spark.deploy.{SparkApplication, SparkHadoopUtil}
+import org.apache.spark.deploy.security.HadoopDelegationTokenManager
 import org.apache.spark.deploy.yarn.config._
-import org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
 import org.apache.spark.internal.config.Python._
@@ -315,7 +314,7 @@ private[spark] class Client(
     val credentials = currentUser.getCredentials()
 
     if (isClusterMode) {
-      val credentialManager = new YARNHadoopDelegationTokenManager(sparkConf, 
hadoopConf, null)
+      val credentialManager = new HadoopDelegationTokenManager(sparkConf, 
hadoopConf, null)
       credentialManager.obtainDelegationTokens(credentials)
     }
 
diff --git 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala
 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala
deleted file mode 100644
index cc24ac4..0000000
--- 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.yarn.security
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.security.{Credentials, UserGroupInformation}
-
-import org.apache.spark.SparkConf
-
-/**
- * A credential provider for a service. User must implement this if they need 
to access a
- * secure service from Spark.
- */
-trait ServiceCredentialProvider {
-
-  /**
-   * Name of the service to provide credentials. This name should unique, 
Spark internally will
-   * use this name to differentiate credential provider.
-   */
-  def serviceName: String
-
-  /**
-   * Returns true if credentials are required by this service. By default, it 
is based on whether
-   * Hadoop security is enabled.
-   */
-  def credentialsRequired(hadoopConf: Configuration): Boolean = {
-    UserGroupInformation.isSecurityEnabled
-  }
-
-  /**
-   * Obtain credentials for this service and get the time of the next renewal.
-   *
-   * @param hadoopConf Configuration of current Hadoop Compatible system.
-   * @param sparkConf Spark configuration.
-   * @param creds Credentials to add tokens and security keys to.
-   * @return If this Credential is renewable and can be renewed, return the 
time of the next
-   *         renewal, otherwise None should be returned.
-   */
-  def obtainCredentials(
-      hadoopConf: Configuration,
-      sparkConf: SparkConf,
-      creds: Credentials): Option[Long]
-}
diff --git 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala
 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala
deleted file mode 100644
index fc1f752..0000000
--- 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.yarn.security
-
-import java.util.ServiceLoader
-
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.security.Credentials
-
-import org.apache.spark.SparkConf
-import org.apache.spark.deploy.security.HadoopDelegationTokenManager
-import org.apache.spark.rpc.RpcEndpointRef
-import org.apache.spark.util.Utils
-
-/**
- * This class loads delegation token providers registered under the 
YARN-specific
- * [[ServiceCredentialProvider]] interface, as well as the builtin providers 
defined
- * in [[HadoopDelegationTokenManager]].
- */
-private[spark] class YARNHadoopDelegationTokenManager(
-    _sparkConf: SparkConf,
-    _hadoopConf: Configuration,
-    _schedulerRef: RpcEndpointRef)
-  extends HadoopDelegationTokenManager(_sparkConf, _hadoopConf, _schedulerRef) 
{
-
-  private val credentialProviders = {
-    ServiceLoader.load(classOf[ServiceCredentialProvider], 
Utils.getContextOrSparkClassLoader)
-      .asScala
-      .toList
-      .filter { p => isServiceEnabled(p.serviceName) }
-      .map { p => (p.serviceName, p) }
-      .toMap
-  }
-  if (credentialProviders.nonEmpty) {
-    logDebug("Using the following YARN-specific credential providers: " +
-      s"${credentialProviders.keys.mkString(", ")}.")
-  }
-
-  override def obtainDelegationTokens(creds: Credentials): Long = {
-    val superInterval = super.obtainDelegationTokens(creds)
-
-    credentialProviders.values.flatMap { provider =>
-      if (provider.credentialsRequired(hadoopConf)) {
-        provider.obtainCredentials(hadoopConf, sparkConf, creds)
-      } else {
-        logDebug(s"Service ${provider.serviceName} does not require a token." +
-          s" Check your configuration to see if security is disabled or not.")
-        None
-      }
-    }.foldLeft(superInterval)(math.min)
-  }
-
-  // For testing.
-  override def isProviderLoaded(serviceName: String): Boolean = {
-    credentialProviders.contains(serviceName) || 
super.isProviderLoaded(serviceName)
-  }
-
-}
diff --git 
a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
 
b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
index 821fbcd..78cd6a2 100644
--- 
a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
+++ 
b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
@@ -31,14 +31,12 @@ import org.eclipse.jetty.servlet.{FilterHolder, 
FilterMapping}
 
 import org.apache.spark.SparkContext
 import org.apache.spark.deploy.security.HadoopDelegationTokenManager
-import org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config
 import org.apache.spark.internal.config.UI._
 import org.apache.spark.rpc._
 import org.apache.spark.scheduler._
 import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
-import org.apache.spark.ui.JettyUtils
 import org.apache.spark.util.{RpcUtils, ThreadUtils}
 
 /**
@@ -223,7 +221,7 @@ private[spark] abstract class YarnSchedulerBackend(
   }
 
   override protected def createTokenManager(): 
Option[HadoopDelegationTokenManager] = {
-    Some(new YARNHadoopDelegationTokenManager(sc.conf, sc.hadoopConfiguration, 
driverEndpoint))
+    Some(new HadoopDelegationTokenManager(sc.conf, sc.hadoopConfiguration, 
driverEndpoint))
   }
 
   /**
diff --git 
a/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
 
b/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
deleted file mode 100644
index f31c232..0000000
--- 
a/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
+++ /dev/null
@@ -1 +0,0 @@
-org.apache.spark.deploy.yarn.security.YARNTestCredentialProvider
diff --git 
a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala
 
b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala
deleted file mode 100644
index f00453c..0000000
--- 
a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.yarn.security
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.security.Credentials
-
-import org.apache.spark.{SparkConf, SparkFunSuite}
-
-class YARNHadoopDelegationTokenManagerSuite extends SparkFunSuite {
-  private var credentialManager: YARNHadoopDelegationTokenManager = null
-  private var sparkConf: SparkConf = null
-  private var hadoopConf: Configuration = null
-
-  override def beforeAll(): Unit = {
-    super.beforeAll()
-    sparkConf = new SparkConf()
-    hadoopConf = new Configuration()
-  }
-
-  test("Correctly loads credential providers") {
-    credentialManager = new YARNHadoopDelegationTokenManager(sparkConf, 
hadoopConf, null)
-    assert(credentialManager.isProviderLoaded("yarn-test"))
-  }
-}
-
-class YARNTestCredentialProvider extends ServiceCredentialProvider {
-  override def serviceName: String = "yarn-test"
-
-  override def credentialsRequired(conf: Configuration): Boolean = true
-
-  override def obtainCredentials(
-      hadoopConf: Configuration,
-      sparkConf: SparkConf,
-      creds: Credentials): Option[Long] = None
-}
diff --git 
a/sql/hive/src/main/resources/META-INF/services/org.apache.spark.deploy.security.HadoopDelegationTokenProvider
 
b/sql/hive/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider
similarity index 100%
rename from 
sql/hive/src/main/resources/META-INF/services/org.apache.spark.deploy.security.HadoopDelegationTokenProvider
rename to 
sql/hive/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/security/HiveDelegationTokenProvider.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/security/HiveDelegationTokenProvider.scala
index c0c4618..faee405 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/security/HiveDelegationTokenProvider.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/security/HiveDelegationTokenProvider.scala
@@ -23,7 +23,6 @@ import java.security.PrivilegedExceptionAction
 import scala.util.control.NonFatal
 
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.FileSystem
 import 
org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier
 import org.apache.hadoop.hive.conf.HiveConf
 import org.apache.hadoop.hive.ql.metadata.Hive
@@ -33,9 +32,9 @@ import org.apache.hadoop.security.token.Token
 
 import org.apache.spark.SparkConf
 import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.deploy.security.HadoopDelegationTokenProvider
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config.KEYTAB
+import org.apache.spark.security.HadoopDelegationTokenProvider
 import org.apache.spark.util.Utils
 
 private[spark] class HiveDelegationTokenProvider


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

Reply via email to