Browse Source

[livy] Make close synchronous, add timeouts, exponential backoff sleep helper

Erick Tryzelaar 10 years ago
parent
commit
59687221da

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

@@ -3,7 +3,10 @@ package com.cloudera.hue.livy
 import java.io.{FileInputStream, InputStreamReader, File}
 import java.io.{FileInputStream, InputStreamReader, File}
 import java.util.Properties
 import java.util.Properties
 
 
+import scala.annotation.tailrec
 import scala.collection.JavaConversions._
 import scala.collection.JavaConversions._
+import scala.concurrent.TimeoutException
+import scala.concurrent.duration.Duration
 
 
 object Utils {
 object Utils {
   def getPropertiesFromFile(filename: String): Map[String, String] = {
   def getPropertiesFromFile(filename: String): Map[String, String] = {
@@ -55,4 +58,37 @@ object Utils {
       None
       None
     }
     }
   }
   }
+
+  /**
+   * Checks if event has occurred during some time period. This performs an exponential backoff
+   * to limit the poll calls.
+   *
+   * @param checkForEvent
+   * @param atMost
+   * @throws java.util.concurrent.TimeoutException
+   * @throws java.lang.InterruptedException
+   * @return
+   */
+  @throws(classOf[TimeoutException])
+  @throws(classOf[InterruptedException])
+  final def waitUntil(checkForEvent: () => Boolean, atMost: Duration) = {
+    val endTime = System.currentTimeMillis() + atMost.toMillis
+
+    @tailrec
+    def aux(count: Int): Unit = {
+      if (!checkForEvent()) {
+        val now = System.currentTimeMillis()
+
+        if (now < endTime) {
+          val sleepTime = Math.max(10 * (2 << (count - 1)), 1000)
+          Thread.sleep(sleepTime)
+          aux(count + 1)
+        } else {
+          throw new TimeoutException
+        }
+      }
+    }
+
+    aux(1)
+  }
 }
 }

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

@@ -98,7 +98,7 @@ class ScalatraBootstrap extends LifeCycle with Logging {
 
 
   override def destroy(context: ServletContext): Unit = {
   override def destroy(context: ServletContext): Unit = {
     if (session != null) {
     if (session != null) {
-      Await.result(session.close(), Duration.Inf)
+      session.close()
     }
     }
   }
   }
 
 
@@ -106,7 +106,7 @@ class ScalatraBootstrap extends LifeCycle with Logging {
     info(s"Notifying $callbackUrl that we're up")
     info(s"Notifying $callbackUrl that we're up")
 
 
     Future {
     Future {
-      session.waitForStateChange(Session.Starting())
+      session.waitForStateChange(Session.Starting(), Duration(10, TimeUnit.SECONDS))
 
 
       // Wait for our url to be discovered.
       // Wait for our url to be discovered.
       val replUrl = waitForReplUrl()
       val replUrl = waitForReplUrl()

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

@@ -1,9 +1,11 @@
 package com.cloudera.hue.livy.repl
 package com.cloudera.hue.livy.repl
 
 
+import com.cloudera.hue.livy.Utils
 import org.json4s.JValue
 import org.json4s.JValue
 
 
 import _root_.scala.annotation.tailrec
 import _root_.scala.annotation.tailrec
-import _root_.scala.concurrent.Future
+import _root_.scala.concurrent.duration.Duration
+import _root_.scala.concurrent.{TimeoutException, Future}
 
 
 object Session {
 object Session {
   sealed trait State
   sealed trait State
@@ -27,13 +29,11 @@ trait Session {
 
 
   def history(id: Int): Option[JValue]
   def history(id: Int): Option[JValue]
 
 
-  def close(): Future[Unit]
+  def close(): Unit
 
 
-  @tailrec
-  final def waitForStateChange(oldState: State): Unit = {
-    if (state == oldState) {
-      Thread.sleep(1000)
-      waitForStateChange(oldState)
-    }
+  @throws(classOf[TimeoutException])
+  @throws(classOf[InterruptedException])
+  final def waitForStateChange(oldState: State, atMost: Duration) = {
+    Utils.waitUntil({ () => state != oldState }, atMost)
   }
   }
 }
 }

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

@@ -7,7 +7,7 @@ import org.json4s.{DefaultFormats, MappingException}
 import org.scalatra._
 import org.scalatra._
 import org.scalatra.json.JacksonJsonSupport
 import org.scalatra.json.JacksonJsonSupport
 
 
-import _root_.scala.concurrent.ExecutionContext
+import _root_.scala.concurrent.{Future, ExecutionContext}
 
 
 object WebApp extends Logging
 object WebApp extends Logging
 
 
@@ -58,7 +58,9 @@ class WebApp(session: Session) extends ScalatraServlet with FutureSupport with J
   }
   }
 
 
   delete("/") {
   delete("/") {
-    session.close().onComplete { _ =>
+    session.close()
+    Future {
+      Thread.sleep(1000)
       System.exit(0)
       System.exit(0)
     }
     }
   }
   }

+ 25 - 15
apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/python/PythonSession.scala

@@ -3,7 +3,8 @@ package com.cloudera.hue.livy.repl.python
 import java.io._
 import java.io._
 import java.lang.ProcessBuilder.Redirect
 import java.lang.ProcessBuilder.Redirect
 import java.nio.file.Files
 import java.nio.file.Files
-import java.util.concurrent.SynchronousQueue
+import java.util.concurrent.{TimeUnit, SynchronousQueue}
+
 
 
 import com.cloudera.hue.livy.Utils
 import com.cloudera.hue.livy.Utils
 import com.cloudera.hue.livy.repl.Session
 import com.cloudera.hue.livy.repl.Session
@@ -16,7 +17,8 @@ import py4j.GatewayServer
 import scala.annotation.tailrec
 import scala.annotation.tailrec
 import scala.collection.JavaConversions._
 import scala.collection.JavaConversions._
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.ArrayBuffer
-import scala.concurrent.{ExecutionContext, Future, Promise}
+import scala.concurrent.duration.Duration
+import scala.concurrent._
 
 
 object PythonSession {
 object PythonSession {
   def createPython(): Session = {
   def createPython(): Session = {
@@ -175,11 +177,18 @@ private class PythonSession(process: Process, gatewayServer: GatewayServer) exte
           require(state == Session.Idle() || state == Session.Error())
           require(state == Session.Idle() || state == Session.Error())
 
 
           _state = Session.ShuttingDown()
           _state = Session.ShuttingDown()
-          process.getInputStream.close()
-          process.getOutputStream.close()
-          process.destroy()
-          _state = Session.ShutDown()
-          promise.success(())
+
+          try {
+            process.getInputStream.close()
+            process.getOutputStream.close()
+
+            try {
+              process.destroy()
+            } finally {
+              _state = Session.ShutDown()
+              promise.success(())
+            }
+          }
       }
       }
     }
     }
   }
   }
@@ -204,22 +213,23 @@ private class PythonSession(process: Process, gatewayServer: GatewayServer) exte
     promise.future
     promise.future
   }
   }
 
 
-  override def close(): Future[Unit] = synchronized {
+  override def close(): Unit = synchronized {
     _state match {
     _state match {
       case Session.ShutDown() =>
       case Session.ShutDown() =>
-        Future.successful(())
       case Session.ShuttingDown() =>
       case Session.ShuttingDown() =>
-        Future {
-          waitForStateChange(Session.ShuttingDown())
-          Future.successful(())
-        }
+        // Another thread must be tearing down the process.
+        waitForStateChange(Session.ShuttingDown(), Duration(10, TimeUnit.SECONDS))
       case _ =>
       case _ =>
         val promise = Promise[Unit]()
         val promise = Promise[Unit]()
         queue.put(ShutdownRequest(promise))
         queue.put(ShutdownRequest(promise))
-        promise.future.map({ case () =>
+
+        // Give ourselves 10 seconds to tear down the process.
+        try {
+          Await.result(promise.future, Duration(10, TimeUnit.SECONDS))
           thread.join()
           thread.join()
+        } finally {
           gatewayServer.shutdown()
           gatewayServer.shutdown()
-        })
+        }
     }
     }
   }
   }
 }
 }

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

@@ -76,9 +76,7 @@ private class SparkSession extends Session {
     }
     }
   }
   }
 
 
-  override def close(): Future[Unit] = {
-    Future {
-      interpreter.shutdown()
-    }
+  override def close(): Unit = {
+    interpreter.shutdown()
   }
   }
 }
 }

+ 4 - 2
apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/PythonSessionSpec.scala

@@ -1,5 +1,7 @@
 package com.cloudera.hue.livy.repl
 package com.cloudera.hue.livy.repl
 
 
+import java.util.concurrent.TimeUnit
+
 import com.cloudera.hue.livy.repl.python.PythonSession
 import com.cloudera.hue.livy.repl.python.PythonSession
 import org.json4s.JsonAST.JValue
 import org.json4s.JsonAST.JValue
 import org.json4s.{Extraction, DefaultFormats}
 import org.json4s.{Extraction, DefaultFormats}
@@ -19,7 +21,7 @@ class PythonSessionSpec extends FunSpec with Matchers with BeforeAndAfter {
   }
   }
 
 
   after {
   after {
-    Await.result(session.close(), Duration.Inf)
+    session.close()
   }
   }
 
 
   describe("A python session") {
   describe("A python session") {
@@ -28,7 +30,7 @@ class PythonSessionSpec extends FunSpec with Matchers with BeforeAndAfter {
     }
     }
 
 
     it("should eventually become the idle state") {
     it("should eventually become the idle state") {
-      session.waitForStateChange(Session.Starting())
+      session.waitForStateChange(Session.Starting(), Duration(10, TimeUnit.SECONDS))
       session.state should equal (Session.Idle())
       session.state should equal (Session.Idle())
     }
     }
 
 

+ 4 - 2
apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/SparkSessionSpec.scala

@@ -1,5 +1,7 @@
 package com.cloudera.hue.livy.repl
 package com.cloudera.hue.livy.repl
 
 
+import java.util.concurrent.TimeUnit
+
 import com.cloudera.hue.livy.repl.scala.SparkSession
 import com.cloudera.hue.livy.repl.scala.SparkSession
 import org.json4s.JsonAST.JValue
 import org.json4s.JsonAST.JValue
 import org.json4s.{DefaultFormats, Extraction}
 import org.json4s.{DefaultFormats, Extraction}
@@ -19,7 +21,7 @@ class SparkSessionSpec extends FunSpec with Matchers with BeforeAndAfter {
   }
   }
 
 
   after {
   after {
-    Await.result(session.close(), Duration.Inf)
+    session.close()
   }
   }
 
 
   describe("A spark session") {
   describe("A spark session") {
@@ -28,7 +30,7 @@ class SparkSessionSpec extends FunSpec with Matchers with BeforeAndAfter {
     }
     }
 
 
     it("should eventually become the idle state") {
     it("should eventually become the idle state") {
-      session.waitForStateChange(Session.Starting())
+      session.waitForStateChange(Session.Starting(), Duration(10, TimeUnit.SECONDS))
       session.state should equal (Session.Idle())
       session.state should equal (Session.Idle())
     }
     }
 
 

+ 1 - 2
apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/WebAppSpec.scala

@@ -24,9 +24,8 @@ class WebAppSpec extends ScalatraSuite with FunSpecLike with BeforeAndAfter {
       Future.successful(rep)
       Future.successful(rep)
     }
     }
 
 
-    override def close(): Future[Unit] = {
+    override def close(): Unit = {
       _state = Session.ShuttingDown()
       _state = Session.ShuttingDown()
-      Future.successful(())
     }
     }
 
 
     override def history(): Seq[JValue] = _history
     override def history(): Seq[JValue] = _history

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

@@ -1,12 +1,14 @@
 package com.cloudera.hue.livy.server.sessions
 package com.cloudera.hue.livy.server.sessions
 
 
 import java.net.URL
 import java.net.URL
+import java.util.concurrent.TimeoutException
 
 
+import com.cloudera.hue.livy.Utils
 import com.cloudera.hue.livy.msgs.ExecuteRequest
 import com.cloudera.hue.livy.msgs.ExecuteRequest
 import com.cloudera.hue.livy.server.Statement
 import com.cloudera.hue.livy.server.Statement
 
 
-import scala.annotation.tailrec
-import scala.concurrent.Future
+import scala.concurrent._
+import scala.concurrent.duration.Duration
 
 
 object Session {
 object Session {
   sealed trait State
   sealed trait State
@@ -47,14 +49,10 @@ trait Session {
 
 
   def stop(): Future[Unit]
   def stop(): Future[Unit]
 
 
-  @tailrec
-  final def waitForStateChange[A](oldState: State, f: => A): A = {
-    if (state == oldState) {
-      Thread.sleep(1000)
-      waitForStateChange(oldState, f)
-    } else {
-      f
-    }
+  @throws(classOf[TimeoutException])
+  @throws(classOf[InterruptedException])
+  final def waitForStateChange(oldState: State, atMost: Duration) = {
+    Utils.waitUntil({ () => state != oldState }, atMost)
   }
   }
 }
 }
 
 

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

@@ -10,7 +10,7 @@ import com.cloudera.hue.livy.server.Statement
 import com.cloudera.hue.livy.server.sessions.Session._
 import com.cloudera.hue.livy.server.sessions.Session._
 
 
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.ArrayBuffer
-import scala.concurrent.Future
+import scala.concurrent.{ExecutionContext, ExecutionContextExecutor, Future}
 
 
 object ThreadSession {
 object ThreadSession {
   val LIVY_HOME = System.getenv("LIVY_HOME")
   val LIVY_HOME = System.getenv("LIVY_HOME")
@@ -28,6 +28,9 @@ object ThreadSession {
 }
 }
 
 
 private class ThreadSession(val id: String, session: com.cloudera.hue.livy.repl.Session) extends Session {
 private class ThreadSession(val id: String, session: com.cloudera.hue.livy.repl.Session) extends Session {
+
+  protected implicit def executor: ExecutionContextExecutor = ExecutionContext.global
+
   private var executedStatements = 0
   private var executedStatements = 0
   private var statements_ = new ArrayBuffer[Statement]
   private var statements_ = new ArrayBuffer[Statement]
 
 
@@ -68,5 +71,7 @@ private class ThreadSession(val id: String, session: com.cloudera.hue.livy.repl.
     stop()
     stop()
   }
   }
 
 
-  override def stop(): Future[Unit] = session.close()
+  override def stop(): Future[Unit] = Future {
+    session.close()
+  }
 }
 }

+ 8 - 3
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/sessions/WebSession.scala

@@ -1,6 +1,7 @@
 package com.cloudera.hue.livy.server.sessions
 package com.cloudera.hue.livy.server.sessions
 
 
 import java.net.URL
 import java.net.URL
+import java.util.concurrent.TimeUnit
 
 
 import com.cloudera.hue.livy._
 import com.cloudera.hue.livy._
 import com.cloudera.hue.livy.msgs.ExecuteRequest
 import com.cloudera.hue.livy.msgs.ExecuteRequest
@@ -12,6 +13,7 @@ import org.json4s.{DefaultFormats, Formats}
 
 
 import scala.annotation.tailrec
 import scala.annotation.tailrec
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.duration.Duration
 import scala.concurrent.{Future, _}
 import scala.concurrent.{Future, _}
 
 
 class WebSession(val id: String) extends Session with Logging {
 class WebSession(val id: String) extends Session with Logging {
@@ -96,15 +98,18 @@ class WebSession(val id: String) extends Session with Logging {
           }
           }
         case NotStarted() =>
         case NotStarted() =>
           Future {
           Future {
-            waitForStateChange(NotStarted(), { stop() })
+            waitForStateChange(NotStarted(), Duration(10, TimeUnit.SECONDS))
+            stop()
           }
           }
         case Starting() =>
         case Starting() =>
           Future {
           Future {
-            waitForStateChange(Starting(), { stop() })
+            waitForStateChange(Starting(), Duration(10, TimeUnit.SECONDS))
+            stop()
           }
           }
         case Busy() =>
         case Busy() =>
           Future {
           Future {
-            waitForStateChange(Busy(), { stop() })
+            waitForStateChange(Busy(), Duration(10, TimeUnit.SECONDS))
+            stop()
           }
           }
         case Error() | Dead() =>
         case Error() | Dead() =>
           Future.successful(Unit)
           Future.successful(Unit)

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

@@ -0,0 +1,73 @@
+package com.cloudera.hue.livy.server
+
+import java.util.concurrent.TimeUnit
+
+import com.cloudera.hue.livy.msgs.ExecuteRequest
+import com.cloudera.hue.livy.server.sessions.Session
+import org.json4s.{DefaultFormats, Extraction}
+import org.scalatest.matchers.ShouldMatchers
+import org.scalatest.{BeforeAndAfter, FunSpec}
+
+import scala.concurrent.Await
+import scala.concurrent.duration.Duration
+
+abstract class BaseSessionSpec extends FunSpec with ShouldMatchers with BeforeAndAfter {
+
+  implicit val formats = DefaultFormats
+
+  var session: Session = null
+
+  def createSession(): Session
+
+  before {
+    session = createSession()
+  }
+
+  after {
+    session.stop()
+  }
+
+  describe("A spark session") {
+    it("should start in the starting or idle state") {
+      session.state should (equal (Session.Starting()) or equal (Session.Idle()))
+    }
+
+    it("should eventually become the idle state") {
+      session.waitForStateChange(Session.Starting(), Duration(30, TimeUnit.SECONDS))
+      session.state should equal (Session.Idle())
+    }
+
+    it("should execute `1 + 2` == 3") {
+      session.waitForStateChange(Session.Starting(), Duration(30, TimeUnit.SECONDS))
+      val stmt = session.executeStatement(ExecuteRequest("1 + 2"))
+      val result = Await.result(stmt.output, Duration.Inf)
+
+      val expectedResult = Extraction.decompose(Map(
+        "status" -> "ok",
+        "execution_count" -> 0,
+        "data" -> Map(
+          "text/plain" -> "res0: Int = 3"
+        )
+      ))
+
+      result should equal (expectedResult)
+    }
+
+    it("should report an error if accessing an unknown variable") {
+      session.waitForStateChange(Session.Starting(), Duration(30, TimeUnit.SECONDS))
+      val stmt = session.executeStatement(ExecuteRequest("x"))
+      val result = Await.result(stmt.output, Duration.Inf)
+      val expectedResult = Extraction.decompose(Map(
+        "status" -> "error",
+        "execution_count" -> 0,
+        "ename" -> "Error",
+        "evalue" ->
+          """<console>:8: error: not found: value x
+            |              x
+            |              ^""".stripMargin
+      ))
+
+      result should equal (expectedResult)
+    }
+  }
+}

+ 4 - 63
apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/ThreadSessionSpec.scala

@@ -1,68 +1,9 @@
 package com.cloudera.hue.livy.server
 package com.cloudera.hue.livy.server
 
 
-import com.cloudera.hue.livy.msgs.ExecuteRequest
-import com.cloudera.hue.livy.server.sessions.{Session, ThreadSession}
-import org.json4s.{DefaultFormats, Extraction}
-import org.scalatest.{BeforeAndAfter, FunSpec, Matchers}
+import com.cloudera.hue.livy.server.sessions.ThreadSession
+import org.scalatest.{BeforeAndAfter, FunSpecLike, Matchers}
 
 
-import scala.concurrent.Await
-import scala.concurrent.duration.Duration
+class ThreadSessionSpec extends BaseSessionSpec with FunSpecLike with Matchers with BeforeAndAfter {
 
 
-class ThreadSessionSpec extends FunSpec with Matchers with BeforeAndAfter {
-
-  implicit val formats = DefaultFormats
-
-  var session: Session = null
-
-  before {
-    session = ThreadSession.create("0", "spark")
-  }
-
-  after {
-    session.stop()
-  }
-
-  describe("A spark session") {
-      it("should start in the starting or idle state") {
-        session.state should (equal (Session.Starting()) or equal (Session.Idle()))
-      }
-
-      it("should eventually become the idle state") {
-        session.waitForStateChange(Session.Starting(), { })
-        session.state should equal (Session.Idle())
-      }
-
-      it("should execute `1 + 2` == 3") {
-        session.waitForStateChange(Session.Starting(), { })
-        val stmt = session.executeStatement(ExecuteRequest("1 + 2"))
-        val result = Await.result(stmt.output, Duration.Inf)
-
-        val expectedResult = Extraction.decompose(Map(
-          "status" -> "ok",
-          "execution_count" -> 0,
-          "data" -> Map(
-            "text/plain" -> "res0: Int = 3"
-          )
-        ))
-
-        result should equal (expectedResult)
-      }
-
-      it("should report an error if accessing an unknown variable") {
-        session.waitForStateChange(Session.Starting(), { })
-        val stmt = session.executeStatement(ExecuteRequest("x"))
-        val result = Await.result(stmt.output, Duration.Inf)
-        val expectedResult = Extraction.decompose(Map(
-          "status" -> "error",
-          "execution_count" -> 0,
-          "ename" -> "Error",
-          "evalue" ->
-            """<console>:8: error: not found: value x
-              |              x
-              |              ^""".stripMargin
-        ))
-
-        result should equal (expectedResult)
-      }
-  }
+  def createSession() = ThreadSession.create("0", "spark")
 }
 }