Skip to content

Commit bcfc374

Browse files
Fix Hadoop-1 build by adding no-op methods in SparkHadoopUtil, with impl in YarnSparkHadoopUtil.
1 parent f8fe694 commit bcfc374

File tree

3 files changed

+117
-104
lines changed

3 files changed

+117
-104
lines changed

core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala

Lines changed: 3 additions & 97 deletions
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configuration
2929
import org.apache.hadoop.fs.{FileUtil, FileStatus, FileSystem, Path}
3030
import org.apache.hadoop.fs.FileSystem.Statistics
3131
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier
32-
import org.apache.hadoop.mapred.{Master, JobConf}
32+
import org.apache.hadoop.mapred.JobConf
3333
import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext}
3434
import org.apache.hadoop.security.Credentials
3535
import org.apache.hadoop.security.UserGroupInformation
@@ -50,10 +50,6 @@ class SparkHadoopUtil extends Logging {
5050
val conf: Configuration = newConfiguration(sparkConf)
5151
UserGroupInformation.setConfiguration(conf)
5252

53-
private var keytabFile: Option[String] = None
54-
private var loginPrincipal: Option[String] = None
55-
private val loggedInViaKeytab = new AtomicBoolean(false)
56-
5753
/**
5854
* Runs the given function with a Hadoop UserGroupInformation as a thread local variable
5955
* (distributed to child threads), used for authenticating HDFS and YARN calls.
@@ -132,99 +128,9 @@ class SparkHadoopUtil extends Logging {
132128
UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename)
133129
}
134130

135-
def setPrincipalAndKeytabForLogin(principal: String, keytab: String): Unit ={
136-
loginPrincipal = Option(principal)
137-
keytabFile = Option(keytab)
138-
}
139-
140-
private[spark] def scheduleLoginFromKeytab(callback: (SerializableBuffer) => Unit): Unit = {
141-
142-
loginPrincipal match {
143-
case Some(principal) =>
144-
val keytab = keytabFile.get
145-
val remoteFs = FileSystem.get(conf)
146-
val remoteKeytabPath = new Path(
147-
remoteFs.getHomeDirectory, System.getenv("SPARK_STAGING_DIR") + Path.SEPARATOR + keytab)
148-
val localFS = FileSystem.getLocal(conf)
149-
// At this point, SparkEnv is likely no initialized, so create a dir, put the keytab there.
150-
val tempDir = Utils.createTempDir()
151-
val localURI = new URI(tempDir.getAbsolutePath + Path.SEPARATOR + keytab)
152-
val qualifiedURI = new URI(localFS.makeQualified(new Path(localURI)).toString)
153-
FileUtil.copy(
154-
remoteFs, remoteKeytabPath, localFS, new Path(qualifiedURI), false, false, conf)
155-
// Get the current credentials, find out when they expire.
156-
val creds = UserGroupInformation.getCurrentUser.getCredentials
157-
val credStream = new ByteArrayOutputStream()
158-
creds.writeTokenStorageToStream(new DataOutputStream(credStream))
159-
val in = new DataInputStream(new ByteArrayInputStream(credStream.toByteArray))
160-
val tokenIdentifier = new DelegationTokenIdentifier()
161-
tokenIdentifier.readFields(in)
162-
val timeToRenewal = (0.6 * (tokenIdentifier.getMaxDate - System.currentTimeMillis())).toLong
163-
Executors.newSingleThreadScheduledExecutor(new ThreadFactory {
164-
override def newThread(r: Runnable): Thread = {
165-
val t = new Thread(r)
166-
t.setName("Delegation Token Refresh Thread")
167-
t.setDaemon(true)
168-
t
169-
}
170-
}).scheduleWithFixedDelay(new Runnable {
171-
override def run(): Unit = {
172-
if (!loggedInViaKeytab.get()) {
173-
loginUserFromKeytab(principal, tempDir.getAbsolutePath + Path.SEPARATOR + keytab)
174-
loggedInViaKeytab.set(true)
175-
}
176-
val nns = getNameNodesToAccess(sparkConf) + remoteKeytabPath
177-
val newCredentials = new Credentials()
178-
obtainTokensForNamenodes(nns, conf, newCredentials)
179-
// Now write this out via Akka to executors.
180-
val outputStream = new ByteArrayOutputStream()
181-
newCredentials.writeTokenStorageToStream(new DataOutputStream(outputStream))
182-
callback(new SerializableBuffer(ByteBuffer.wrap(outputStream.toByteArray)))
183-
}
184-
}, timeToRenewal, timeToRenewal, TimeUnit.MILLISECONDS)
185-
case None =>
186-
}
187-
}
188-
189-
/**
190-
* Get the list of namenodes the user may access.
191-
*/
192-
def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = {
193-
sparkConf.get("spark.yarn.access.namenodes", "")
194-
.split(",")
195-
.map(_.trim())
196-
.filter(!_.isEmpty)
197-
.map(new Path(_))
198-
.toSet
199-
}
200-
201-
def getTokenRenewer(conf: Configuration): String = {
202-
val delegTokenRenewer = Master.getMasterPrincipal(conf)
203-
logDebug("delegation token renewer is: " + delegTokenRenewer)
204-
if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
205-
val errorMessage = "Can't get Master Kerberos principal for use as renewer"
206-
logError(errorMessage)
207-
throw new SparkException(errorMessage)
208-
}
209-
delegTokenRenewer
210-
}
131+
def setPrincipalAndKeytabForLogin(principal: String, keytab: String): Unit = {}
211132

212-
/**
213-
* Obtains tokens for the namenodes passed in and adds them to the credentials.
214-
*/
215-
def obtainTokensForNamenodes(
216-
paths: Set[Path],
217-
conf: Configuration,
218-
creds: Credentials): Unit = {
219-
if (UserGroupInformation.isSecurityEnabled()) {
220-
val delegTokenRenewer = getTokenRenewer(conf)
221-
paths.foreach { dst =>
222-
val dstFs = dst.getFileSystem(conf)
223-
logDebug("getting token for namenode: " + dst)
224-
dstFs.addDelegationTokens(delegTokenRenewer, creds)
225-
}
226-
}
227-
}
133+
private[spark] def scheduleLoginFromKeytab(callback: (SerializableBuffer) => Unit): Unit = {}
228134

229135
/**
230136
* Returns a function that can be called to find Hadoop FileSystem bytes read. If

yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -907,11 +907,11 @@ object Client extends Logging {
907907
* Get the list of namenodes the user may access.
908908
*/
909909
private[yarn] def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = {
910-
SparkHadoopUtil.get.getNameNodesToAccess(sparkConf)
910+
SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil].getNameNodesToAccess(sparkConf)
911911
}
912912

913913
private[yarn] def getTokenRenewer(conf: Configuration): String = {
914-
SparkHadoopUtil.get.getTokenRenewer(conf)
914+
SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil].getTokenRenewer(conf)
915915
}
916916

917917
/**
@@ -921,7 +921,8 @@ object Client extends Logging {
921921
paths: Set[Path],
922922
conf: Configuration,
923923
creds: Credentials): Unit = {
924-
SparkHadoopUtil.get.obtainTokensForNamenodes(paths, conf, creds)
924+
SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil]
925+
.obtainTokensForNamenodes(paths, conf, creds)
925926
}
926927

927928
/**

yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala

Lines changed: 110 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -17,15 +17,21 @@
1717

1818
package org.apache.spark.deploy.yarn
1919

20-
import java.io.File
20+
import java.io._
21+
import java.net.URI
22+
import java.nio.ByteBuffer
23+
import java.util.concurrent.atomic.AtomicBoolean
24+
import java.util.concurrent.{TimeUnit, ThreadFactory, Executors}
2125
import java.util.regex.Matcher
2226
import java.util.regex.Pattern
2327

2428
import scala.collection.mutable.HashMap
2529
import scala.util.Try
2630

31+
import org.apache.hadoop.fs.{FileUtil, Path, FileSystem}
32+
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier
2733
import org.apache.hadoop.io.Text
28-
import org.apache.hadoop.mapred.JobConf
34+
import org.apache.hadoop.mapred.{Master, JobConf}
2935
import org.apache.hadoop.security.Credentials
3036
import org.apache.hadoop.security.UserGroupInformation
3137
import org.apache.hadoop.yarn.conf.YarnConfiguration
@@ -34,15 +40,19 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
3440
import org.apache.hadoop.yarn.api.records.{Priority, ApplicationAccessType}
3541
import org.apache.hadoop.conf.Configuration
3642

37-
import org.apache.spark.{SecurityManager, SparkConf}
43+
import org.apache.spark.{SparkException, SecurityManager, SparkConf}
3844
import org.apache.spark.deploy.SparkHadoopUtil
39-
import org.apache.spark.util.Utils
45+
import org.apache.spark.util.{SerializableBuffer, Utils}
4046

4147
/**
4248
* Contains util methods to interact with Hadoop from spark.
4349
*/
4450
class YarnSparkHadoopUtil extends SparkHadoopUtil {
4551

52+
private var keytabFile: Option[String] = None
53+
private var loginPrincipal: Option[String] = None
54+
private val loggedInViaKeytab = new AtomicBoolean(false)
55+
4656
override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) {
4757
dest.addCredentials(source.getCredentials())
4858
}
@@ -82,6 +92,101 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil {
8292
if (credentials != null) credentials.getSecretKey(new Text(key)) else null
8393
}
8494

95+
override def setPrincipalAndKeytabForLogin(principal: String, keytab: String): Unit ={
96+
loginPrincipal = Option(principal)
97+
keytabFile = Option(keytab)
98+
}
99+
100+
private[spark] override def scheduleLoginFromKeytab(
101+
callback: (SerializableBuffer) => Unit): Unit = {
102+
103+
loginPrincipal match {
104+
case Some(principal) =>
105+
val keytab = keytabFile.get
106+
val remoteFs = FileSystem.get(conf)
107+
val remoteKeytabPath = new Path(
108+
remoteFs.getHomeDirectory, System.getenv("SPARK_STAGING_DIR") + Path.SEPARATOR + keytab)
109+
val localFS = FileSystem.getLocal(conf)
110+
// At this point, SparkEnv is likely no initialized, so create a dir, put the keytab there.
111+
val tempDir = Utils.createTempDir()
112+
val localURI = new URI(tempDir.getAbsolutePath + Path.SEPARATOR + keytab)
113+
val qualifiedURI = new URI(localFS.makeQualified(new Path(localURI)).toString)
114+
FileUtil.copy(
115+
remoteFs, remoteKeytabPath, localFS, new Path(qualifiedURI), false, false, conf)
116+
// Get the current credentials, find out when they expire.
117+
val creds = UserGroupInformation.getCurrentUser.getCredentials
118+
val credStream = new ByteArrayOutputStream()
119+
creds.writeTokenStorageToStream(new DataOutputStream(credStream))
120+
val in = new DataInputStream(new ByteArrayInputStream(credStream.toByteArray))
121+
val tokenIdentifier = new DelegationTokenIdentifier()
122+
tokenIdentifier.readFields(in)
123+
val timeToRenewal = (0.6 * (tokenIdentifier.getMaxDate - System.currentTimeMillis())).toLong
124+
Executors.newSingleThreadScheduledExecutor(new ThreadFactory {
125+
override def newThread(r: Runnable): Thread = {
126+
val t = new Thread(r)
127+
t.setName("Delegation Token Refresh Thread")
128+
t.setDaemon(true)
129+
t
130+
}
131+
}).scheduleWithFixedDelay(new Runnable {
132+
override def run(): Unit = {
133+
if (!loggedInViaKeytab.get()) {
134+
loginUserFromKeytab(principal, tempDir.getAbsolutePath + Path.SEPARATOR + keytab)
135+
loggedInViaKeytab.set(true)
136+
}
137+
val nns = getNameNodesToAccess(sparkConf) + remoteKeytabPath
138+
val newCredentials = new Credentials()
139+
obtainTokensForNamenodes(nns, conf, newCredentials)
140+
// Now write this out via Akka to executors.
141+
val outputStream = new ByteArrayOutputStream()
142+
newCredentials.writeTokenStorageToStream(new DataOutputStream(outputStream))
143+
callback(new SerializableBuffer(ByteBuffer.wrap(outputStream.toByteArray)))
144+
}
145+
}, timeToRenewal, timeToRenewal, TimeUnit.MILLISECONDS)
146+
case None =>
147+
}
148+
}
149+
150+
/**
151+
* Get the list of namenodes the user may access.
152+
*/
153+
def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = {
154+
sparkConf.get("spark.yarn.access.namenodes", "")
155+
.split(",")
156+
.map(_.trim())
157+
.filter(!_.isEmpty)
158+
.map(new Path(_))
159+
.toSet
160+
}
161+
162+
def getTokenRenewer(conf: Configuration): String = {
163+
val delegTokenRenewer = Master.getMasterPrincipal(conf)
164+
logDebug("delegation token renewer is: " + delegTokenRenewer)
165+
if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
166+
val errorMessage = "Can't get Master Kerberos principal for use as renewer"
167+
logError(errorMessage)
168+
throw new SparkException(errorMessage)
169+
}
170+
delegTokenRenewer
171+
}
172+
173+
/**
174+
* Obtains tokens for the namenodes passed in and adds them to the credentials.
175+
*/
176+
def obtainTokensForNamenodes(
177+
paths: Set[Path],
178+
conf: Configuration,
179+
creds: Credentials): Unit = {
180+
if (UserGroupInformation.isSecurityEnabled()) {
181+
val delegTokenRenewer = getTokenRenewer(conf)
182+
paths.foreach { dst =>
183+
val dstFs = dst.getFileSystem(conf)
184+
logDebug("getting token for namenode: " + dst)
185+
dstFs.addDelegationTokens(delegTokenRenewer, creds)
186+
}
187+
}
188+
}
189+
85190
}
86191

87192
object YarnSparkHadoopUtil {
@@ -211,4 +316,5 @@ object YarnSparkHadoopUtil {
211316
def getClassPathSeparator(): String = {
212317
classPathSeparatorField.get(null).asInstanceOf[String]
213318
}
319+
214320
}

0 commit comments

Comments
 (0)