浏览代码

[livy] Time out sessions and batches that have been done for a while

This closes #223.
Erick Tryzelaar 10 年之前
父节点
当前提交
a4deeae93b
共有 15 个文件被更改,包括 154 次插入70 次删除
  1. 1 6
      apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/LineBufferedProcess.scala
  2. 12 0
      apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/Utils.scala
  3. 7 7
      apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/sessions/State.scala
  4. 2 2
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/Session.scala
  5. 2 12
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/process/ProcessInterpreter.scala
  6. 10 1
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/Session.scala
  7. 18 9
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/SessionManager.scala
  8. 28 20
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchSessionProcess.scala
  9. 6 4
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchSessionYarn.scala
  10. 1 1
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionProcess.scala
  11. 1 1
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveWebSession.scala
  12. 52 0
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/SessionManagerSpec.scala
  13. 5 3
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/batches/BatchProcessSpec.scala
  14. 5 3
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/batches/BatchServletSpec.scala
  15. 4 1
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/interactive/BaseSessionSpec.scala

+ 1 - 6
apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/LineBufferedProcess.scala

@@ -34,12 +34,7 @@ class LineBufferedProcess(process: Process) extends Logging {
   }
 
   /** Returns if the process is still actively running. */
-  def isAlive: Boolean = try {
-    exitValue()
-    false
-  } catch {
-    case _: IllegalStateException => true
-  }
+  def isAlive: Boolean = Utils.isProcessAlive(process)
 
   def exitValue(): Int = {
     process.exitValue()

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

@@ -109,4 +109,16 @@ object Utils {
 
     aux(1)
   }
+
+  /** Returns if the process is still running */
+  def isProcessAlive(process: Process): Boolean = {
+    try {
+      process.exitValue()
+      false
+    } catch {
+      case _: IllegalThreadStateException =>
+        true
+    }
+  }
+
 }

+ 7 - 7
apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/sessions/State.scala

@@ -48,22 +48,22 @@ case class Busy() extends State {
   override def toString = "busy"
 }
 
-case class Error() extends State {
-  override def isActive = true
-  override def toString = "error"
-}
-
 case class ShuttingDown() extends State {
   override def isActive = false
   override def toString = "shutting_down"
 }
 
-case class Dead() extends State {
+case class Error(time: Long = System.currentTimeMillis()) extends State {
+  override def isActive = true
+  override def toString = "error"
+}
+
+case class Dead(time: Long = System.currentTimeMillis()) extends State {
   override def isActive = false
   override def toString = "dead"
 }
 
-case class Success() extends State {
+case class Success(time: Long = System.currentTimeMillis()) extends State {
   override def isActive = false
   override def toString = "success"
 }

+ 2 - 2
apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/Session.scala

@@ -57,7 +57,7 @@ class Session(interpreter: Interpreter)
     interpreter.start()
     _state = Idle()
   }.onFailure { case _ =>
-    _state = Error()
+    _state = Error(System.currentTimeMillis())
   }
 
   def kind: String = interpreter.kind
@@ -117,7 +117,7 @@ class Session(interpreter: Interpreter)
           (EVALUE -> evalue) ~
           (TRACEBACK -> traceback)
         case Interpreter.ExecuteAborted(message) =>
-          _state = Error()
+          _state = Error(System.currentTimeMillis())
 
           (STATUS -> ERROR) ~
           (EXECUTION_COUNT -> executionCount) ~

+ 2 - 12
apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/process/ProcessInterpreter.scala

@@ -21,7 +21,7 @@ package com.cloudera.hue.livy.repl.process
 import java.io.{BufferedReader, IOException, InputStreamReader, PrintWriter}
 import java.util.concurrent.locks.ReentrantLock
 
-import com.cloudera.hue.livy.Logging
+import com.cloudera.hue.livy.{Utils, Logging}
 import com.cloudera.hue.livy.repl.Interpreter
 import org.json4s.JValue
 
@@ -60,7 +60,7 @@ abstract class ProcessInterpreter(process: Process)
   }
 
   override def close(): Unit = {
-    if (isProcessAlive) {
+    if (Utils.isProcessAlive(process)) {
       logger.info("Shutting down process")
       sendShutdownRequest()
 
@@ -79,16 +79,6 @@ abstract class ProcessInterpreter(process: Process)
     }
   }
 
-  private def isProcessAlive = {
-    try {
-      process.exitValue()
-      false
-    } catch {
-      case _: IllegalThreadStateException =>
-        true
-    }
-  }
-
   protected def sendExecuteRequest(request: String): Interpreter.ExecuteResponse
 
   protected def sendShutdownRequest(): Unit = {}

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

@@ -18,7 +18,7 @@
 
 package com.cloudera.hue.livy.server
 
-import com.cloudera.hue.livy.sessions.State
+import com.cloudera.hue.livy.sessions.{Dead, Error, Success, State}
 
 import scala.concurrent.Future
 
@@ -27,6 +27,15 @@ trait Session {
 
   def lastActivity: Option[Long] = None
 
+  def stoppedTime: Option[Long] = {
+    state match {
+      case Error(time) => Some(time)
+      case Dead(time) => Some(time)
+      case Success(time) => Some(time)
+      case _ => None
+    }
+  }
+
   def state: State
 
   def stop(): Future[Unit]

+ 18 - 9
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/SessionManager.scala

@@ -26,16 +26,22 @@ import org.json4s.JValue
 import scala.collection.mutable
 import scala.concurrent.{ExecutionContext, Future}
 
+object SessionManager {
+  val LIVY_SERVER_SESSION_TIMEOUT = "livy.server.session.timeout"
+}
+
 class SessionManager[S <: Session](livyConf: LivyConf, factory: SessionFactory[S])
   extends Logging {
 
+  import SessionManager._
+
   private implicit def executor: ExecutionContext = ExecutionContext.global
 
-  private[this] val _idCounter = new AtomicInteger()
-  private[this] val _sessions = mutable.Map[Int, S]()
+  private[this] final val _idCounter = new AtomicInteger()
+  private[this] final val _sessions = mutable.Map[Int, S]()
 
-  private[this] val sessionTimeout = livyConf.getInt("livy.server.session.timeout", 1000 * 60 * 60)
-  private[this] val garbageCollector = new GarbageCollector
+  private[this] final val sessionTimeout = livyConf.getInt(LIVY_SERVER_SESSION_TIMEOUT, 1000 * 60 * 60)
+  private[this] final val garbageCollector = new GarbageCollector
   garbageCollector.setDaemon(true)
   garbageCollector.start()
 
@@ -73,15 +79,18 @@ class SessionManager[S <: Session](livyConf: LivyConf, factory: SessionFactory[S
 
   def shutdown(): Unit = {}
 
-  def collectGarbage() = {
+  def collectGarbage(): Future[Iterable[Unit]] = {
     def expired(session: Session): Boolean = {
-      session.lastActivity match {
-        case Some(lastActivity) => System.currentTimeMillis() - lastActivity > sessionTimeout
-        case None => false
+      session.lastActivity.orElse(session.stoppedTime) match {
+        case Some(lastActivity) =>
+          val currentTime = System.currentTimeMillis()
+          currentTime - lastActivity > sessionTimeout
+        case None =>
+          false
       }
     }
 
-    all().filter(expired).foreach(delete)
+    Future.sequence(all().filter(expired).map(delete))
   }
 
   private class GarbageCollector extends Thread("session gc thread") {

+ 28 - 20
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchSessionProcess.scala

@@ -20,9 +20,9 @@ package com.cloudera.hue.livy.server.batch
 
 import java.lang.ProcessBuilder.Redirect
 
-import com.cloudera.hue.livy.sessions.{Success, Running, State}
+import com.cloudera.hue.livy.sessions.{Error, Success, Running, State}
 import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder.RelativePath
-import com.cloudera.hue.livy.{LivyConf, LineBufferedProcess}
+import com.cloudera.hue.livy.{Utils, LivyConf, LineBufferedProcess}
 import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder
 
 import scala.concurrent.{Future, ExecutionContext, ExecutionContextExecutor}
@@ -58,24 +58,12 @@ object BatchSessionProcess {
 }
 
 private class BatchSessionProcess(val id: Int,
-                           process: LineBufferedProcess) extends BatchSession {
+                                  process: LineBufferedProcess) extends BatchSession {
   protected implicit def executor: ExecutionContextExecutor = ExecutionContext.global
 
-  private[this] var isAlive = true
+  private[this] var _state: State = Running()
 
-  override def state: State = {
-    if (isAlive) {
-      try {
-        process.exitValue()
-      } catch {
-        case e: IllegalThreadStateException => return Running()
-      }
-
-      destroyProcess()
-    }
-
-    Success()
-  }
+  override def state: State = _state
 
   override def logLines(): IndexedSeq[String] = process.inputLines
 
@@ -86,8 +74,28 @@ private class BatchSessionProcess(val id: Int,
   }
 
   private def destroyProcess() = {
-    process.destroy()
-    process.waitFor()
-    isAlive = false
+    if (process.isAlive) {
+      process.destroy()
+      reapProcess(process.waitFor())
+    }
+  }
+
+  private def reapProcess(exitCode: Int) = synchronized {
+    if (_state.isActive) {
+      if (exitCode == 0) {
+        _state = Success()
+      } else {
+        _state = Error()
+      }
+    }
+  }
+
+  /** Simple daemon thread to make sure we change state when the process exits. */
+  private[this] val thread = new Thread("Batch Process Reaper") {
+    override def run(): Unit = {
+      reapProcess(process.waitFor())
+    }
   }
+  thread.setDaemon(true)
+  thread.start()
 }

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

@@ -78,7 +78,9 @@ private class BatchSessionYarn(val id: Int, process: LineBufferedProcess, jobFut
   private var _jobThread: Thread = _
 
   jobFuture.onComplete {
-    case util.Failure(_) => _state = Error()
+    case util.Failure(_) =>
+      _state = Error(System.currentTimeMillis())
+
     case util.Success(job) =>
       _state = Running()
 
@@ -90,9 +92,9 @@ private class BatchSessionYarn(val id: Int, process: LineBufferedProcess, jobFut
               Thread.sleep(5000)
               job.getStatus match {
                 case Client.SuccessfulFinish() =>
-                  _state = Success()
+                  _state = Success(System.currentTimeMillis())
                 case Client.UnsuccessfulFinish() =>
-                  _state = Error()
+                  _state = Error(System.currentTimeMillis())
                 case _ => aux()
               }
             }
@@ -110,7 +112,7 @@ private class BatchSessionYarn(val id: Int, process: LineBufferedProcess, jobFut
   override def stop(): Future[Unit] = {
     jobFuture.map { job =>
       job.stop()
-      _state = Success()
+      _state = Success(System.currentTimeMillis())
       ()
     }
   }

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

@@ -126,7 +126,7 @@ private class InteractiveSessionProcess(id: Int,
     } else {
       // Set the state to done if the session shut down before contacting us.
       _state match {
-        case (Dead() | Error() | Success()) =>
+        case (Dead(_) | Error(_) | Success(_)) =>
         case _ =>
           _state = Success()
       }

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

@@ -181,7 +181,7 @@ abstract class InteractiveWebSession(val id: Int, createInteractiveRequest: Crea
             waitForStateChange(ShuttingDown(), Duration(10, TimeUnit.SECONDS))
             stop()
           }
-        case Error() | Dead() | Success() =>
+        case Error(_) | Dead(_) | Success(_) =>
           Future.successful(Unit)
       }
     }

+ 52 - 0
apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/SessionManagerSpec.scala

@@ -0,0 +1,52 @@
+/*
+ * 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
+
+import com.cloudera.hue.livy.LivyConf
+import com.cloudera.hue.livy.sessions.{State, Success}
+import org.json4s.JsonAST.{JNothing, JValue}
+import org.scalatest.{FlatSpec, Matchers}
+
+import scala.concurrent.duration.Duration
+import scala.concurrent.{Await, Future}
+
+class SessionManagerSpec extends FlatSpec with Matchers {
+
+  class MockSession(val id: Int) extends Session {
+    override def stop(): Future[Unit] = Future.successful(())
+
+    override def logLines(): IndexedSeq[String] = IndexedSeq()
+
+    override def state: State = Success(0)
+  }
+
+  class MockSessionFactory extends SessionFactory[MockSession] {
+    override def create(id: Int, createRequest: JValue): MockSession = new MockSession(id)
+  }
+
+  it should "garbage collect old sessions" in {
+    val livyConf = new LivyConf()
+    livyConf.set(SessionManager.LIVY_SERVER_SESSION_TIMEOUT, "100")
+    val manager = new SessionManager(livyConf, new MockSessionFactory)
+    val session = manager.create(JNothing)
+    manager.get(session.id).isDefined should be(true)
+    Await.result(manager.collectGarbage(), Duration.Inf)
+    manager.get(session.id).isEmpty should be(true)
+  }
+}

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

@@ -56,9 +56,11 @@ class BatchProcessSpec
       )
       val batch = BatchSessionProcess(new LivyConf(), 0, req)
 
-      Utils.waitUntil({ () =>
-        batch.state == Success()
-      }, Duration(10, TimeUnit.SECONDS))
+      Utils.waitUntil({ () => !batch.state.isActive }, Duration(10, TimeUnit.SECONDS))
+      (batch.state match {
+        case Success(_) => true
+        case _ => false
+      }) should be (true)
 
       batch.logLines() should contain("hello world")
     }

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

@@ -92,9 +92,11 @@ class BatchServletSpec extends ScalatraSuite with FunSpecLike with BeforeAndAfte
       // Wait for the process to finish.
       {
         val batch: BatchSession = batchManager.get(0).get
-        Utils.waitUntil({ () =>
-          batch.state == Success()
-        }, Duration(10, TimeUnit.SECONDS))
+        Utils.waitUntil({ () => !batch.state.isActive }, Duration(10, TimeUnit.SECONDS))
+        (batch.state match {
+          case Success(_) => true
+          case _ => false
+        }) should be (true)
       }
 
       get("/0") {

+ 4 - 1
apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/interactive/BaseSessionSpec.scala

@@ -93,7 +93,10 @@ abstract class BaseSessionSpec extends FunSpec with Matchers with BeforeAndAfter
       session.waitForStateChange(Starting(), Duration(30, TimeUnit.SECONDS))
       val stmt = session.executeStatement(ExecuteRequest("import os; os._exit(1)"))
       val result = Await.result(stmt.output(), Duration.Inf)
-      session.state should equal (Error())
+      (session.state match {
+        case Error(_) => true
+        case _ => false
+      }) should equal (true)
     }
   }
 }