diff options
author | Jey Kottalam <jey@cs.berkeley.edu> | 2013-07-24 13:07:27 -0700 |
---|---|---|
committer | Jey Kottalam <jey@cs.berkeley.edu> | 2013-08-15 16:50:37 -0700 |
commit | bd0bab47c9602462628b1d3c90d5eb5d889f4596 (patch) | |
tree | f6b77c9b351cc46e7a061ae46cb4ee8b9e7f3c22 /core | |
parent | 4f43fd791ab0e84693e2337358c6b880a1593e54 (diff) | |
download | spark-bd0bab47c9602462628b1d3c90d5eb5d889f4596.tar.gz spark-bd0bab47c9602462628b1d3c90d5eb5d889f4596.tar.bz2 spark-bd0bab47c9602462628b1d3c90d5eb5d889f4596.zip |
SparkEnv isn't available this early, and not needed anyway
Diffstat (limited to 'core')
-rw-r--r-- | core/src/main/scala/spark/deploy/SparkHadoopUtil.scala | 11 | ||||
-rw-r--r-- | core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala | 14 |
2 files changed, 0 insertions, 25 deletions
diff --git a/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala index c4ed0bb17e..882161e669 100644 --- a/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala @@ -25,17 +25,6 @@ import org.apache.hadoop.mapred.JobConf */ class SparkHadoopUtil { - def getUserNameFromEnvironment(): String = { - // defaulting to -D ... - System.getProperty("user.name") - } - - def runAsUser(func: (Product) => Unit, args: Product) { - - // Add support, if exists - for now, simply run func ! - func(args) - } - // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems def newConfiguration(): Configuration = new Configuration() diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index a9e06f8d54..b5fb6dbe29 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -81,20 +81,6 @@ private[spark] class StandaloneExecutorBackend( private[spark] object StandaloneExecutorBackend { def run(driverUrl: String, executorId: String, hostname: String, cores: Int) { - val env = SparkEnv.get - env.hadoop.runAsUser(run0, Tuple4[Any, Any, Any, Any] (driverUrl, executorId, hostname, cores)) - } - - // This will be run 'as' the user - def run0(args: Product) { - assert(4 == args.productArity) - runImpl(args.productElement(0).asInstanceOf[String], - args.productElement(1).asInstanceOf[String], - args.productElement(2).asInstanceOf[String], - args.productElement(3).asInstanceOf[Int]) - } - - private def runImpl(driverUrl: String, executorId: String, hostname: String, cores: Int) { // Debug code Utils.checkHost(hostname) |