Skip to content
Closed
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 4 additions & 0 deletions core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
Original file line number Diff line number Diff line change
Expand Up @@ -545,6 +545,10 @@ object SparkSubmit {
if (args.isPython) {
sysProps.put("spark.yarn.isPython", "true")
}
}

// assure a keytab is available from any place in a JVM
if (clusterManager == YARN || clusterManager == LOCAL) {
if (args.principal != null) {
require(args.keytab != null, "Keytab must be specified when principal is specified")
if (!new File(args.keytab).exists()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,12 @@ package org.apache.spark.sql.hive.client
import java.io.{File, PrintStream}
import java.util.{Map => JMap}

import org.apache.hadoop.conf.Configuration

import scala.collection.JavaConverters._
import scala.language.reflectiveCalls

import org.apache.hadoop.fs.Path
import org.apache.hadoop.fs.{CommonConfigurationKeysPublic, Path}
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.metastore.api.{Database, FieldSchema}
import org.apache.hadoop.hive.metastore.{TableType => HTableType}
Expand Down Expand Up @@ -167,6 +169,8 @@ private[hive] class ClientWrapper(
} else {
logInfo("Attempting to login to Kerberos" +
s" using principal: ${principalName} and keytab: ${keytabFileName}")
// make sure the core-site.xml is provided in your classpath
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Better than this would be to update docs/sql-programming-guide.md. It currently only mentions hive-site.xml, when really you need core-site.xml and hdfs-site.xml too (otherwise kerberos, in your case, or things like HDFS HA won't work).

The other changes in this file can be reverted, too.

// (with hadoop.security.authentication=kerberos)
UserGroupInformation.loginUserFromKeytab(principalName, keytabFileName)
}
}
Expand Down