Browse Source

[livy] Move session state variants into it's own namespace

Erick Tryzelaar 10 năm trước cách đây
mục cha
commit
38eb953
19 tập tin đã thay đổi với 173 bổ sung166 xóa
  1. 81 0
      apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/sessions/SessionState.scala
  2. 0 69
      apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/sessions/State.scala
  3. 2 2
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/Main.scala
  4. 11 11
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/Session.scala
  5. 2 2
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/WebApp.scala
  6. 5 6
      apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/BaseSessionSpec.scala
  7. 5 5
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/Session.scala
  8. 5 6
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchSessionProcess.scala
  9. 8 9
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchSessionYarn.scala
  10. 2 2
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSession.scala
  11. 4 4
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionProcess.scala
  12. 4 4
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionServlet.scala
  13. 4 4
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionYarn.scala
  14. 23 23
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveWebSession.scala
  15. 2 2
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/SessionManagerSpec.scala
  16. 2 3
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/batch/BatchProcessSpec.scala
  17. 3 4
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/batch/BatchServletSpec.scala
  18. 9 9
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/interactive/BaseSessionSpec.scala
  19. 1 1
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionServletSpec.scala

+ 81 - 0
apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/sessions/SessionState.scala

@@ -0,0 +1,81 @@
+/*
+ * 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.sessions
+
+sealed trait SessionState {
+  /** Returns true if the State represents a process that can eventually execute commands */
+  def isActive: Boolean
+}
+
+object SessionState {
+
+  case class NotStarted() extends SessionState {
+    override def isActive = true
+
+    override def toString = "not_started"
+  }
+
+  case class Starting() extends SessionState {
+    override def isActive = true
+
+    override def toString = "starting"
+  }
+
+  case class Idle() extends SessionState {
+    override def isActive = true
+
+    override def toString = "idle"
+  }
+
+  case class Running() extends SessionState {
+    override def isActive = true
+
+    override def toString = "running"
+  }
+
+  case class Busy() extends SessionState {
+    override def isActive = true
+
+    override def toString = "busy"
+  }
+
+  case class ShuttingDown() extends SessionState {
+    override def isActive = false
+
+    override def toString = "shutting_down"
+  }
+
+  case class Error(time: Long = System.currentTimeMillis()) extends SessionState {
+    override def isActive = true
+
+    override def toString = "error"
+  }
+
+  case class Dead(time: Long = System.currentTimeMillis()) extends SessionState {
+    override def isActive = false
+
+    override def toString = "dead"
+  }
+
+  case class Success(time: Long = System.currentTimeMillis()) extends SessionState {
+    override def isActive = false
+
+    override def toString = "success"
+  }
+}

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

@@ -1,69 +0,0 @@
-/*
- * 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.sessions
-
-sealed trait State {
-  /** Returns true if the State represents a process that can eventually execute commands */
-  def isActive: Boolean
-}
-
-case class NotStarted() extends State {
-  override def isActive = true
-  override def toString = "not_started"
-}
-
-case class Starting() extends State {
-  override def isActive = true
-  override def toString = "starting"
-}
-
-case class Idle() extends State {
-  override def isActive = true
-  override def toString = "idle"
-}
-
-case class Running() extends State {
-  override def isActive = true
-  override def toString = "running"
-}
-
-case class Busy() extends State {
-  override def isActive = true
-  override def toString = "busy"
-}
-
-case class ShuttingDown() extends State {
-  override def isActive = false
-  override def toString = "shutting_down"
-}
-
-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(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/Main.scala

@@ -24,7 +24,7 @@ import javax.servlet.ServletContext
 import com.cloudera.hue.livy.repl.python.PythonInterpreter
 import com.cloudera.hue.livy.repl.scala.SparkInterpreter
 import com.cloudera.hue.livy.repl.sparkr.SparkRInterpreter
-import com.cloudera.hue.livy.sessions.Starting
+import com.cloudera.hue.livy.sessions.SessionState
 import com.cloudera.hue.livy.{LivyConf, Logging, WebServer}
 import dispatch._
 import org.json4s.jackson.Serialization.write
@@ -134,7 +134,7 @@ class ScalatraBootstrap extends LifeCycle with Logging {
     info(s"Notifying $callbackUrl that we're up")
 
     Future {
-      session.waitForStateChange(Starting(), Duration(30, TimeUnit.SECONDS))
+      session.waitForStateChange(SessionState.Starting(), Duration(30, TimeUnit.SECONDS))
 
       // Wait for our url to be discovered.
       val replUrl = waitForReplUrl()

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

@@ -49,15 +49,15 @@ class Session(interpreter: Interpreter)
   private implicit val executor = ExecutionContext.fromExecutorService(Executors.newSingleThreadExecutor())
   private implicit val formats = DefaultFormats
 
-  private var _state: State = NotStarted()
+  private var _state: SessionState = SessionState.NotStarted()
   private var _history = IndexedSeq[Statement]()
 
   Future {
-    _state = Starting()
+    _state = SessionState.Starting()
     interpreter.start()
-    _state = Idle()
+    _state = SessionState.Idle()
   }.onFailure { case _ =>
-    _state = Error(System.currentTimeMillis())
+    _state = SessionState.Error(System.currentTimeMillis())
   }
 
   def kind: String = interpreter.kind
@@ -84,24 +84,24 @@ class Session(interpreter: Interpreter)
 
   @throws(classOf[TimeoutException])
   @throws(classOf[InterruptedException])
-  def waitForStateChange(oldState: State, atMost: Duration) = {
+  def waitForStateChange(oldState: SessionState, atMost: Duration) = {
     Utils.waitUntil({ () => state != oldState }, atMost)
   }
 
   private def executeCode(executionCount: Int, code: String) = {
-    _state = Busy()
+    _state = SessionState.Busy()
 
     try {
 
       interpreter.execute(code) match {
         case Interpreter.ExecuteSuccess(data) =>
-          _state = Idle()
+          _state = SessionState.Idle()
 
           (STATUS -> OK) ~
           (EXECUTION_COUNT -> executionCount) ~
           (DATA -> data)
         case Interpreter.ExecuteIncomplete() =>
-          _state = Idle()
+          _state = SessionState.Idle()
 
           (STATUS -> ERROR) ~
           (EXECUTION_COUNT -> executionCount) ~
@@ -109,7 +109,7 @@ class Session(interpreter: Interpreter)
           (EVALUE -> "incomplete statement") ~
           (TRACEBACK -> List())
         case Interpreter.ExecuteError(ename, evalue, traceback) =>
-          _state = Idle()
+          _state = SessionState.Idle()
 
           (STATUS -> ERROR) ~
           (EXECUTION_COUNT -> executionCount) ~
@@ -117,7 +117,7 @@ class Session(interpreter: Interpreter)
           (EVALUE -> evalue) ~
           (TRACEBACK -> traceback)
         case Interpreter.ExecuteAborted(message) =>
-          _state = Error(System.currentTimeMillis())
+          _state = SessionState.Error(System.currentTimeMillis())
 
           (STATUS -> ERROR) ~
           (EXECUTION_COUNT -> executionCount) ~
@@ -129,7 +129,7 @@ class Session(interpreter: Interpreter)
       case e: Throwable =>
         error("Exception when executing code", e)
 
-        _state = Idle()
+        _state = SessionState.Idle()
 
 
         (STATUS -> ERROR) ~

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

@@ -42,8 +42,8 @@ class WebApp(session: Session) extends ScalatraServlet with FutureSupport with J
     contentType = formats("json")
 
     session.state match {
-      case ShuttingDown() => halt(500, "Shutting down")
-      case _ => {}
+      case SessionState.ShuttingDown() => halt(500, "Shutting down")
+      case _ =>
     }
   }
 

+ 5 - 6
apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/BaseSessionSpec.scala

@@ -19,8 +19,7 @@
 package com.cloudera.hue.livy.repl
 
 import java.util.concurrent.TimeUnit
-
-import com.cloudera.hue.livy.sessions.{NotStarted, Idle, Starting}
+import com.cloudera.hue.livy.sessions.SessionState
 import org.json4s.DefaultFormats
 import org.scalatest.{FlatSpec, Matchers}
 
@@ -32,7 +31,7 @@ abstract class BaseSessionSpec extends FlatSpec with Matchers {
 
   def withSession(testCode: Session => Any) = {
     val session = Session(createInterpreter())
-    session.waitForStateChange(NotStarted(), Duration(30, TimeUnit.SECONDS))
+    session.waitForStateChange(SessionState.NotStarted(), Duration(30, TimeUnit.SECONDS))
     try {
       testCode(session)
     } finally session.close()
@@ -41,11 +40,11 @@ abstract class BaseSessionSpec extends FlatSpec with Matchers {
   def createInterpreter(): Interpreter
 
   it should "start in the starting or idle state" in withSession { session =>
-    session.state should (equal (Starting()) or equal (Idle()))
+    session.state should (equal (SessionState.Starting()) or equal (SessionState.Idle()))
   }
 
   it should "eventually become the idle state" in withSession { session =>
-    session.waitForStateChange(Starting(), Duration(30, TimeUnit.SECONDS))
-    session.state should equal (Idle())
+    session.waitForStateChange(SessionState.Starting(), Duration(30, TimeUnit.SECONDS))
+    session.state should equal (SessionState.Idle())
   }
 }

+ 5 - 5
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.{Dead, Error, Success, State}
+import com.cloudera.hue.livy.sessions.SessionState
 
 import scala.concurrent.Future
 
@@ -29,14 +29,14 @@ trait Session {
 
   def stoppedTime: Option[Long] = {
     state match {
-      case Error(time) => Some(time)
-      case Dead(time) => Some(time)
-      case Success(time) => Some(time)
+      case SessionState.Error(time) => Some(time)
+      case SessionState.Dead(time) => Some(time)
+      case SessionState.Success(time) => Some(time)
       case _ => None
     }
   }
 
-  def state: State
+  def state: SessionState
 
   def stop(): Future[Unit]
 

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

@@ -19,8 +19,7 @@
 package com.cloudera.hue.livy.server.batch
 
 import java.lang.ProcessBuilder.Redirect
-
-import com.cloudera.hue.livy.sessions.{Error, Success, Running, State}
+import com.cloudera.hue.livy.sessions._
 import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder.RelativePath
 import com.cloudera.hue.livy.{Utils, LivyConf, LineBufferedProcess}
 import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder
@@ -63,9 +62,9 @@ private class BatchSessionProcess(val id: Int,
                                   process: LineBufferedProcess) extends BatchSession {
   protected implicit def executor: ExecutionContextExecutor = ExecutionContext.global
 
-  private[this] var _state: State = Running()
+  private[this] var _state: SessionState = SessionState.Running()
 
-  override def state: State = _state
+  override def state: SessionState = _state
 
   override def logLines(): IndexedSeq[String] = process.inputLines
 
@@ -85,9 +84,9 @@ private class BatchSessionProcess(val id: Int,
   private def reapProcess(exitCode: Int) = synchronized {
     if (_state.isActive) {
       if (exitCode == 0) {
-        _state = Success()
+        _state = SessionState.Success()
       } else {
-        _state = Error()
+        _state = SessionState.Error()
       }
     }
   }

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

@@ -19,7 +19,6 @@
 package com.cloudera.hue.livy.server.batch
 
 import java.lang.ProcessBuilder.Redirect
-
 import com.cloudera.hue.livy.sessions._
 import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder.RelativePath
 import com.cloudera.hue.livy.{LineBufferedProcess, LivyConf}
@@ -75,28 +74,28 @@ private class BatchSessionYarn(val id: Int, process: LineBufferedProcess, jobFut
 
   implicit def executor: ExecutionContextExecutor = ExecutionContext.global
 
-  private var _state: State = Starting()
+  private var _state: SessionState = SessionState.Starting()
 
   private var _jobThread: Thread = _
 
   jobFuture.onComplete {
     case util.Failure(_) =>
-      _state = Error(System.currentTimeMillis())
+      _state = SessionState.Error(System.currentTimeMillis())
 
     case util.Success(job) =>
-      _state = Running()
+      _state = SessionState.Running()
 
       _jobThread = new Thread {
         override def run(): Unit = {
           @tailrec
           def aux(): Unit = {
-            if (_state == Running()) {
+            if (_state == SessionState.Running()) {
               Thread.sleep(5000)
               job.getStatus match {
                 case Client.SuccessfulFinish() =>
-                  _state = Success(System.currentTimeMillis())
+                  _state = SessionState.Success(System.currentTimeMillis())
                 case Client.UnsuccessfulFinish() =>
-                  _state = Error(System.currentTimeMillis())
+                  _state = SessionState.Error(System.currentTimeMillis())
                 case _ => aux()
               }
             }
@@ -109,12 +108,12 @@ private class BatchSessionYarn(val id: Int, process: LineBufferedProcess, jobFut
       _jobThread.start()
   }
 
-  override def state: State = _state
+  override def state: SessionState = _state
 
   override def stop(): Future[Unit] = {
     jobFuture.map { job =>
       job.stop()
-      _state = Success(System.currentTimeMillis())
+      _state = SessionState.Success(System.currentTimeMillis())
       ()
     }
   }

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

@@ -24,7 +24,7 @@ import java.util.concurrent.TimeoutException
 import com.cloudera.hue.livy.Utils
 import com.cloudera.hue.livy.msgs.ExecuteRequest
 import com.cloudera.hue.livy.server.Session
-import com.cloudera.hue.livy.sessions.{Kind, State}
+import com.cloudera.hue.livy.sessions.{SessionState, Kind}
 
 import scala.concurrent._
 import scala.concurrent.duration.Duration
@@ -54,7 +54,7 @@ trait InteractiveSession extends Session {
 
   @throws(classOf[TimeoutException])
   @throws(classOf[InterruptedException])
-  final def waitForStateChange(oldState: State, atMost: Duration) = {
+  final def waitForStateChange(oldState: SessionState, atMost: Duration) = {
     Utils.waitUntil({ () => state != oldState }, atMost)
   }
 }

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

@@ -21,7 +21,7 @@ package com.cloudera.hue.livy.server.interactive
 import java.lang.ProcessBuilder.Redirect
 import java.net.URL
 
-import com.cloudera.hue.livy.sessions.{Success, Dead, Error}
+import com.cloudera.hue.livy.sessions.SessionState
 import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder.{AbsolutePath, RelativePath}
 import com.cloudera.hue.livy.spark.{SparkProcess, SparkSubmitProcessBuilder}
 import com.cloudera.hue.livy.{LivyConf, Logging, Utils}
@@ -124,13 +124,13 @@ private class InteractiveSessionProcess(id: Int,
   // Error out the job if the process errors out.
   Future {
     if (process.waitFor() != 0) {
-      _state = Error()
+      _state = SessionState.Error()
     } else {
       // Set the state to done if the session shut down before contacting us.
       _state match {
-        case (Dead(_) | Error(_) | Success(_)) =>
+        case (SessionState.Dead(_) | SessionState.Error(_) | SessionState.Success(_)) =>
         case _ =>
-          _state = Success()
+          _state = SessionState.Success()
       }
     }
   }

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

@@ -47,7 +47,7 @@ class InteractiveSessionServlet(sessionManager: SessionManager[InteractiveSessio
 
     sessionManager.get(sessionId) match {
       case Some(session) =>
-        if (session.state == Starting()) {
+        if (session.state == SessionState.Starting()) {
           session.url = new URL(callback.url)
           Accepted()
         } else {
@@ -143,7 +143,7 @@ private object Serializers {
   def StatementFormats: List[CustomSerializer[_]] = List(StatementSerializer, StatementStateSerializer)
   def Formats: List[CustomSerializer[_]] = SessionFormats ++ StatementFormats
 
-  private def serializeSessionState(state: State) = JString(state.toString)
+  private def serializeSessionState(state: SessionState) = JString(state.toString)
 
   private def serializeSessionKind(kind: Kind) = JString(kind.toString)
 
@@ -193,11 +193,11 @@ private object Serializers {
     )
   )
 
-  case object SessionStateSerializer extends CustomSerializer[State](implicit formats => ( {
+  case object SessionStateSerializer extends CustomSerializer[SessionState](implicit formats => ( {
     // We don't support deserialization.
     PartialFunction.empty
   }, {
-    case state: State => JString(state.toString)
+    case state: SessionState => JString(state.toString)
   }
     )
   )

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

@@ -21,7 +21,7 @@ package com.cloudera.hue.livy.server.interactive
 import java.lang.ProcessBuilder.Redirect
 import java.util.concurrent.TimeUnit
 
-import com.cloudera.hue.livy.sessions.{PySpark, Error, Spark}
+import com.cloudera.hue.livy.sessions.{PySpark, SessionState}
 import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder.{AbsolutePath, RelativePath}
 import com.cloudera.hue.livy.spark.{SparkProcess, SparkSubmitProcessBuilder}
 import com.cloudera.hue.livy.yarn.Client
@@ -93,7 +93,7 @@ private class InteractiveSessionYarn(id: Int,
   // Error out the job if the process errors out.
   Future {
     if (process.waitFor() != 0) {
-      _state = Error()
+      _state = SessionState.Error()
     }
   }
 
@@ -104,7 +104,7 @@ private class InteractiveSessionYarn(id: Int,
   }
 
   job.onFailure { case _ =>
-    _state = Error()
+    _state = SessionState.Error()
   }
 
   override def logLines() = process.inputLines
@@ -121,7 +121,7 @@ private class InteractiveSessionYarn(id: Int,
           }
         } catch {
           case e: Throwable =>
-            _state = Error()
+            _state = SessionState.Error()
             throw e
         }
     }

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

@@ -39,7 +39,7 @@ abstract class InteractiveWebSession(val id: Int, createInteractiveRequest: Crea
   protected implicit def executor: ExecutionContextExecutor = ExecutionContext.global
   protected implicit def jsonFormats: Formats = DefaultFormats
 
-  protected[this] var _state: State = Starting()
+  protected[this] var _state: SessionState = SessionState.Starting()
 
   private[this] var _lastActivity = Long.MaxValue
   private[this] var _url: Option[URL] = None
@@ -54,8 +54,8 @@ abstract class InteractiveWebSession(val id: Int, createInteractiveRequest: Crea
   override def url: Option[URL] = _url
 
   override def url_=(url: URL) = {
-    ensureState(Starting(), {
-      _state = Idle()
+    ensureState(SessionState.Starting(), {
+      _state = SessionState.Idle()
       _url = Some(url)
     })
   }
@@ -67,11 +67,11 @@ abstract class InteractiveWebSession(val id: Int, createInteractiveRequest: Crea
 
   override def lastActivity: Option[Long] = Some(_lastActivity)
 
-  override def state: State = _state
+  override def state: SessionState = _state
 
   override def executeStatement(content: ExecuteRequest): Statement = {
     ensureRunning {
-      _state = Busy()
+      _state = SessionState.Busy()
       touchLastActivity()
 
       val req = (svc / "execute").setContentType("application/json", "UTF-8") << write(content)
@@ -115,11 +115,11 @@ abstract class InteractiveWebSession(val id: Int, createInteractiveRequest: Crea
         result \ "status" match {
           case JString("error") =>
             if (replErroredOut()) {
-              transition(Error())
+              transition(SessionState.Error())
             } else {
-              transition(Idle())
+              transition(SessionState.Idle())
             }
-          case _ => transition(Idle())
+          case _ => transition(SessionState.Idle())
         }
 
         Some(result)
@@ -145,15 +145,15 @@ abstract class InteractiveWebSession(val id: Int, createInteractiveRequest: Crea
   override def stop(): Future[Unit] = {
     synchronized {
       _state match {
-        case Idle() =>
-          _state = Busy()
+        case SessionState.Idle() =>
+          _state = SessionState.Busy()
 
           Http(svc.DELETE OK as.String).either() match {
             case (Right(_) | Left(_: ConnectException)) =>
               // Make sure to eat any connection errors because the repl shut down before it sent
               // out an OK.
               synchronized {
-                _state = Dead()
+                _state = SessionState.Dead()
               }
 
               Future.successful(())
@@ -161,33 +161,33 @@ abstract class InteractiveWebSession(val id: Int, createInteractiveRequest: Crea
             case Left(t: Throwable) =>
               Future.failed(t)
           }
-        case NotStarted() =>
+        case SessionState.NotStarted() =>
           Future {
-            waitForStateChange(NotStarted(), Duration(10, TimeUnit.SECONDS))
+            waitForStateChange(SessionState.NotStarted(), Duration(10, TimeUnit.SECONDS))
             stop()
           }
-        case Starting() =>
+        case SessionState.Starting() =>
           Future {
-            waitForStateChange(Starting(), Duration(10, TimeUnit.SECONDS))
+            waitForStateChange(SessionState.Starting(), Duration(10, TimeUnit.SECONDS))
             stop()
           }
-        case Busy() | Running() =>
+        case SessionState.Busy() | SessionState.Running() =>
           Future {
-            waitForStateChange(Busy(), Duration(10, TimeUnit.SECONDS))
+            waitForStateChange(SessionState.Busy(), Duration(10, TimeUnit.SECONDS))
             stop()
           }
-        case ShuttingDown() =>
+        case SessionState.ShuttingDown() =>
           Future {
-            waitForStateChange(ShuttingDown(), Duration(10, TimeUnit.SECONDS))
+            waitForStateChange(SessionState.ShuttingDown(), Duration(10, TimeUnit.SECONDS))
             stop()
           }
-        case Error(_) | Dead(_) | Success(_) =>
+        case SessionState.Error(_) | SessionState.Dead(_) | SessionState.Success(_) =>
           Future.successful(Unit)
       }
     }
   }
 
-  private def transition(state: State) = synchronized {
+  private def transition(state: SessionState) = synchronized {
     _state = state
   }
 
@@ -195,7 +195,7 @@ abstract class InteractiveWebSession(val id: Int, createInteractiveRequest: Crea
     _lastActivity = System.currentTimeMillis()
   }
 
-  private def ensureState[A](state: State, f: => A) = {
+  private def ensureState[A](state: SessionState, f: => A) = {
     synchronized {
       if (_state == state) {
         f
@@ -208,7 +208,7 @@ abstract class InteractiveWebSession(val id: Int, createInteractiveRequest: Crea
   private def ensureRunning[A](f: => A) = {
     synchronized {
       _state match {
-        case Idle() | Busy() =>
+        case SessionState.Idle() | SessionState.Busy() =>
           f
         case _ =>
           throw new IllegalStateException("Session is in state %s" format _state)

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

@@ -19,7 +19,7 @@
 package com.cloudera.hue.livy.server
 
 import com.cloudera.hue.livy.LivyConf
-import com.cloudera.hue.livy.sessions.{State, Success}
+import com.cloudera.hue.livy.sessions._
 import org.json4s.JsonAST.{JNothing, JValue}
 import org.scalatest.{FlatSpec, Matchers}
 
@@ -33,7 +33,7 @@ class SessionManagerSpec extends FlatSpec with Matchers {
 
     override def logLines(): IndexedSeq[String] = IndexedSeq()
 
-    override def state: State = Success(0)
+    override def state: SessionState = SessionState.Success(0)
   }
 
   class MockSessionFactory extends SessionFactory[MockSession] {

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

@@ -21,8 +21,7 @@ package com.cloudera.hue.livy.server.batch
 import java.io.FileWriter
 import java.nio.file.{Files, Path}
 import java.util.concurrent.TimeUnit
-
-import com.cloudera.hue.livy.sessions.Success
+import com.cloudera.hue.livy.sessions.SessionState
 import com.cloudera.hue.livy.{LivyConf, Utils}
 import org.scalatest.{BeforeAndAfterAll, FunSpec, ShouldMatchers}
 
@@ -57,7 +56,7 @@ class BatchProcessSpec
 
       Utils.waitUntil({ () => !batch.state.isActive }, Duration(10, TimeUnit.SECONDS))
       (batch.state match {
-        case Success(_) => true
+        case SessionState.Success(_) => true
         case _ => false
       }) should be (true)
 

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

@@ -23,9 +23,8 @@ import java.nio.file.{Files, Path}
 import java.util.concurrent.TimeUnit
 
 import com.cloudera.hue.livy.server.SessionManager
-import com.cloudera.hue.livy.sessions.Success
+import com.cloudera.hue.livy.sessions.SessionState
 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._
 import org.json4s.jackson.Serialization.write
@@ -91,10 +90,10 @@ class BatchServletSpec extends ScalatraSuite with FunSpecLike with BeforeAndAfte
 
       // Wait for the process to finish.
       {
-        val batch: BatchSession = batchManager.get(0).get
+        val batch = batchManager.get(0).get
         Utils.waitUntil({ () => !batch.state.isActive }, Duration(10, TimeUnit.SECONDS))
         (batch.state match {
-          case Success(_) => true
+          case SessionState.Success(_) => true
           case _ => false
         }) should be (true)
       }

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

@@ -21,7 +21,7 @@ package com.cloudera.hue.livy.server.interactive
 import java.util.concurrent.TimeUnit
 
 import com.cloudera.hue.livy.msgs.ExecuteRequest
-import com.cloudera.hue.livy.sessions.{Error, Idle, Starting}
+import com.cloudera.hue.livy.sessions._
 import org.json4s.{DefaultFormats, Extraction}
 import org.scalatest.{BeforeAndAfter, FunSpec, Matchers}
 
@@ -46,16 +46,16 @@ abstract class BaseSessionSpec extends FunSpec with Matchers with BeforeAndAfter
 
   describe("A spark session") {
     it("should start in the starting or idle state") {
-      session.state should (equal (Starting()) or equal (Idle()))
+      session.state should (equal (SessionState.Starting()) or equal (SessionState.Idle()))
     }
 
     it("should eventually become the idle state") {
-      session.waitForStateChange(Starting(), Duration(30, TimeUnit.SECONDS))
-      session.state should equal (Idle())
+      session.waitForStateChange(SessionState.Starting(), Duration(30, TimeUnit.SECONDS))
+      session.state should equal (SessionState.Idle())
     }
 
     it("should execute `1 + 2` == 3") {
-      session.waitForStateChange(Starting(), Duration(30, TimeUnit.SECONDS))
+      session.waitForStateChange(SessionState.Starting(), Duration(30, TimeUnit.SECONDS))
       val stmt = session.executeStatement(ExecuteRequest("1 + 2"))
       val result = Await.result(stmt.output(), Duration.Inf)
 
@@ -71,7 +71,7 @@ abstract class BaseSessionSpec extends FunSpec with Matchers with BeforeAndAfter
     }
 
     it("should report an error if accessing an unknown variable") {
-      session.waitForStateChange(Starting(), Duration(30, TimeUnit.SECONDS))
+      session.waitForStateChange(SessionState.Starting(), Duration(30, TimeUnit.SECONDS))
       val stmt = session.executeStatement(ExecuteRequest("x"))
       val result = Await.result(stmt.output(), Duration.Inf)
       val expectedResult = Extraction.decompose(Map(
@@ -86,15 +86,15 @@ abstract class BaseSessionSpec extends FunSpec with Matchers with BeforeAndAfter
       ))
 
       result should equal (expectedResult)
-      session.state should equal (Idle())
+      session.state should equal (SessionState.Idle())
     }
 
     it("should error out the session if the interpreter dies") {
-      session.waitForStateChange(Starting(), Duration(30, TimeUnit.SECONDS))
+      session.waitForStateChange(SessionState.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 match {
-        case Error(_) => true
+        case SessionState.Error(_) => true
         case _ => false
       }) should equal (true)
     }

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

@@ -39,7 +39,7 @@ class InteractiveSessionServletSpec extends ScalatraSuite with FunSpecLike {
   protected implicit def jsonFormats: Formats = DefaultFormats ++ Serializers.SessionFormats
 
   class MockInteractiveSession(val id: Int) extends InteractiveSession {
-    var _state: State = Idle()
+    var _state: SessionState = SessionState.Idle()
 
     var _idCounter = new AtomicInteger()
     var _statements = IndexedSeq[Statement]()