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 773efed  [SPARK-26254][CORE] Extract Hive + Kafka dependencies from 
Core.
773efed is described below

commit 773efede2080639ef7eb546c11bf81401d160db2
Author: Gabor Somogyi <gabor.g.somo...@gmail.com>
AuthorDate: Fri Jan 25 10:36:00 2019 -0800

    [SPARK-26254][CORE] Extract Hive + Kafka dependencies from Core.
    
    ## What changes were proposed in this pull request?
    
    There are ugly provided dependencies inside core for the following:
    * Hive
    * Kafka
    
    In this PR I've extracted them out. This PR contains the following:
    * Token providers are now loaded with service loader
    * Hive token provider moved to hive project
    * Kafka token provider extracted into a new project
    
    ## How was this patch tested?
    
    Existing + newly added unit tests.
    Additionally tested on cluster.
    
    Closes #23499 from gaborgsomogyi/SPARK-26254.
    
    Authored-by: Gabor Somogyi <gabor.g.somo...@gmail.com>
    Signed-off-by: Marcelo Vanzin <van...@cloudera.com>
---
 core/pom.xml                                       |  40 --------
 ...k.deploy.security.HadoopDelegationTokenProvider |   2 +
 .../security/HBaseDelegationTokenProvider.scala    |   2 +
 .../security/HadoopDelegationTokenManager.scala    |  38 ++++----
 .../security/HadoopDelegationTokenProvider.scala   |   3 +
 .../security/HadoopFSDelegationTokenProvider.scala |   8 +-
 ...k.deploy.security.HadoopDelegationTokenProvider |   1 +
 .../HadoopDelegationTokenManagerSuite.scala        | 102 ++++++---------------
 external/kafka-0-10-sql/pom.xml                    |   5 +
 .../spark/sql/kafka010/KafkaConfigUpdater.scala    |   2 +-
 .../spark/sql/kafka010/KafkaSecurityHelper.scala   |   2 +-
 .../sql/kafka010/KafkaDelegationTokenTest.scala    |   4 +-
 .../pom.xml                                        |  72 +--------------
 ...k.deploy.security.HadoopDelegationTokenProvider |   1 +
 .../kafka010}/KafkaDelegationTokenProvider.scala   |   9 +-
 .../apache/spark/kafka010}/KafkaTokenUtil.scala    |  10 +-
 .../src/test/resources/log4j.properties            |  28 ++++++
 .../KafkaHadoopDelegationTokenManagerSuite.scala   |  32 +++++++
 .../spark/kafka010}/KafkaTokenUtilSuite.scala      |   2 +-
 pom.xml                                            |   1 +
 project/MimaExcludes.scala                         |   3 +
 project/SparkBuild.scala                           |   6 +-
 .../sql/hive/thriftserver/SparkSQLCLIDriver.scala  |   4 +-
 ...k.deploy.security.HadoopDelegationTokenProvider |   1 +
 .../security/HiveDelegationTokenProvider.scala     |   5 +-
 .../HiveHadoopDelegationTokenManagerSuite.scala    |  23 +----
 26 files changed, 165 insertions(+), 241 deletions(-)

diff --git a/core/pom.xml b/core/pom.xml
index 1cd1ad9..c87d9d5 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -385,46 +385,6 @@
       <artifactId>commons-crypto</artifactId>
     </dependency>
 
-    <!--
-     The following dependencies are depended upon in HiveCredentialProvider, 
but are only executed if Hive is enabled in
-     the user's Hadoop configuration.  So in order to prevent spark-core from 
depending on Hive, these deps have been
-     placed in the "provided" scope, rather than the "compile" scope, and 
NoClassDefFoundError exceptions are handled
-     when the user has not explicitly compiled with the Hive module.
-    -->
-    <dependency>
-      <groupId>${hive.group}</groupId>
-      <artifactId>hive-exec</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>${hive.group}</groupId>
-      <artifactId>hive-metastore</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.thrift</groupId>
-      <artifactId>libthrift</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.thrift</groupId>
-      <artifactId>libfb303</artifactId>
-      <scope>provided</scope>
-    </dependency>
-
-    <!--
-     The following kafka dependency used to obtain delegation token.
-     In order to prevent spark-core from depending on kafka, these deps have 
been placed in the
-     "provided" scope, rather than the "compile" scope, and 
NoClassDefFoundError exceptions are
-     handled when the user explicitly use neither spark-streaming-kafka nor 
spark-sql-kafka modules.
-    -->
-    <dependency>
-      <groupId>org.apache.kafka</groupId>
-      <artifactId>kafka-clients</artifactId>
-      <version>${kafka.version}</version>
-      <scope>provided</scope>
-    </dependency>
-
   </dependencies>
   <build>
     
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
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.deploy.security.HadoopDelegationTokenProvider
new file mode 100644
index 0000000..c1f2060
--- /dev/null
+++ 
b/core/src/main/resources/META-INF/services/org.apache.spark.deploy.security.HadoopDelegationTokenProvider
@@ -0,0 +1,2 @@
+org.apache.spark.deploy.security.HadoopFSDelegationTokenProvider
+org.apache.spark.deploy.security.HBaseDelegationTokenProvider
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 6ef6835..3bf8c14 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
@@ -21,6 +21,7 @@ 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}
 
@@ -36,6 +37,7 @@ private[security] class HBaseDelegationTokenProvider
   override def obtainDelegationTokens(
       hadoopConf: Configuration,
       sparkConf: SparkConf,
+      fileSystems: Set[FileSystem],
       creds: Credentials): Option[Long] = {
     try {
       val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader)
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 d97857a..2763a46 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
@@ -20,8 +20,11 @@ package org.apache.spark.deploy.security
 import java.io.File
 import java.net.URI
 import java.security.PrivilegedExceptionAction
+import java.util.ServiceLoader
 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}
@@ -33,7 +36,7 @@ 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
+import org.apache.spark.util.{ThreadUtils, Utils}
 
 /**
  * Manager for delegation tokens in a Spark application.
@@ -132,7 +135,7 @@ private[spark] class HadoopDelegationTokenManager(
   def obtainDelegationTokens(creds: Credentials): Long = {
     delegationTokenProviders.values.flatMap { provider =>
       if (provider.delegationTokensRequired(sparkConf, hadoopConf)) {
-        provider.obtainDelegationTokens(hadoopConf, sparkConf, creds)
+        provider.obtainDelegationTokens(hadoopConf, sparkConf, 
fileSystemsToAccess(), creds)
       } else {
         logDebug(s"Service ${provider.serviceName} does not require a token." +
           s" Check your configuration to see if security is disabled or not.")
@@ -244,12 +247,19 @@ private[spark] class HadoopDelegationTokenManager(
   }
 
   private def loadProviders(): Map[String, HadoopDelegationTokenProvider] = {
-    val providers = Seq(
-      new HadoopFSDelegationTokenProvider(
-        () => HadoopDelegationTokenManager.this.fileSystemsToAccess())) ++
-      safeCreateProvider(new HiveDelegationTokenProvider) ++
-      safeCreateProvider(new HBaseDelegationTokenProvider) ++
-      safeCreateProvider(new KafkaDelegationTokenProvider)
+    val loader = ServiceLoader.load(classOf[HadoopDelegationTokenProvider],
+      Utils.getContextOrSparkClassLoader)
+    val providers = mutable.ArrayBuffer[HadoopDelegationTokenProvider]()
+
+    val iterator = loader.iterator
+    while (iterator.hasNext) {
+      try {
+        providers += iterator.next
+      } catch {
+        case t: Throwable =>
+          logDebug(s"Failed to load built in provider.", t)
+      }
+    }
 
     // Filter out providers for which 
spark.security.credentials.{service}.enabled is false.
     providers
@@ -257,16 +267,4 @@ private[spark] class HadoopDelegationTokenManager(
       .map { p => (p.serviceName, p) }
       .toMap
   }
-
-  private def safeCreateProvider(
-      createFn: => HadoopDelegationTokenProvider): 
Option[HadoopDelegationTokenProvider] = {
-    try {
-      Some(createFn)
-    } catch {
-      case t: Throwable =>
-        logDebug(s"Failed to load built in provider.", t)
-        None
-    }
-  }
-
 }
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala
 
b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala
index ed09050..cb4c97b 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.deploy.security
 
 import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.FileSystem
 import org.apache.hadoop.security.Credentials
 
 import org.apache.spark.SparkConf
@@ -43,11 +44,13 @@ private[spark] trait HadoopDelegationTokenProvider {
    * Obtain delegation tokens for this service and get the time of the next 
renewal.
    * @param hadoopConf Configuration of current Hadoop Compatible system.
    * @param creds Credentials to add tokens and security keys to.
+   * @param fileSystems List of file systems for which to obtain delegation 
tokens.
    * @return If the returned tokens are renewable and can be renewed, return 
the time of the next
    *         renewal, otherwise None should be returned.
    */
   def obtainDelegationTokens(
     hadoopConf: Configuration,
     sparkConf: SparkConf,
+    fileSystems: Set[FileSystem],
     creds: Credentials): Option[Long]
 }
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 00200f8..2cd160c 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
@@ -31,7 +31,7 @@ import org.apache.spark.{SparkConf, SparkException}
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
 
-private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: () => 
Set[FileSystem])
+private[deploy] class HadoopFSDelegationTokenProvider
     extends HadoopDelegationTokenProvider with Logging {
 
   // This tokenRenewalInterval will be set in the first call to 
obtainDelegationTokens.
@@ -44,14 +44,14 @@ private[deploy] class 
HadoopFSDelegationTokenProvider(fileSystems: () => Set[Fil
   override def obtainDelegationTokens(
       hadoopConf: Configuration,
       sparkConf: SparkConf,
+      fileSystems: Set[FileSystem],
       creds: Credentials): Option[Long] = {
     try {
-      val fsToGetTokens = fileSystems()
-      val fetchCreds = fetchDelegationTokens(getTokenRenewer(hadoopConf), 
fsToGetTokens, creds)
+      val fetchCreds = fetchDelegationTokens(getTokenRenewer(hadoopConf), 
fileSystems, creds)
 
       // Get the token renewal interval if it is not set. It will only be 
called once.
       if (tokenRenewalInterval == null) {
-        tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf, 
fsToGetTokens)
+        tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf, 
fileSystems)
       }
 
       // Get the time of next renewal.
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.deploy.security.HadoopDelegationTokenProvider
new file mode 100644
index 0000000..f4107be
--- /dev/null
+++ 
b/core/src/test/resources/META-INF/services/org.apache.spark.deploy.security.HadoopDelegationTokenProvider
@@ -0,0 +1 @@
+org.apache.spark.deploy.security.ExceptionThrowingDelegationTokenProvider
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 af7d44b..9cc4a91 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
@@ -17,99 +17,57 @@
 
 package org.apache.spark.deploy.security
 
-import org.apache.commons.io.IOUtils
 import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.FileSystem
+import org.apache.hadoop.security.Credentials
 
 import org.apache.spark.{SparkConf, SparkFunSuite}
-import org.apache.spark.util.Utils
+
+private class ExceptionThrowingDelegationTokenProvider extends 
HadoopDelegationTokenProvider {
+  ExceptionThrowingDelegationTokenProvider.constructed = true
+  throw new IllegalArgumentException
+
+  override def serviceName: String = "throw"
+
+  override def delegationTokensRequired(
+    sparkConf: SparkConf,
+    hadoopConf: Configuration): Boolean = throw new IllegalArgumentException
+
+  override def obtainDelegationTokens(
+    hadoopConf: Configuration,
+    sparkConf: SparkConf,
+    fileSystems: Set[FileSystem],
+    creds: Credentials): Option[Long] = throw new IllegalArgumentException
+}
+
+private object ExceptionThrowingDelegationTokenProvider {
+  var constructed = false
+}
 
 class HadoopDelegationTokenManagerSuite extends SparkFunSuite {
   private val hadoopConf = new Configuration()
 
   test("default configuration") {
+    ExceptionThrowingDelegationTokenProvider.constructed = false
     val manager = new HadoopDelegationTokenManager(new SparkConf(false), 
hadoopConf, null)
     assert(manager.isProviderLoaded("hadoopfs"))
     assert(manager.isProviderLoaded("hbase"))
-    assert(manager.isProviderLoaded("hive"))
-    assert(manager.isProviderLoaded("kafka"))
+    // This checks that providers are loaded independently and they have no 
effect on each other
+    assert(ExceptionThrowingDelegationTokenProvider.constructed)
+    assert(!manager.isProviderLoaded("throw"))
   }
 
-  test("disable hive credential provider") {
-    val sparkConf = new 
SparkConf(false).set("spark.security.credentials.hive.enabled", "false")
+  test("disable hadoopfs credential provider") {
+    val sparkConf = new 
SparkConf(false).set("spark.security.credentials.hadoopfs.enabled", "false")
     val manager = new HadoopDelegationTokenManager(sparkConf, hadoopConf, null)
-    assert(manager.isProviderLoaded("hadoopfs"))
-    assert(manager.isProviderLoaded("hbase"))
-    assert(!manager.isProviderLoaded("hive"))
-    assert(manager.isProviderLoaded("kafka"))
+    assert(!manager.isProviderLoaded("hadoopfs"))
   }
 
   test("using deprecated configurations") {
     val sparkConf = new SparkConf(false)
       .set("spark.yarn.security.tokens.hadoopfs.enabled", "false")
-      .set("spark.yarn.security.credentials.hive.enabled", "false")
     val manager = new HadoopDelegationTokenManager(sparkConf, hadoopConf, null)
     assert(!manager.isProviderLoaded("hadoopfs"))
     assert(manager.isProviderLoaded("hbase"))
-    assert(!manager.isProviderLoaded("hive"))
-    assert(manager.isProviderLoaded("kafka"))
-  }
-
-  test("SPARK-23209: obtain tokens when Hive classes are not available") {
-    // This test needs a custom class loader to hide Hive classes which are in 
the classpath.
-    // Because the manager code loads the Hive provider directly instead of 
using reflection, we
-    // need to drive the test through the custom class loader so a new copy 
that cannot find
-    // Hive classes is loaded.
-    val currentLoader = Thread.currentThread().getContextClassLoader()
-    val noHive = new ClassLoader() {
-      override def loadClass(name: String, resolve: Boolean): Class[_] = {
-        if (name.startsWith("org.apache.hive") || 
name.startsWith("org.apache.hadoop.hive")) {
-          throw new ClassNotFoundException(name)
-        }
-
-        val prefixBlacklist = Seq("java", "scala", "com.sun.", "sun.")
-        if (prefixBlacklist.exists(name.startsWith(_))) {
-          return currentLoader.loadClass(name)
-        }
-
-        val found = findLoadedClass(name)
-        if (found != null) {
-          return found
-        }
-
-        val classFileName = name.replaceAll("\\.", "/") + ".class"
-        val in = currentLoader.getResourceAsStream(classFileName)
-        if (in != null) {
-          val bytes = IOUtils.toByteArray(in)
-          return defineClass(name, bytes, 0, bytes.length)
-        }
-
-        throw new ClassNotFoundException(name)
-      }
-    }
-
-    Utils.withContextClassLoader(noHive) {
-      val test = 
noHive.loadClass(NoHiveTest.getClass.getName().stripSuffix("$"))
-      test.getMethod("runTest").invoke(null)
-    }
   }
 }
-
-/** Test code for SPARK-23209 to avoid using too much reflection above. */
-private object NoHiveTest {
-
-  def runTest(): Unit = {
-    try {
-      val manager = new HadoopDelegationTokenManager(new SparkConf(), new 
Configuration(), null)
-      require(!manager.isProviderLoaded("hive"))
-    } catch {
-      case e: Throwable =>
-        // Throw a better exception in case the test fails, since there may be 
a lot of nesting.
-        var cause = e
-        while (cause.getCause() != null) {
-          cause = cause.getCause()
-        }
-        throw cause
-    }
-  }
-
-}
diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml
index 1c77906..827ceb8 100644
--- a/external/kafka-0-10-sql/pom.xml
+++ b/external/kafka-0-10-sql/pom.xml
@@ -37,6 +37,11 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.spark</groupId>
+      
<artifactId>spark-token-provider-kafka-0-10_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-sql_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
       <scope>provided</scope>
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaConfigUpdater.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaConfigUpdater.scala
index bc1b801..38bf5d7 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaConfigUpdater.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaConfigUpdater.scala
@@ -24,9 +24,9 @@ import scala.collection.JavaConverters._
 import org.apache.kafka.common.config.SaslConfigs
 
 import org.apache.spark.SparkEnv
-import org.apache.spark.deploy.security.KafkaTokenUtil
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config.Kafka
+import org.apache.spark.kafka010.KafkaTokenUtil
 
 /**
  * Class to conveniently update Kafka config params, while logging the changes
diff --git 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelper.scala
 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelper.scala
index 7215295..a11d54f 100644
--- 
a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelper.scala
+++ 
b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelper.scala
@@ -21,9 +21,9 @@ import org.apache.hadoop.security.UserGroupInformation
 import org.apache.kafka.common.security.scram.ScramLoginModule
 
 import org.apache.spark.SparkConf
-import org.apache.spark.deploy.security.KafkaTokenUtil
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
+import org.apache.spark.kafka010.KafkaTokenUtil
 
 private[kafka010] object KafkaSecurityHelper extends Logging {
   def isTokenAvailable(): Boolean = {
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenTest.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenTest.scala
index 31247ab..d0cefc4 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenTest.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenTest.scala
@@ -26,8 +26,8 @@ import org.mockito.Mockito.mock
 import org.scalatest.BeforeAndAfterEach
 
 import org.apache.spark.{SparkConf, SparkEnv, SparkFunSuite}
-import org.apache.spark.deploy.security.KafkaTokenUtil
-import 
org.apache.spark.deploy.security.KafkaTokenUtil.KafkaDelegationTokenIdentifier
+import org.apache.spark.kafka010.KafkaTokenUtil
+import org.apache.spark.kafka010.KafkaTokenUtil.KafkaDelegationTokenIdentifier
 
 /**
  * This is a trait which provides functionalities for Kafka delegation token 
related test suites.
diff --git a/external/kafka-0-10-sql/pom.xml 
b/external/kafka-0-10-token-provider/pom.xml
similarity index 53%
copy from external/kafka-0-10-sql/pom.xml
copy to external/kafka-0-10-token-provider/pom.xml
index 1c77906..b2abcd9 100644
--- a/external/kafka-0-10-sql/pom.xml
+++ b/external/kafka-0-10-token-provider/pom.xml
@@ -26,38 +26,23 @@
   </parent>
 
   <groupId>org.apache.spark</groupId>
-  <artifactId>spark-sql-kafka-0-10_2.12</artifactId>
+  <artifactId>spark-token-provider-kafka-0-10_2.12</artifactId>
   <properties>
-    <sbt.project.name>sql-kafka-0-10</sbt.project.name>
+    <sbt.project.name>token-provider-kafka-0-10</sbt.project.name>
   </properties>
   <packaging>jar</packaging>
-  <name>Kafka 0.10+ Source for Structured Streaming</name>
+  <name>Kafka 0.10+ Token Provider for Streaming</name>
   <url>http://spark.apache.org/</url>
 
   <dependencies>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-sql_${scala.binary.version}</artifactId>
-      <version>${project.version}</version>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
       <artifactId>spark-core_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
-      <type>test-jar</type>
-      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
-      <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
-      <version>${project.version}</version>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-sql_${scala.binary.version}</artifactId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
@@ -68,55 +53,6 @@
       <version>${kafka.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.apache.kafka</groupId>
-      <artifactId>kafka_${scala.binary.version}</artifactId>
-      <version>${kafka.version}</version>
-      <scope>test</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>com.fasterxml.jackson.core</groupId>
-          <artifactId>jackson-core</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.fasterxml.jackson.core</groupId>
-          <artifactId>jackson-databind</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.fasterxml.jackson.core</groupId>
-          <artifactId>jackson-annotations</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <!-- Kafka embedded server uses Zookeeper 3.4.7 API -->
-    <dependency>
-      <groupId>org.apache.zookeeper</groupId>
-      <artifactId>zookeeper</artifactId>
-      <version>3.4.7</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>net.sf.jopt-simple</groupId>
-      <artifactId>jopt-simple</artifactId>
-      <version>3.2</version>
-      <scope>test</scope>
-    </dependency>
-     <dependency>
-        <groupId>org.eclipse.jetty</groupId>
-        <artifactId>jetty-servlet</artifactId>
-        <version>${jetty.version}</version>
-        <scope>test</scope>
-      </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-core</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.scalacheck</groupId>
-      <artifactId>scalacheck_${scala.binary.version}</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-tags_${scala.binary.version}</artifactId>
     </dependency>
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.deploy.security.HadoopDelegationTokenProvider
new file mode 100644
index 0000000..3401401
--- /dev/null
+++ 
b/external/kafka-0-10-token-provider/src/main/resources/META-INF/services/org.apache.spark.deploy.security.HadoopDelegationTokenProvider
@@ -0,0 +1 @@
+org.apache.spark.kafka010.KafkaDelegationTokenProvider
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala
 
b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala
similarity index 88%
rename from 
core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala
rename to 
external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala
index f67cb26..ac6baa0 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala
+++ 
b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala
@@ -15,20 +15,22 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy.security
+package org.apache.spark.kafka010
 
 import scala.language.existentials
 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.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._
+import org.apache.spark.internal.config.Kafka
 
-private[security] class KafkaDelegationTokenProvider
+private[spark] class KafkaDelegationTokenProvider
   extends HadoopDelegationTokenProvider with Logging {
 
   override def serviceName: String = "kafka"
@@ -36,6 +38,7 @@ private[security] class KafkaDelegationTokenProvider
   override def obtainDelegationTokens(
       hadoopConf: Configuration,
       sparkConf: SparkConf,
+      fileSystems: Set[FileSystem],
       creds: Credentials): Option[Long] = {
     try {
       logDebug("Attempting to fetch Kafka security token.")
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala 
b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala
similarity index 96%
rename from 
core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala
rename to 
external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala
index f363853..574d58b 100644
--- a/core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala
+++ 
b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy.security
+package org.apache.spark.kafka010
 
 import java.{util => ju}
 import java.text.SimpleDateFormat
@@ -46,7 +46,7 @@ private[spark] object KafkaTokenUtil extends Logging {
     override def getKind: Text = TOKEN_KIND
   }
 
-  private[security] def obtainToken(sparkConf: SparkConf): (Token[_ <: 
TokenIdentifier], Long) = {
+  private[kafka010] def obtainToken(sparkConf: SparkConf): (Token[_ <: 
TokenIdentifier], Long) = {
     checkProxyUser()
 
     val adminClient = 
AdminClient.create(createAdminClientProperties(sparkConf))
@@ -63,7 +63,7 @@ private[spark] object KafkaTokenUtil extends Logging {
     ), token.tokenInfo.expiryTimestamp)
   }
 
-  private[security] def checkProxyUser(): Unit = {
+  private[kafka010] def checkProxyUser(): Unit = {
     val currentUser = UserGroupInformation.getCurrentUser()
     // Obtaining delegation token for proxy user is planned but not yet 
implemented
     // See https://issues.apache.org/jira/browse/KAFKA-6945
@@ -71,7 +71,7 @@ private[spark] object KafkaTokenUtil extends Logging {
       "user is not yet supported.")
   }
 
-  private[security] def createAdminClientProperties(sparkConf: SparkConf): 
ju.Properties = {
+  private[kafka010] def createAdminClientProperties(sparkConf: SparkConf): 
ju.Properties = {
     val adminClientProperties = new ju.Properties
 
     val bootstrapServers = sparkConf.get(Kafka.BOOTSTRAP_SERVERS)
@@ -154,7 +154,7 @@ private[spark] object KafkaTokenUtil extends Logging {
     }
   }
 
-  private[security] def getKeytabJaasParams(sparkConf: SparkConf): String = {
+  private[kafka010] def getKeytabJaasParams(sparkConf: SparkConf): String = {
     val params =
       s"""
       |${getKrb5LoginModuleName} required
diff --git 
a/external/kafka-0-10-token-provider/src/test/resources/log4j.properties 
b/external/kafka-0-10-token-provider/src/test/resources/log4j.properties
new file mode 100644
index 0000000..75e3b53
--- /dev/null
+++ b/external/kafka-0-10-token-provider/src/test/resources/log4j.properties
@@ -0,0 +1,28 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file target/unit-tests.log
+log4j.rootCategory=INFO, file
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=true
+log4j.appender.file.file=target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p 
%c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.spark-project.jetty=WARN
+
diff --git 
a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaHadoopDelegationTokenManagerSuite.scala
 
b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaHadoopDelegationTokenManagerSuite.scala
new file mode 100644
index 0000000..9aa7618
--- /dev/null
+++ 
b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaHadoopDelegationTokenManagerSuite.scala
@@ -0,0 +1,32 @@
+/*
+ * 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.kafka010
+
+import org.apache.hadoop.conf.Configuration
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.security.HadoopDelegationTokenManager
+
+class KafkaHadoopDelegationTokenManagerSuite extends SparkFunSuite {
+  private val hadoopConf = new Configuration()
+
+  test("default configuration") {
+    val manager = new HadoopDelegationTokenManager(new SparkConf(false), 
hadoopConf, null)
+    assert(manager.isProviderLoaded("kafka"))
+  }
+}
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/security/KafkaTokenUtilSuite.scala
 
b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala
similarity index 99%
rename from 
core/src/test/scala/org/apache/spark/deploy/security/KafkaTokenUtilSuite.scala
rename to 
external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala
index daa7e54..5da6260 100644
--- 
a/core/src/test/scala/org/apache/spark/deploy/security/KafkaTokenUtilSuite.scala
+++ 
b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy.security
+package org.apache.spark.kafka010
 
 import java.{util => ju}
 import java.security.PrivilegedExceptionAction
diff --git a/pom.xml b/pom.xml
index de14d6a..29a281a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -101,6 +101,7 @@
     <module>examples</module>
     <module>repl</module>
     <module>launcher</module>
+    <module>external/kafka-0-10-token-provider</module>
     <module>external/kafka-0-10</module>
     <module>external/kafka-0-10-assembly</module>
     <module>external/kafka-0-10-sql</module>
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index 4cf312d..0cdef00 100644
--- a/project/MimaExcludes.scala
+++ b/project/MimaExcludes.scala
@@ -36,6 +36,9 @@ object MimaExcludes {
 
   // Exclude rules for 3.0.x
   lazy val v30excludes = v24excludes ++ Seq(
+    // [SPARK-26254][CORE] Extract Hive + Kafka dependencies from Core.
+    
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.deploy.security.HiveDelegationTokenProvider"),
+
     // [SPARK-25765][ML] Add training cost to BisectingKMeans summary
     
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.BisectingKMeansModel.this"),
 
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index a8f0ca4..8d836da 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -41,8 +41,8 @@ object BuildCommons {
 
   private val buildLocation = file(".").getAbsoluteFile.getParentFile
 
-  val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer, sqlKafka010, 
avro) = Seq(
-    "catalyst", "sql", "hive", "hive-thriftserver", "sql-kafka-0-10", "avro"
+  val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer, 
tokenProviderKafka010, sqlKafka010, avro) = Seq(
+    "catalyst", "sql", "hive", "hive-thriftserver", 
"token-provider-kafka-0-10", "sql-kafka-0-10", "avro"
   ).map(ProjectRef(buildLocation, _))
 
   val streamingProjects@Seq(streaming, streamingKafka010) =
@@ -331,7 +331,7 @@ object SparkBuild extends PomBuild {
   val mimaProjects = allProjects.filterNot { x =>
     Seq(
       spark, hive, hiveThriftServer, catalyst, repl, networkCommon, 
networkShuffle, networkYarn,
-      unsafe, tags, sqlKafka010, kvstore, avro
+      unsafe, tags, tokenProviderKafka010, sqlKafka010, kvstore, avro
     ).contains(x)
   }
 
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
index bb96cea..463cf16 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
@@ -40,10 +40,10 @@ import org.apache.thrift.transport.TSocket
 
 import org.apache.spark.SparkConf
 import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.deploy.security.HiveDelegationTokenProvider
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.hive.HiveUtils
+import org.apache.spark.sql.hive.security.HiveDelegationTokenProvider
 import org.apache.spark.util.ShutdownHookManager
 
 /**
@@ -126,7 +126,7 @@ private[hive] object SparkSQLCLIDriver extends Logging {
     val tokenProvider = new HiveDelegationTokenProvider()
     if (tokenProvider.delegationTokensRequired(sparkConf, hadoopConf)) {
       val credentials = new Credentials()
-      tokenProvider.obtainDelegationTokens(hadoopConf, sparkConf, credentials)
+      tokenProvider.obtainDelegationTokens(hadoopConf, sparkConf, Set.empty, 
credentials)
       UserGroupInformation.getCurrentUser.addCredentials(credentials)
     }
 
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.deploy.security.HadoopDelegationTokenProvider
new file mode 100644
index 0000000..2b0acc0
--- /dev/null
+++ 
b/sql/hive/src/main/resources/META-INF/services/org.apache.spark.deploy.security.HadoopDelegationTokenProvider
@@ -0,0 +1 @@
+org.apache.spark.sql.hive.security.HiveDelegationTokenProvider
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/security/HiveDelegationTokenProvider.scala
similarity index 96%
rename from 
core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala
rename to 
sql/hive/src/main/scala/org/apache/spark/sql/hive/security/HiveDelegationTokenProvider.scala
index 4ca0136..25eb251 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/security/HiveDelegationTokenProvider.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy.security
+package org.apache.spark.sql.hive.security
 
 import java.lang.reflect.UndeclaredThrowableException
 import java.security.PrivilegedExceptionAction
@@ -23,6 +23,7 @@ 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
@@ -32,6 +33,7 @@ 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.util.Utils
@@ -83,6 +85,7 @@ private[spark] class HiveDelegationTokenProvider
   override def obtainDelegationTokens(
       hadoopConf: Configuration,
       sparkConf: SparkConf,
+      fileSystems: Set[FileSystem],
       creds: Credentials): Option[Long] = {
     try {
       val conf = hiveConf(hadoopConf)
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala
similarity index 80%
copy from 
core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
copy to 
sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala
index af7d44b..ce40cf5 100644
--- 
a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala
@@ -15,43 +15,28 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy.security
+package org.apache.spark.sql.hive.security
 
 import org.apache.commons.io.IOUtils
 import org.apache.hadoop.conf.Configuration
 
 import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.security.HadoopDelegationTokenManager
 import org.apache.spark.util.Utils
 
-class HadoopDelegationTokenManagerSuite extends SparkFunSuite {
+class HiveHadoopDelegationTokenManagerSuite extends SparkFunSuite {
   private val hadoopConf = new Configuration()
 
   test("default configuration") {
     val manager = new HadoopDelegationTokenManager(new SparkConf(false), 
hadoopConf, null)
-    assert(manager.isProviderLoaded("hadoopfs"))
-    assert(manager.isProviderLoaded("hbase"))
     assert(manager.isProviderLoaded("hive"))
-    assert(manager.isProviderLoaded("kafka"))
-  }
-
-  test("disable hive credential provider") {
-    val sparkConf = new 
SparkConf(false).set("spark.security.credentials.hive.enabled", "false")
-    val manager = new HadoopDelegationTokenManager(sparkConf, hadoopConf, null)
-    assert(manager.isProviderLoaded("hadoopfs"))
-    assert(manager.isProviderLoaded("hbase"))
-    assert(!manager.isProviderLoaded("hive"))
-    assert(manager.isProviderLoaded("kafka"))
   }
 
   test("using deprecated configurations") {
     val sparkConf = new SparkConf(false)
-      .set("spark.yarn.security.tokens.hadoopfs.enabled", "false")
       .set("spark.yarn.security.credentials.hive.enabled", "false")
     val manager = new HadoopDelegationTokenManager(sparkConf, hadoopConf, null)
-    assert(!manager.isProviderLoaded("hadoopfs"))
-    assert(manager.isProviderLoaded("hbase"))
     assert(!manager.isProviderLoaded("hive"))
-    assert(manager.isProviderLoaded("kafka"))
   }
 
   test("SPARK-23209: obtain tokens when Hive classes are not available") {
@@ -100,6 +85,8 @@ private object NoHiveTest {
   def runTest(): Unit = {
     try {
       val manager = new HadoopDelegationTokenManager(new SparkConf(), new 
Configuration(), null)
+      assert(manager.isProviderLoaded("hadoopfs"))
+      assert(manager.isProviderLoaded("hbase"))
       require(!manager.isProviderLoaded("hive"))
     } catch {
       case e: Throwable =>


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

Reply via email to