Browse Source

[livy] Initial support for batch jobs

Erick Tryzelaar 10 years ago
parent
commit
da0ec82306

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

@@ -5,7 +5,13 @@ import com.cloudera.hue.livy.Logging
 import scala.collection.JavaConversions._
 import scala.collection.mutable.ArrayBuffer
 
-class SparkProcessBuilder extends Logging {
+object SparkSubmitProcessBuilder {
+  def apply(): SparkSubmitProcessBuilder = {
+    new SparkSubmitProcessBuilder()
+  }
+}
+
+class SparkSubmitProcessBuilder extends Logging {
 
   private[this] var _executable = "spark-submit"
   private[this] var _master: Option[String] = None
@@ -33,142 +39,150 @@ class SparkProcessBuilder extends Logging {
   private[this] var _redirectError: Option[ProcessBuilder.Redirect] = None
   private[this] var _redirectErrorStream: Option[Boolean] = None
 
-  def executable(executable: String): SparkProcessBuilder = {
+  def executable(executable: String): SparkSubmitProcessBuilder = {
     _executable = executable
     this
   }
 
-  def master(masterUrl: String): SparkProcessBuilder = {
+  def master(masterUrl: String): SparkSubmitProcessBuilder = {
     _master = Some(masterUrl)
     this
   }
 
-  def deployMode(deployMode: String): SparkProcessBuilder = {
+  def deployMode(deployMode: String): SparkSubmitProcessBuilder = {
     _deployMode = Some(deployMode)
     this
   }
 
-  def className(className: String): SparkProcessBuilder = {
+  def className(className: String): SparkSubmitProcessBuilder = {
     _className = Some(className)
     this
   }
 
-  def name(name: String): SparkProcessBuilder = {
+  def name(name: String): SparkSubmitProcessBuilder = {
     _name = Some(name)
     this
   }
 
-  def jar(jar: String): SparkProcessBuilder = {
+  def jar(jar: String): SparkSubmitProcessBuilder = {
     this._jars += jar
     this
   }
 
-  def jars(jars: Traversable[String]): SparkProcessBuilder = {
+  def jars(jars: Traversable[String]): SparkSubmitProcessBuilder = {
     this._jars ++= jars
     this
   }
 
-  def pyFile(pyFile: String): SparkProcessBuilder = {
+  def pyFile(pyFile: String): SparkSubmitProcessBuilder = {
     this._pyFiles += pyFile
     this
   }
 
-  def pyFiles(pyFiles: Traversable[String]): SparkProcessBuilder = {
+  def pyFiles(pyFiles: Traversable[String]): SparkSubmitProcessBuilder = {
     this._pyFiles ++= pyFiles
     this
   }
 
-  def file(file: String): SparkProcessBuilder = {
+  def file(file: String): SparkSubmitProcessBuilder = {
     this._files += file
     this
   }
 
-  def files(files: Traversable[String]): SparkProcessBuilder = {
+  def files(files: Traversable[String]): SparkSubmitProcessBuilder = {
     this._files ++= files
     this
   }
 
-  def conf(key: String, value: String): SparkProcessBuilder = {
+  def conf(key: String, value: String): SparkSubmitProcessBuilder = {
     this._conf += ((key, value))
     this
   }
 
-  def conf(conf: Traversable[(String, String)]): SparkProcessBuilder = {
+  def conf(conf: Traversable[(String, String)]): SparkSubmitProcessBuilder = {
     this._conf ++= conf
     this
   }
 
-  def driverMemory(driverMemory: String): SparkProcessBuilder = {
+  def driverMemory(driverMemory: String): SparkSubmitProcessBuilder = {
     _driverMemory = Some(driverMemory)
     this
   }
 
-  def driverJavaOptions(driverJavaOptions: String): SparkProcessBuilder = {
+  def driverJavaOptions(driverJavaOptions: String): SparkSubmitProcessBuilder = {
     _driverJavaOptions = Some(driverJavaOptions)
     this
   }
 
-  def driverClassPath(classPath: String): SparkProcessBuilder = {
+  def driverClassPath(classPath: String): SparkSubmitProcessBuilder = {
     _driverClassPath += classPath
     this
   }
 
-  def driverClassPaths(classPaths: Traversable[String]): SparkProcessBuilder = {
+  def driverClassPaths(classPaths: Traversable[String]): SparkSubmitProcessBuilder = {
     _driverClassPath ++= classPaths
     this
   }
 
-  def executorMemory(executorMemory: String): SparkProcessBuilder = {
+  def executorMemory(executorMemory: String): SparkSubmitProcessBuilder = {
     _executorMemory = Some(executorMemory)
     this
   }
 
-  def proxyUser(proxyUser: String): SparkProcessBuilder = {
+  def proxyUser(proxyUser: String): SparkSubmitProcessBuilder = {
     _proxyUser = Some(proxyUser)
     this
   }
 
-  def driverCores(driverCores: String): SparkProcessBuilder = {
+  def driverCores(driverCores: Int): SparkSubmitProcessBuilder = {
+    this.driverCores(driverCores.toString)
+  }
+
+  def driverCores(driverCores: String): SparkSubmitProcessBuilder = {
     _driverCores = Some(driverCores)
     this
   }
 
-  def executorCores(executorCores: String): SparkProcessBuilder = {
+  def executorCores(executorCores: Int): SparkSubmitProcessBuilder = {
+    this.executorCores(executorCores.toString)
+  }
+
+  def executorCores(executorCores: String): SparkSubmitProcessBuilder = {
     _executorCores = Some(executorCores)
     this
   }
 
-  def queue(queue: String): SparkProcessBuilder = {
+  def queue(queue: String): SparkSubmitProcessBuilder = {
     _queue = Some(queue)
     this
   }
 
-  def archive(archive: String): SparkProcessBuilder = {
+  def archive(archive: String): SparkSubmitProcessBuilder = {
     _archives += archive
     this
   }
 
-  def archives(archives: Traversable[String]): SparkProcessBuilder = {
+  def archives(archives: Traversable[String]): SparkSubmitProcessBuilder = {
     _archives ++= archives
     this
   }
 
-  def env(key: String, value: String): SparkProcessBuilder = {
+  def env(key: String, value: String): SparkSubmitProcessBuilder = {
     _env += ((key, value))
     this
   }
 
-  def redirectOutput(redirect: ProcessBuilder.Redirect): SparkProcessBuilder = {
+  def redirectOutput(redirect: ProcessBuilder.Redirect): SparkSubmitProcessBuilder = {
     _redirectOutput = Some(redirect)
     this
   }
 
-  def redirectError(redirect: ProcessBuilder.Redirect): SparkProcessBuilder = {
+  def redirectError(redirect: ProcessBuilder.Redirect): SparkSubmitProcessBuilder = {
     _redirectError = Some(redirect)
     this
   }
 
-  def redirectErrorStream(redirect: Boolean): SparkProcessBuilder = {
+  def redirectErrorStream(redirect: Boolean): SparkSubmitProcessBuilder = {
     _redirectErrorStream = Some(redirect)
     this
   }

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

@@ -2,6 +2,7 @@ package com.cloudera.hue.livy.server
 
 import javax.servlet.ServletContext
 
+import com.cloudera.hue.livy.server.batch.{BatchProcessFactory, BatchServlet, BatchManager}
 import com.cloudera.hue.livy.server.sessions._
 import com.cloudera.hue.livy.{Utils, Logging, LivyConf, WebServer}
 import org.scalatra._
@@ -44,6 +45,7 @@ object Main {
 class ScalatraBootstrap extends LifeCycle with Logging {
 
   var sessionManager: SessionManager = null
+  var batchManager: BatchManager = null
 
   override def init(context: ServletContext): Unit = {
     val livyConf = new LivyConf()
@@ -63,12 +65,20 @@ class ScalatraBootstrap extends LifeCycle with Logging {
 
     sessionManager = new SessionManager(sessionFactory)
 
+    val batchFactory = new BatchProcessFactory()
+    batchManager = new BatchManager(batchFactory)
+
     context.mount(new SessionServlet(sessionManager), "/sessions/*")
+    context.mount(new BatchServlet(batchManager), "/batches/*")
   }
 
   override def destroy(context: ServletContext): Unit = {
     if (sessionManager != null) {
       sessionManager.shutdown()
     }
+
+    if (batchManager != null) {
+      batchManager.shutdown()
+    }
   }
 }

+ 98 - 0
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchManager.scala

@@ -0,0 +1,98 @@
+package com.cloudera.hue.livy.server.batch
+
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicInteger
+
+import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder
+
+import scala.collection.JavaConversions._
+import scala.concurrent.{ExecutionContext, ExecutionContextExecutor, Future}
+
+class BatchManager(batchFactory: BatchFactory) {
+  private[this] val _idCounter = new AtomicInteger()
+  private[this] val _batches = new ConcurrentHashMap[Int, Batch]
+
+  def getBatch(id: Int): Option[Batch] = Option(_batches.get(id))
+
+  def getBatches: Array[Batch] = _batches.values().iterator().toArray
+
+  def createBatch(createBatchRequest: CreateBatchRequest): Batch = {
+    val id = _idCounter.getAndIncrement
+    val batch = batchFactory.createBatch(id, createBatchRequest)
+    _batches.put(id, batch)
+
+    batch
+  }
+
+  def remove(id: Int): Option[Batch] = {
+    Option(_batches.remove(id))
+  }
+
+  def delete(batch: Batch): Future[Unit] = {
+    _batches.remove(batch.id)
+    batch.stop()
+  }
+
+  def shutdown() = {
+
+  }
+}
+
+case class CreateBatchRequest(file: String,
+                              args: List[String] = List(),
+                              className: Option[String] = None,
+                              jars: List[String] = List(),
+                              pyFiles: List[String] = List(),
+                              files: List[String] = List(),
+                              driverMemory: Option[String] = None,
+                              driverCores: Option[Int] = None,
+                              executorMemory: Option[String] = None,
+                              executorCores: Option[Int] = None,
+                              archives: List[String] = List())
+
+abstract class BatchFactory {
+  def createBatch(id: Int, createBatchRequest: CreateBatchRequest): Batch
+}
+
+class BatchProcessFactory extends BatchFactory {
+  def createBatch(id: Int, createBatchRequest: CreateBatchRequest): Batch =
+    BatchProcess(id, createBatchRequest)
+}
+
+abstract class Batch {
+  def id: Int
+
+  def stop(): Future[Unit]
+}
+
+object BatchProcess {
+  def apply(id: Int, createBatchRequest: CreateBatchRequest): Batch = {
+    val builder = SparkSubmitProcessBuilder()
+
+    createBatchRequest.className.foreach(builder.className)
+    createBatchRequest.jars.foreach(builder.jar)
+    createBatchRequest.pyFiles.foreach(builder.pyFile)
+    createBatchRequest.files.foreach(builder.file)
+    createBatchRequest.driverMemory.foreach(builder.driverMemory)
+    createBatchRequest.driverCores.foreach(builder.driverCores)
+    createBatchRequest.executorMemory.foreach(builder.executorMemory)
+    createBatchRequest.executorCores.foreach(builder.executorCores)
+    createBatchRequest.archives.foreach(builder.archive)
+
+    val process = builder.start(createBatchRequest.file, createBatchRequest.args)
+    new BatchProcess(id, process)
+  }
+}
+
+private class BatchProcess(val id: Int,
+                           @transient
+                           process: Process) extends Batch {
+  protected implicit def executor: ExecutionContextExecutor = ExecutionContext.global
+
+  override def stop(): Future[Unit] = {
+    Future {
+      process.destroy()
+      process.waitFor()
+    }
+  }
+}

+ 93 - 0
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchServlet.scala

@@ -0,0 +1,93 @@
+package com.cloudera.hue.livy.server.batch
+
+import com.cloudera.hue.livy.Logging
+import com.fasterxml.jackson.core.JsonParseException
+import org.json4s._
+import org.scalatra._
+import org.scalatra.json.JacksonJsonSupport
+
+import scala.concurrent.{Future, ExecutionContext, ExecutionContextExecutor}
+
+object BatchServlet extends Logging
+
+class BatchServlet(batchManager: BatchManager)
+  extends ScalatraServlet
+  with FutureSupport
+  with MethodOverride
+  with JacksonJsonSupport
+  with UrlGeneratorSupport
+{
+  override protected implicit def executor: ExecutionContextExecutor = ExecutionContext.global
+  override protected implicit def jsonFormats: Formats = DefaultFormats ++ Serializers.Formats
+
+  before() {
+    contentType = formats("json")
+  }
+
+  get("/") {
+    Map(
+      "batches" -> batchManager.getBatches
+    )
+  }
+
+  post("/") {
+    val createBatchRequest = parsedBody.extract[CreateBatchRequest]
+
+    new AsyncResult {
+      val is = Future {
+        val batch = batchManager.createBatch(createBatchRequest)
+        Created(batch,
+          headers = Map(
+            "Location" -> url(getBatch, "id" -> batch.id.toString)
+          )
+        )
+      }
+    }
+  }
+
+  val getBatch = get("/:id") {
+    val id = params("id").toInt
+    batchManager.getBatch(id) match {
+      case None => NotFound("batch not found")
+      case Some(batch) => batch
+    }
+  }
+
+  delete("/:id") {
+    val id = params("id").toInt
+
+    batchManager.remove(id) match {
+      case None => NotFound("batch not found")
+      case Some(batch) =>
+        new AsyncResult {
+          val is = batch.stop().map { case () =>
+            batchManager.delete(batch)
+            Ok(Map("msg" -> "deleted"))
+          }
+        }
+    }
+  }
+
+  error {
+    case e: JsonParseException => BadRequest(e.getMessage)
+    case e: MappingException => BadRequest(e.getMessage)
+    case e =>
+      BatchServlet.error("internal error", e)
+      InternalServerError(e.toString)
+  }
+}
+
+private object Serializers {
+  import JsonDSL._
+
+  def Formats: List[CustomSerializer[_]] = List(BatchSerializer)
+
+  case object BatchSerializer extends CustomSerializer[Batch](implicit formats => ( {
+    // We don't support deserialization.
+    PartialFunction.empty
+  }, {
+    case batch: Batch => JObject(JField("id", batch.id))
+  }
+    )
+  )
+}

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

@@ -4,7 +4,7 @@ import java.lang.ProcessBuilder.Redirect
 import java.net.URL
 
 import com.cloudera.hue.livy.sessions.Kind
-import com.cloudera.hue.livy.spark.SparkProcessBuilder
+import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder
 import com.cloudera.hue.livy.{LivyConf, Logging, Utils}
 
 import scala.annotation.tailrec
@@ -27,7 +27,7 @@ object ProcessSession extends Logging {
   // Loop until we've started a process with a valid port.
   private def startProcess(livyConf: LivyConf, id: String, kind: Kind, proxyUser: Option[String]): Process = {
 
-    val builder = new SparkProcessBuilder()
+    val builder = new SparkSubmitProcessBuilder()
 
     builder.className("com.cloudera.hue.livy.repl.Main")
 

+ 0 - 2
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/sessions/SessionServlet.scala

@@ -175,8 +175,6 @@ private object Serializers {
     PartialFunction.empty
   }, {
     case session: Session =>
-      import JsonDSL._
-
       ("id", session.id) ~
       ("state", serializeSessionState(session.state)) ~
       ("kind", serializeSessionKind(session.kind)) ~

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

@@ -0,0 +1,82 @@
+package com.cloudera.hue.livy.server.batches
+
+import java.io.FileWriter
+import java.nio.file.{Files, Path}
+
+import com.cloudera.hue.livy.server.batch._
+import org.json4s.JsonAST.{JArray, JInt, JObject, JString}
+import org.json4s.jackson.JsonMethods._
+import org.json4s.jackson.Serialization.write
+import org.json4s.{DefaultFormats, Formats}
+import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSpecLike}
+import org.scalatra.test.scalatest.ScalatraSuite
+
+class BatchServletSpec extends ScalatraSuite with FunSpecLike with BeforeAndAfterAll with BeforeAndAfter {
+
+  protected implicit def jsonFormats: Formats = DefaultFormats
+
+  var script: Path = _
+
+  val batchFactory = new BatchProcessFactory()
+  val batchManager = new BatchManager(batchFactory)
+  val servlet = new BatchServlet(batchManager)
+
+  addServlet(servlet, "/*")
+
+  override def beforeAll() = {
+    super.beforeAll()
+    script = Files.createTempFile("test", "livy-test")
+    val writer = new FileWriter(script.toFile)
+    try {
+      writer.write("print 'hello world'")
+    } finally {
+      writer.close()
+    }
+  }
+
+  override def afterAll() = {
+    script.toFile.delete()
+    super.afterAll()
+  }
+
+  after {
+    batchManager.shutdown()
+  }
+
+  describe("Batch Servlet") {
+    it("should create and tear down a batch") {
+      get("/") {
+        status should equal (200)
+        header("Content-Type") should include("application/json")
+        val parsedBody = parse(body)
+        parsedBody \ "batches" should equal (JArray(List()))
+      }
+
+      val createBatchRequest = write(CreateBatchRequest(
+        file = script.toString
+      ))
+
+      post("/", body = createBatchRequest, headers = Map("Content-Type" -> "application/json")) {
+        status should equal (201)
+        header("Content-Type") should include("application/json")
+        header("Location") should equal("/0")
+        val parsedBody = parse(body)
+        parsedBody \ "id" should equal (JInt(0))
+
+        val batch = batchManager.getBatch(0)
+        batch should be (defined)
+      }
+
+      delete("/0") {
+        status should equal (200)
+        header("Content-Type") should include("application/json")
+        val parsedBody = parse(body)
+        parsedBody should equal (JObject(("msg", JString("deleted"))))
+
+        val batch = batchManager.getBatch(0)
+        batch should not be defined
+      }
+    }
+  }
+
+}

+ 2 - 2
apps/spark/java/livy-yarn/src/main/scala/com/cloudera/hue/livy/yarn/Client.scala

@@ -3,7 +3,7 @@ package com.cloudera.hue.livy.yarn
 import java.io.{BufferedReader, InputStreamReader}
 import java.lang.ProcessBuilder.Redirect
 
-import com.cloudera.hue.livy.spark.SparkProcessBuilder
+import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder
 import com.cloudera.hue.livy.{LivyConf, Logging, Utils}
 import org.apache.hadoop.yarn.api.records.{ApplicationId, FinalApplicationStatus, YarnApplicationState}
 import org.apache.hadoop.yarn.client.api.YarnClient
@@ -46,7 +46,7 @@ class Client(livyConf: LivyConf) extends Logging {
                         callbackUrl: String): Future[Job] = {
     val url = f"$callbackUrl/sessions/$id/callback"
 
-    val builder = new SparkProcessBuilder()
+    val builder = new SparkSubmitProcessBuilder()
 
     builder.master("yarn-cluster")
     builder.className("com.cloudera.hue.livy.repl.Main")