Skip to content
Closed
Show file tree
Hide file tree
Changes from all 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
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,10 @@ class SparkHadoopUtil {

def getSecretKeyFromUserCredentials(key: String): Array[Byte] = { null }

def loginUserFromKeytab(principalName: String, keytabFilename: String) {
UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename)
}

}

object SparkHadoopUtil {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import scala.collection.mutable
import org.apache.hadoop.fs.{FileStatus, Path}

import org.apache.spark.{Logging, SecurityManager, SparkConf}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.scheduler._
import org.apache.spark.ui.{WebUI, SparkUI}
import org.apache.spark.ui.JettyUtils._
Expand Down Expand Up @@ -257,6 +258,7 @@ object HistoryServer {
val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR

def main(argStrings: Array[String]) {
initSecurity()
val args = new HistoryServerArguments(argStrings)
val securityManager = new SecurityManager(conf)
val server = new HistoryServer(args.logDir, securityManager, conf)
Expand All @@ -266,6 +268,20 @@ object HistoryServer {
while(true) { Thread.sleep(Int.MaxValue) }
server.stop()
}

def initSecurity() {
// If we are accessing HDFS and it has security enabled (Kerberos), we have to login
// from a keytab file so that we can access HDFS beyond the kerberos ticket expiration.
// As long as it is using Hadoop rpc (hdfs://), a relogin will automatically
// occur from the keytab.
if (conf.getBoolean("spark.history.use.kerberos", false)) {
Copy link
Contributor

Choose a reason for hiding this comment

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

could this be spark.history.kerberos.enabled to be more consistent with the other on/off options.

// if you have enabled kerberos the following 2 params must be set
val principalName = conf.get("spark.history.kerberos.principal")
val keytabFilename = conf.get("spark.history.kerberos.keytab")
SparkHadoopUtil.get.loginUserFromKeytab(principalName, keytabFilename)
}
}

}


Expand Down
24 changes: 24 additions & 0 deletions docs/monitoring.md
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,30 @@ represents an application's event logs. This creates a web interface at
The port to which the web interface of the history server binds.
</td>
</tr>
<tr>
<td>spark.history.use.kerberos</td>
<td>false</td>
<td>
Indicates whether the history server should use kerberos to login. This is useful
if the history server is accessing HDFS files on a secure Hadoop cluster. If this is
true it looks uses the configs <code>spark.history.kerberos.principal</code> and
<code>spark.history.kerberos.keytab</code>.
</td>
</tr>
<tr>
<td>spark.history.kerberos.principal</td>
<td>(none)</td>
<td>
Kerberos principal name for the History Server.
</td>
</tr>
<tr>
<td>spark.history.kerberos.keytab</td>
<td>(none)</td>
<td>
Location of the kerberos keytab file for the History Server.
</td>
</tr>
</table>

Note that in all of these UIs, the tables are sortable by clicking their headers,
Expand Down