浏览代码

[livy] Add filesystem prefix to protect against accessing arbitrary files

Erick Tryzelaar 10 年之前
父节点
当前提交
2cbe783

+ 21 - 0
apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/LivyConf.scala

@@ -23,7 +23,12 @@ import java.util.concurrent.ConcurrentHashMap
 import scala.collection.JavaConverters._
 
 object LivyConf {
+  val SESSION_FACTORY_KEY = "livy.server.session.factory"
 
+  sealed trait SessionKind
+  case class Thread() extends SessionKind
+  case class Process() extends SessionKind
+  case class Yarn() extends SessionKind
 }
 
 /**
@@ -31,6 +36,9 @@ object LivyConf {
  * @param loadDefaults whether to also load values from the Java system properties
  */
 class LivyConf(loadDefaults: Boolean) {
+
+  import LivyConf._
+
   /**
    * Create a LivyConf that loads defaults from the system properties and the classpath.
    * @return
@@ -81,4 +89,17 @@ class LivyConf(loadDefaults: Boolean) {
 
   /** Return if the configuration includes this setting */
   def contains(key: String): Boolean = settings.containsKey(key)
+
+  def sessionKind(): SessionKind = getOption(SESSION_FACTORY_KEY).getOrElse("process") match {
+    case "process" => Process()
+    case "thread" => Thread()
+    case "yarn" => Yarn()
+    case kind => throw new IllegalStateException(f"unknown kind $kind")
+  }
+
+  /** Return the filesystem root. Defaults to the local filesystem. */
+  def filesystemRoot(): String = sessionKind() match {
+    case Process() | Thread() => "file://"
+    case Yarn() => "hdfs://"
+  }
 }

+ 17 - 13
apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/spark/SparkSubmitProcessBuilder.scala

@@ -18,18 +18,20 @@
 
 package com.cloudera.hue.livy.spark
 
-import com.cloudera.hue.livy.Logging
+import com.cloudera.hue.livy.{LivyConf, Logging}
 
 import scala.collection.JavaConversions._
 import scala.collection.mutable.ArrayBuffer
 
 object SparkSubmitProcessBuilder {
-  def apply(): SparkSubmitProcessBuilder = {
-    new SparkSubmitProcessBuilder()
+  def apply(livyConf: LivyConf): SparkSubmitProcessBuilder = {
+    new SparkSubmitProcessBuilder(livyConf)
   }
 }
 
-class SparkSubmitProcessBuilder extends Logging {
+class SparkSubmitProcessBuilder(livyConf: LivyConf) extends Logging {
+
+  private[this] val fsRoot = livyConf.filesystemRoot()
 
   private[this] var _executable = "spark-submit"
   private[this] var _master: Option[String] = None
@@ -83,32 +85,32 @@ class SparkSubmitProcessBuilder extends Logging {
   }
 
   def jar(jar: String): SparkSubmitProcessBuilder = {
-    this._jars += jar
+    this._jars += buildPath(jar)
     this
   }
 
   def jars(jars: Traversable[String]): SparkSubmitProcessBuilder = {
-    this._jars ++= jars
+    jars.foreach(jar)
     this
   }
 
   def pyFile(pyFile: String): SparkSubmitProcessBuilder = {
-    this._pyFiles += pyFile
+    this._pyFiles += buildPath(pyFile)
     this
   }
 
   def pyFiles(pyFiles: Traversable[String]): SparkSubmitProcessBuilder = {
-    this._pyFiles ++= pyFiles
+    pyFiles.foreach(pyFile)
     this
   }
 
   def file(file: String): SparkSubmitProcessBuilder = {
-    this._files += file
+    this._files += buildPath(file)
     this
   }
 
   def files(files: Traversable[String]): SparkSubmitProcessBuilder = {
-    this._files ++= files
+    files.foreach(file)
     this
   }
 
@@ -176,12 +178,12 @@ class SparkSubmitProcessBuilder extends Logging {
   }
 
   def archive(archive: String): SparkSubmitProcessBuilder = {
-    _archives += archive
+    _archives += buildPath(archive)
     this
   }
 
   def archives(archives: Traversable[String]): SparkSubmitProcessBuilder = {
-    _archives ++= archives
+    archives.foreach(archive)
     this
   }
 
@@ -241,7 +243,7 @@ class SparkSubmitProcessBuilder extends Logging {
     addOpt("--queue", _queue)
     addList("--archives", _archives)
 
-    args_ += file
+    args_ += buildPath(file)
     args_ ++= args
 
     info(s"Running ${args.mkString(" ")}")
@@ -259,4 +261,6 @@ class SparkSubmitProcessBuilder extends Logging {
 
     pb.start()
   }
+
+  private def buildPath(path: String) = fsRoot + "/" + path
 }

+ 14 - 17
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/Main.scala

@@ -68,29 +68,26 @@ class ScalatraBootstrap extends LifeCycle with Logging {
   override def init(context: ServletContext): Unit = {
     val livyConf = new LivyConf()
 
-    val sessionFactoryKind = livyConf.get("livy.server.session.factory", "process")
-
-    info(f"Using $sessionFactoryKind sessions")
-
-    val sessionFactory = sessionFactoryKind match {
-      case "thread" => new ThreadSessionFactory(livyConf)
-      case "process" => new ProcessSessionFactory(livyConf)
-      case "yarn" => new YarnSessionFactory(livyConf)
-      case _ =>
-        println(f"Unknown session factory: $sessionFactoryKind")
+    val sessionFactoryKind = try {
+      livyConf.sessionKind()
+    } catch {
+      case e: IllegalStateException =>
+        println(f"Unknown session factory: $e}")
         sys.exit(1)
     }
 
-    sessionManager = new SessionManager(sessionFactory)
+    info(f"Using $sessionFactoryKind sessions")
 
-    val batchFactory = sessionFactoryKind match {
-      case "thread" | "process" => new BatchProcessFactory()
-      case "yarn" => new BatchYarnFactory(livyConf)
-      case _ =>
-        println(f"Unknown batch factory: $sessionFactoryKind")
-        sys.exit(1)
+    val (sessionFactory, batchFactory) = sessionFactoryKind match {
+      case LivyConf.Thread() =>
+        (new ThreadSessionFactory(livyConf), new BatchProcessFactory(livyConf) )
+      case LivyConf.Process() =>
+        (new ProcessSessionFactory(livyConf), new BatchProcessFactory(livyConf))
+      case LivyConf.Yarn() =>
+        (new YarnSessionFactory(livyConf), new BatchYarnFactory(livyConf))
     }
 
+    sessionManager = new SessionManager(sessionFactory)
     batchManager = new BatchManager(batchFactory)
 
     context.mount(new SessionServlet(sessionManager), "/sessions/*")

+ 5 - 5
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchProcess.scala

@@ -20,21 +20,21 @@ package com.cloudera.hue.livy.server.batch
 
 import java.lang.ProcessBuilder.Redirect
 
-import com.cloudera.hue.livy.LineBufferedProcess
+import com.cloudera.hue.livy.{LivyConf, LineBufferedProcess}
 import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder
 
 import scala.concurrent.{Future, ExecutionContext, ExecutionContextExecutor}
 
 object BatchProcess {
-  def apply(id: Int, createBatchRequest: CreateBatchRequest): Batch = {
-    val builder = sparkBuilder(createBatchRequest)
+  def apply(livyConf: LivyConf, id: Int, createBatchRequest: CreateBatchRequest): Batch = {
+    val builder = sparkBuilder(livyConf, createBatchRequest)
 
     val process = builder.start(createBatchRequest.file, createBatchRequest.args)
     new BatchProcess(id, new LineBufferedProcess(process))
   }
 
-  private def sparkBuilder(createBatchRequest: CreateBatchRequest): SparkSubmitProcessBuilder = {
-    val builder = SparkSubmitProcessBuilder()
+  private def sparkBuilder(livyConf: LivyConf, createBatchRequest: CreateBatchRequest): SparkSubmitProcessBuilder = {
+    val builder = SparkSubmitProcessBuilder(livyConf)
 
     createBatchRequest.className.foreach(builder.className)
     createBatchRequest.jars.foreach(builder.jar)

+ 4 - 2
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchProcessFactory.scala

@@ -18,7 +18,9 @@
 
 package com.cloudera.hue.livy.server.batch
 
-class BatchProcessFactory extends BatchFactory {
+import com.cloudera.hue.livy.LivyConf
+
+class BatchProcessFactory(livyConf: LivyConf) extends BatchFactory {
   def create(id: Int, createBatchRequest: CreateBatchRequest): Batch =
-    BatchProcess(id, createBatchRequest)
+    BatchProcess(livyConf, id, createBatchRequest)
 }

+ 3 - 3
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchYarn.scala

@@ -33,7 +33,7 @@ object BatchYarn {
   implicit def executor: ExecutionContextExecutor = ExecutionContext.global
 
   def apply(livyConf: LivyConf, client: Client, id: Int, createBatchRequest: CreateBatchRequest): Batch = {
-    val builder = sparkBuilder(createBatchRequest)
+    val builder = sparkBuilder(livyConf, createBatchRequest)
 
     val process = new LineBufferedProcess(builder.start(createBatchRequest.file, createBatchRequest.args))
     val job = Future {
@@ -42,8 +42,8 @@ object BatchYarn {
     new BatchYarn(id, process, job)
   }
 
-  private def sparkBuilder(createBatchRequest: CreateBatchRequest): SparkSubmitProcessBuilder = {
-    val builder = SparkSubmitProcessBuilder()
+  private def sparkBuilder(livyConf: LivyConf, createBatchRequest: CreateBatchRequest): SparkSubmitProcessBuilder = {
+    val builder = SparkSubmitProcessBuilder(livyConf)
 
     builder.master("yarn-cluster")
 

+ 1 - 1
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/sessions/ProcessSession.scala

@@ -45,7 +45,7 @@ object ProcessSession extends Logging {
   // Loop until we've started a process with a valid port.
   private def startProcess(livyConf: LivyConf, id: Int, kind: Kind, proxyUser: Option[String]): Process = {
 
-    val builder = new SparkSubmitProcessBuilder()
+    val builder = new SparkSubmitProcessBuilder(livyConf)
 
     builder.className("com.cloudera.hue.livy.repl.Main")
 

+ 1 - 1
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/sessions/YarnSession.scala

@@ -39,7 +39,7 @@ object YarnSession {
     val callbackUrl = System.getProperty("livy.server.callback-url")
     val url = f"$callbackUrl/sessions/$id/callback"
 
-    val builder = SparkSubmitProcessBuilder()
+    val builder = SparkSubmitProcessBuilder(livyConf)
 
     builder.master("yarn-cluster")
     builder.className("com.cloudera.hue.livy.repl.Main")

+ 2 - 2
apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/batches/BatchProcessSpec.scala

@@ -22,7 +22,7 @@ import java.io.FileWriter
 import java.nio.file.{Files, Path}
 import java.util.concurrent.TimeUnit
 
-import com.cloudera.hue.livy.Utils
+import com.cloudera.hue.livy.{LivyConf, Utils}
 import com.cloudera.hue.livy.server.batch.{Success, CreateBatchRequest, BatchProcess}
 import org.scalatest.{ShouldMatchers, BeforeAndAfterAll, FunSpec}
 
@@ -53,7 +53,7 @@ class BatchProcessSpec
       val req = CreateBatchRequest(
         file = script.toString
       )
-      val batch = BatchProcess(0, req)
+      val batch = BatchProcess(new LivyConf(), 0, req)
 
       Utils.waitUntil({ () =>
         batch.state == Success()

+ 2 - 2
apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/batches/BatchServletSpec.scala

@@ -22,7 +22,7 @@ import java.io.FileWriter
 import java.nio.file.{Files, Path}
 import java.util.concurrent.TimeUnit
 
-import com.cloudera.hue.livy.Utils
+import com.cloudera.hue.livy.{LivyConf, Utils}
 import com.cloudera.hue.livy.server.batch._
 import org.json4s.JsonAST.{JArray, JInt, JObject, JString}
 import org.json4s.jackson.JsonMethods._
@@ -52,7 +52,7 @@ class BatchServletSpec extends ScalatraSuite with FunSpecLike with BeforeAndAfte
     script
   }
 
-  val batchFactory = new BatchProcessFactory()
+  val batchFactory = new BatchProcessFactory(new LivyConf())
   val batchManager = new BatchManager(batchFactory)
   val servlet = new BatchServlet(batchManager)