Pārlūkot izejas kodu

[livy] Refactor yarn code to support batch yarn-cluster jobs

Erick Tryzelaar 10 gadi atpakaļ
vecāks
revīzija
ba33ba826d

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

@@ -85,7 +85,7 @@ class ScalatraBootstrap extends LifeCycle with Logging {
 
 
     val batchFactory = sessionFactoryKind match {
     val batchFactory = sessionFactoryKind match {
       case "thread" | "process" => new BatchProcessFactory()
       case "thread" | "process" => new BatchProcessFactory()
-      case "yarn" => new BatchYarnFactory()
+      case "yarn" => new BatchYarnFactory(livyConf)
       case _ =>
       case _ =>
         println(f"Unknown batch factory: $sessionFactoryKind")
         println(f"Unknown batch factory: $sessionFactoryKind")
         sys.exit(1)
         sys.exit(1)

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

@@ -0,0 +1,31 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.cloudera.hue.livy.server.batch
+
+import scala.concurrent.Future
+
+abstract class Batch {
+  def id: Int
+
+  def state: State
+
+  def lines: IndexedSeq[String]
+
+  def stop(): Future[Unit]
+}

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

@@ -0,0 +1,23 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.cloudera.hue.livy.server.batch
+
+abstract class BatchFactory {
+  def create(id: Int, createBatchRequest: CreateBatchRequest): Batch
+}

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

@@ -18,15 +18,11 @@
 
 
 package com.cloudera.hue.livy.server.batch
 package com.cloudera.hue.livy.server.batch
 
 
-import java.lang.ProcessBuilder.Redirect
 import java.util.concurrent.ConcurrentHashMap
 import java.util.concurrent.ConcurrentHashMap
 import java.util.concurrent.atomic.AtomicInteger
 import java.util.concurrent.atomic.AtomicInteger
 
 
-import com.cloudera.hue.livy.LineBufferedProcess
-import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder
-
 import scala.collection.JavaConversions._
 import scala.collection.JavaConversions._
-import scala.concurrent.{ExecutionContext, ExecutionContextExecutor, Future}
+import scala.concurrent.Future
 
 
 class BatchManager(batchFactory: BatchFactory) {
 class BatchManager(batchFactory: BatchFactory) {
   private[this] val _idCounter = new AtomicInteger()
   private[this] val _idCounter = new AtomicInteger()
@@ -38,7 +34,7 @@ class BatchManager(batchFactory: BatchFactory) {
 
 
   def createBatch(createBatchRequest: CreateBatchRequest): Batch = {
   def createBatch(createBatchRequest: CreateBatchRequest): Batch = {
     val id = _idCounter.getAndIncrement
     val id = _idCounter.getAndIncrement
-    val batch = batchFactory.createBatch(id, createBatchRequest)
+    val batch = batchFactory.create(id, createBatchRequest)
     _batches.put(id, batch)
     _batches.put(id, batch)
 
 
     batch
     batch
@@ -69,101 +65,3 @@ case class CreateBatchRequest(file: String,
                               executorMemory: Option[String] = None,
                               executorMemory: Option[String] = None,
                               executorCores: Option[Int] = None,
                               executorCores: Option[Int] = None,
                               archives: List[String] = List())
                               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, "local[*]", createBatchRequest)
-}
-
-class BatchYarnFactory extends BatchFactory {
-  def createBatch(id: Int, createBatchRequest: CreateBatchRequest): Batch =
-    BatchProcess(id, "yarn-client", createBatchRequest)
-}
-
-sealed trait State
-
-case class Running() extends State {
-  override def toString = "running"
-}
-
-case class Dead() extends State {
-  override def toString = "dead"
-}
-
-abstract class Batch {
-  def id: Int
-
-  def state: State
-
-  def lines: IndexedSeq[String]
-
-  def stop(): Future[Unit]
-}
-
-object BatchProcess {
-  def apply(id: Int, master: String, createBatchRequest: CreateBatchRequest): Batch = {
-    val builder = sparkBuilder(createBatchRequest)
-
-    builder.master(master)
-
-    val process = builder.start(createBatchRequest.file, createBatchRequest.args)
-    new BatchProcess(id, new LineBufferedProcess(process))
-  }
-
-  private def sparkBuilder(createBatchRequest: CreateBatchRequest): SparkSubmitProcessBuilder = {
-    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)
-
-    builder.redirectOutput(Redirect.PIPE)
-
-    builder
-  }
-}
-
-private class BatchProcess(val id: Int,
-                           process: LineBufferedProcess) extends Batch {
-  protected implicit def executor: ExecutionContextExecutor = ExecutionContext.global
-
-  private[this] var isAlive = true
-
-  override def state: State = {
-    if (isAlive) {
-      try {
-        process.exitValue()
-      } catch {
-        case e: IllegalThreadStateException => return Running()
-      }
-
-      destroyProcess()
-    }
-
-    Dead()
-  }
-
-  override def lines: IndexedSeq[String] = process.stdoutLines
-
-  override def stop(): Future[Unit] = {
-    Future {
-      destroyProcess()
-    }
-  }
-
-  private def destroyProcess() = {
-    process.destroy()
-    process.waitFor()
-    isAlive = false
-  }
-}

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

@@ -0,0 +1,88 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.cloudera.hue.livy.server.batch
+
+import java.lang.ProcessBuilder.Redirect
+
+import com.cloudera.hue.livy.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)
+
+    val process = builder.start(createBatchRequest.file, createBatchRequest.args)
+    new BatchProcess(id, new LineBufferedProcess(process))
+  }
+
+  private def sparkBuilder(createBatchRequest: CreateBatchRequest): SparkSubmitProcessBuilder = {
+    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)
+
+    builder.redirectOutput(Redirect.PIPE)
+
+    builder
+  }
+}
+
+private class BatchProcess(val id: Int,
+                           process: LineBufferedProcess) extends Batch {
+  protected implicit def executor: ExecutionContextExecutor = ExecutionContext.global
+
+  private[this] var isAlive = true
+
+  override def state: State = {
+    if (isAlive) {
+      try {
+        process.exitValue()
+      } catch {
+        case e: IllegalThreadStateException => return Running()
+      }
+
+      destroyProcess()
+    }
+
+    Dead()
+  }
+
+  override def lines: IndexedSeq[String] = process.stdoutLines
+
+  override def stop(): Future[Unit] = {
+    Future {
+      destroyProcess()
+    }
+  }
+
+  private def destroyProcess() = {
+    process.destroy()
+    process.waitFor()
+    isAlive = false
+  }
+}

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

@@ -0,0 +1,24 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.cloudera.hue.livy.server.batch
+
+class BatchProcessFactory extends BatchFactory {
+  def create(id: Int, createBatchRequest: CreateBatchRequest): Batch =
+    BatchProcess(id, createBatchRequest)
+}

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

@@ -0,0 +1,110 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.cloudera.hue.livy.server.batch
+
+import java.lang.ProcessBuilder.Redirect
+
+import com.cloudera.hue.livy.LivyConf
+import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder
+import com.cloudera.hue.livy.yarn._
+
+import scala.annotation.tailrec
+import scala.concurrent.{ExecutionContextExecutor, ExecutionContext, Future}
+import scala.util.{Failure, Success}
+
+object BatchYarn {
+
+  implicit def executor: ExecutionContextExecutor = ExecutionContext.global
+
+  def apply(livyConf: LivyConf, client: Client, id: Int, createBatchRequest: CreateBatchRequest): Batch = {
+    val builder = sparkBuilder(createBatchRequest)
+
+    val process = builder.start(createBatchRequest.file, createBatchRequest.args)
+    new BatchYarn(id, Future { client.getJobFromProcess(process) })
+  }
+
+  private def sparkBuilder(createBatchRequest: CreateBatchRequest): SparkSubmitProcessBuilder = {
+    val builder = SparkSubmitProcessBuilder()
+
+    builder.master("yarn-cluster")
+
+    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)
+
+    builder.redirectOutput(Redirect.PIPE)
+
+    builder
+  }
+}
+
+private class BatchYarn(val id: Int, jobFuture: Future[Job]) extends Batch {
+
+  implicit def executor: ExecutionContextExecutor = ExecutionContext.global
+
+  private var _state: State = Starting()
+
+  private var _jobThread: Thread = _
+
+  jobFuture.onComplete {
+    case Failure(_) => _state = Error()
+    case Success(job) =>
+      _state = Running()
+
+      _jobThread = new Thread {
+        override def run(): Unit = {
+          @tailrec
+          def aux(): Unit = {
+            if (_state == Running()) {
+              Thread.sleep(5000)
+              job.getStatus match {
+                case Client.SuccessfulFinish() =>
+                  _state = Dead()
+                case Client.UnsuccessfulFinish() =>
+                  _state = Error()
+                case _ => aux()
+              }
+            }
+          }
+
+          aux()
+        }
+      }
+      _jobThread.setDaemon(true)
+      _jobThread.start()
+  }
+
+  override def state: State = _state
+
+  override def stop(): Future[Unit] = {
+    jobFuture.map { job =>
+      job.stop()
+      _state = Dead()
+      ()
+    }
+  }
+
+  override def lines: IndexedSeq[String] = IndexedSeq()
+}

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

@@ -0,0 +1,30 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.cloudera.hue.livy.server.batch
+
+import com.cloudera.hue.livy.LivyConf
+import com.cloudera.hue.livy.yarn.Client
+
+class BatchYarnFactory(livyConf: LivyConf) extends BatchFactory {
+
+  val client = new Client(livyConf)
+
+  def create(id: Int, createBatchRequest: CreateBatchRequest): Batch =
+    BatchYarn(livyConf, client, id, createBatchRequest)
+}

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

@@ -0,0 +1,37 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.cloudera.hue.livy.server.batch
+
+sealed trait State
+
+case class Starting() extends State {
+  override def toString = "starting"
+}
+
+case class Running() extends State {
+  override def toString = "running"
+}
+
+case class Error() extends State {
+  override def toString = "error"
+}
+
+case class Dead() extends State {
+  override def toString = "dead"
+}

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

@@ -58,11 +58,16 @@ class ProcessSessionFactory(livyConf: LivyConf) extends SessionFactory {
 
 
 class YarnSessionFactory(livyConf: LivyConf) extends SessionFactory {
 class YarnSessionFactory(livyConf: LivyConf) extends SessionFactory {
 
 
+  implicit def executor: ExecutionContext = ExecutionContext.global
+
   val client = new Client(livyConf)
   val client = new Client(livyConf)
 
 
   override def createSession(kind: Kind, proxyUser: Option[String] = None): Future[Session] = {
   override def createSession(kind: Kind, proxyUser: Option[String] = None): Future[Session] = {
     val id = UUID.randomUUID().toString
     val id = UUID.randomUUID().toString
-    YarnSession.create(client, id, kind, proxyUser)
+
+    Future {
+      YarnSession.create(livyConf, client, id, kind, proxyUser)
+    }
   }
   }
 
 
   override def close(): Unit = {
   override def close(): Unit = {

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

@@ -18,8 +18,11 @@
 
 
 package com.cloudera.hue.livy.server.sessions
 package com.cloudera.hue.livy.server.sessions
 
 
+import java.lang.ProcessBuilder.Redirect
 import java.util.concurrent.TimeUnit
 import java.util.concurrent.TimeUnit
 
 
+import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder
+import com.cloudera.hue.livy.{Utils, LivyConf}
 import com.cloudera.hue.livy.sessions.{Kind, Error}
 import com.cloudera.hue.livy.sessions.{Kind, Error}
 import com.cloudera.hue.livy.yarn.{Client, Job}
 import com.cloudera.hue.livy.yarn.{Client, Job}
 
 
@@ -29,15 +32,38 @@ import scala.concurrent.duration._
 object YarnSession {
 object YarnSession {
   protected implicit def executor: ExecutionContextExecutor = ExecutionContext.global
   protected implicit def executor: ExecutionContextExecutor = ExecutionContext.global
 
 
-  def create(client: Client, id: String, kind: Kind, proxyUser: Option[String] = None): Future[Session] = {
+  private val CONF_LIVY_JAR = "livy.yarn.jar"
+  private lazy val regex = """Application report for (\w+)""".r.unanchored
+
+  def create(livyConf: LivyConf, client: Client, id: String, kind: Kind, proxyUser: Option[String] = None): Session = {
     val callbackUrl = System.getProperty("livy.server.callback-url")
     val callbackUrl = System.getProperty("livy.server.callback-url")
-    val job = client.submitApplication(
-      id = id,
-      kind = kind.toString,
-      proxyUser = proxyUser,
-      callbackUrl = callbackUrl)
+    val url = f"$callbackUrl/sessions/$id/callback"
+
+    val builder = SparkSubmitProcessBuilder()
+
+    builder.master("yarn-cluster")
+    builder.className("com.cloudera.hue.livy.repl.Main")
+    builder.driverJavaOptions(f"-Dlivy.repl.callback-url=$url -Dlivy.repl.port=0")
+    proxyUser.foreach(builder.proxyUser)
+
+    builder.redirectOutput(Redirect.PIPE)
+    builder.redirectErrorStream(redirect = true)
+
+    val process = builder.start(livyJar(livyConf), List(kind.toString))
 
 
-    Future.successful(new YarnSession(id, kind, proxyUser, job))
+    val job = Future {
+      client.getJobFromProcess(process)
+    }
+
+    new YarnSession(id, kind, proxyUser, job)
+  }
+
+  private def livyJar(livyConf: LivyConf) = {
+    if (livyConf.contains(CONF_LIVY_JAR)) {
+      livyConf.get(CONF_LIVY_JAR)
+    } else {
+      Utils.jarOfClass(classOf[Client]).head
+    }
   }
   }
 }
 }
 
 

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

@@ -53,7 +53,7 @@ class BatchProcessSpec
       val req = CreateBatchRequest(
       val req = CreateBatchRequest(
         file = script.toString
         file = script.toString
       )
       )
-      val batch = BatchProcess(0, "local[*]", req)
+      val batch = BatchProcess(0, req)
 
 
       Utils.waitUntil({ () =>
       Utils.waitUntil({ () =>
         batch.state == Dead()
         batch.state == Dead()

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

@@ -19,74 +19,50 @@
 package com.cloudera.hue.livy.yarn
 package com.cloudera.hue.livy.yarn
 
 
 import java.io.{BufferedReader, InputStreamReader}
 import java.io.{BufferedReader, InputStreamReader}
-import java.lang.ProcessBuilder.Redirect
 
 
-import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder
-import com.cloudera.hue.livy.{LivyConf, Logging, Utils}
+import com.cloudera.hue.livy.yarn.Client._
+import com.cloudera.hue.livy.{LivyConf, Logging}
 import org.apache.hadoop.yarn.api.records.{ApplicationId, FinalApplicationStatus, YarnApplicationState}
 import org.apache.hadoop.yarn.api.records.{ApplicationId, FinalApplicationStatus, YarnApplicationState}
 import org.apache.hadoop.yarn.client.api.YarnClient
 import org.apache.hadoop.yarn.client.api.YarnClient
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.util.ConverterUtils
 import org.apache.hadoop.yarn.util.ConverterUtils
 
 
 import scala.annotation.tailrec
 import scala.annotation.tailrec
-import scala.collection.JavaConversions._
-import scala.collection.mutable.ArrayBuffer
-import scala.concurrent.{ExecutionContext, Future}
+import scala.concurrent.ExecutionContext
 
 
-object Client extends Logging {
-  private val CONF_LIVY_JAR = "livy.yarn.jar"
+object Client {
   private lazy val regex = """Application report for (\w+)""".r.unanchored
   private lazy val regex = """Application report for (\w+)""".r.unanchored
 
 
-  private def livyJar(conf: LivyConf) = {
-    if (conf.contains(CONF_LIVY_JAR)) {
-      conf.get(CONF_LIVY_JAR)
-    } else {
-      Utils.jarOfClass(classOf[Client]).head
-    }
-  }
+  sealed trait ApplicationStatus
+  case class New() extends ApplicationStatus
+  case class Accepted() extends ApplicationStatus
+  case class Running() extends ApplicationStatus
+  case class SuccessfulFinish() extends ApplicationStatus
+  case class UnsuccessfulFinish() extends ApplicationStatus
 }
 }
 
 
 class FailedToSubmitApplication extends Exception
 class FailedToSubmitApplication extends Exception
 
 
 class Client(livyConf: LivyConf) extends Logging {
 class Client(livyConf: LivyConf) extends Logging {
-  import com.cloudera.hue.livy.yarn.Client._
+  import Client._
 
 
   protected implicit def executor: ExecutionContext = ExecutionContext.global
   protected implicit def executor: ExecutionContext = ExecutionContext.global
 
 
-  val yarnConf = new YarnConfiguration()
-  val yarnClient = YarnClient.createYarnClient()
+  private[this] val yarnConf = new YarnConfiguration()
+  private[this] val yarnClient = YarnClient.createYarnClient()
   yarnClient.init(yarnConf)
   yarnClient.init(yarnConf)
   yarnClient.start()
   yarnClient.start()
 
 
-  def submitApplication(id: String,
-                        kind: String,
-                        proxyUser: Option[String],
-                        callbackUrl: String): Future[Job] = {
-    val url = f"$callbackUrl/sessions/$id/callback"
-
-    val builder = new SparkSubmitProcessBuilder()
-
-    builder.master("yarn-cluster")
-    builder.className("com.cloudera.hue.livy.repl.Main")
-    builder.driverJavaOptions(f"-Dlivy.repl.callback-url=$url -Dlivy.repl.port=0")
-    proxyUser.foreach(builder.proxyUser)
-
-    builder.redirectOutput(Redirect.PIPE)
-    builder.redirectErrorStream(redirect = true)
-
-    val process = builder.start(livyJar(livyConf), List(kind.toString))
+  def getJobFromProcess(process: Process): Job = {
+    val stdout = new BufferedReader(new InputStreamReader(process.getInputStream), 1)
 
 
-    Future {
-      val stdout = new BufferedReader(new InputStreamReader(process.getInputStream), 1)
+    val applicationId = parseApplicationId(stdout).getOrElse(throw new FailedToSubmitApplication)
 
 
-      val applicationId = parseApplicationId(stdout).getOrElse(throw new FailedToSubmitApplication)
+    // Application has been submitted, so we don't need to keep the process around anymore.
+    stdout.close()
+    process.destroy()
 
 
-      // Application has been submitted, so we don't need to keep the process around anymore.
-      stdout.close()
-      process.destroy()
-
-      new Job(yarnClient, ConverterUtils.toApplicationId(applicationId))
-    }
+    new Job(yarnClient, ConverterUtils.toApplicationId(applicationId))
   }
   }
 
 
   def close() = {
   def close() = {
@@ -168,11 +144,15 @@ class Job(yarnClient: YarnClient, appId: ApplicationId) {
     statusResponse.getRpcPort
     statusResponse.getRpcPort
   }
   }
 
 
-  private def getStatus: ApplicationStatus = {
+  def getStatus: ApplicationStatus = {
     val statusResponse = yarnClient.getApplicationReport(appId)
     val statusResponse = yarnClient.getApplicationReport(appId)
     convertState(statusResponse.getYarnApplicationState, statusResponse.getFinalApplicationStatus)
     convertState(statusResponse.getYarnApplicationState, statusResponse.getFinalApplicationStatus)
   }
   }
 
 
+  def stop(): Unit = {
+    yarnClient.killApplication(appId)
+  }
+
   private def convertState(state: YarnApplicationState, status: FinalApplicationStatus): ApplicationStatus = {
   private def convertState(state: YarnApplicationState, status: FinalApplicationStatus): ApplicationStatus = {
     (state, status) match {
     (state, status) match {
       case (YarnApplicationState.FINISHED, FinalApplicationStatus.SUCCEEDED) => SuccessfulFinish()
       case (YarnApplicationState.FINISHED, FinalApplicationStatus.SUCCEEDED) => SuccessfulFinish()
@@ -187,10 +167,3 @@ class Job(yarnClient: YarnClient, appId: ApplicationId) {
     }
     }
   }
   }
 }
 }
-
-trait ApplicationStatus
-case class New() extends ApplicationStatus
-case class Accepted() extends ApplicationStatus
-case class Running() extends ApplicationStatus
-case class SuccessfulFinish() extends ApplicationStatus
-case class UnsuccessfulFinish() extends ApplicationStatus