You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@livy.apache.org by js...@apache.org on 2017/11/15 03:42:57 UTC

incubator-livy git commit: [LIVY-415] Use objects and abstract classes in for Kind and SessionState.

Repository: incubator-livy
Updated Branches:
  refs/heads/master ef5dccbb0 -> 54715447e


[LIVY-415] Use objects and abstract classes in for Kind and SessionState.

## What changes were proposed in this pull request?

- Use a singleton (object) rather than having a case class and creating an object of only one type whenever we need an object of the type. Using an object helps with code efficiency and readability.
- Use abstract classes to set up methods with default definitions to follow the DRY principles.

## How was this patch tested?

All the existing tests were modified to use the new changes. The changes were then tested on Travis with existing settings to make sure it's compatible with the the upstream repository.

Please review https://livy.incubator.apache.org/community/ before opening a pull request.

I have read through the page and felt like this change does not require a JIRA. If you feel otherwise, I can create a JIRA and then attach it to the PR.

Author: Arun Allamsetty <ar...@gmail.com>

Closes #62 from aa8y/singleton.


Project: http://git-wip-us.apache.org/repos/asf/incubator-livy/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-livy/commit/54715447
Tree: http://git-wip-us.apache.org/repos/asf/incubator-livy/tree/54715447
Diff: http://git-wip-us.apache.org/repos/asf/incubator-livy/diff/54715447

Branch: refs/heads/master
Commit: 54715447e0865483826dec6ad1db8a4187983189
Parents: ef5dccb
Author: Arun Allamsetty <ar...@gmail.com>
Authored: Wed Nov 15 11:42:45 2017 +0800
Committer: jerryshao <ss...@hortonworks.com>
Committed: Wed Nov 15 11:42:45 2017 +0800

----------------------------------------------------------------------
 .../livy/client/http/HttpClientSpec.scala       |   4 +-
 .../scala/org/apache/livy/sessions/Kind.scala   |  28 ++---
 .../org/apache/livy/sessions/SessionState.scala | 106 ++++++-------------
 .../livy/test/framework/LivyRestClient.scala    |  10 +-
 .../scala/org/apache/livy/test/BatchIT.scala    |   4 +-
 .../org/apache/livy/test/InteractiveIT.scala    |  20 ++--
 .../scala/org/apache/livy/repl/ReplDriver.scala |   8 +-
 .../scala/org/apache/livy/repl/Session.scala    |  30 +++---
 .../org/apache/livy/repl/BaseSessionSpec.scala  |   6 +-
 .../apache/livy/repl/PythonSessionSpec.scala    |   2 +-
 .../org/apache/livy/repl/ReplDriverSuite.scala  |   2 +-
 .../apache/livy/repl/SharedSessionSpec.scala    |   2 +-
 .../apache/livy/repl/SparkRSessionSpec.scala    |   2 +-
 .../org/apache/livy/repl/SparkSessionSpec.scala |   2 +-
 .../apache/livy/server/batch/BatchSession.scala |  10 +-
 .../interactive/CreateInteractiveRequest.scala  |   2 +-
 .../server/interactive/InteractiveSession.scala |  28 +++--
 .../apache/livy/server/SessionServletSpec.scala |   2 +-
 .../livy/server/batch/BatchServletSpec.scala    |   2 +-
 .../livy/server/batch/BatchSessionSpec.scala    |   2 +-
 .../BaseInteractiveServletSpec.scala            |   2 +-
 .../CreateInteractiveRequestSpec.scala          |   2 +-
 .../InteractiveSessionServletSpec.scala         |   8 +-
 .../interactive/InteractiveSessionSpec.scala    |  22 ++--
 .../livy/server/interactive/JobApiSpec.scala    |   2 +-
 .../org/apache/livy/sessions/MockSession.scala  |   2 +-
 .../livy/sessions/SessionManagerSpec.scala      |  18 ++--
 27 files changed, 142 insertions(+), 186 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/client-http/src/test/scala/org/apache/livy/client/http/HttpClientSpec.scala
----------------------------------------------------------------------
diff --git a/client-http/src/test/scala/org/apache/livy/client/http/HttpClientSpec.scala b/client-http/src/test/scala/org/apache/livy/client/http/HttpClientSpec.scala
index 5ea6f8d..837c24c 100644
--- a/client-http/src/test/scala/org/apache/livy/client/http/HttpClientSpec.scala
+++ b/client-http/src/test/scala/org/apache/livy/client/http/HttpClientSpec.scala
@@ -275,9 +275,9 @@ private class HttpClientTestBootstrap extends LifeCycle {
         when(session.id).thenReturn(id)
         when(session.appId).thenReturn(None)
         when(session.appInfo).thenReturn(AppInfo())
-        when(session.state).thenReturn(SessionState.Idle())
+        when(session.state).thenReturn(SessionState.Idle)
         when(session.proxyUser).thenReturn(None)
-        when(session.kind).thenReturn(Spark())
+        when(session.kind).thenReturn(Spark)
         when(session.stop()).thenReturn(Future.successful(()))
         require(HttpClientSpec.session == null, "Session already created?")
         HttpClientSpec.session = session

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/core/src/main/scala/org/apache/livy/sessions/Kind.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/livy/sessions/Kind.scala b/core/src/main/scala/org/apache/livy/sessions/Kind.scala
index 5e7fa00..4779822 100644
--- a/core/src/main/scala/org/apache/livy/sessions/Kind.scala
+++ b/core/src/main/scala/org/apache/livy/sessions/Kind.scala
@@ -21,33 +21,27 @@ import com.fasterxml.jackson.core.{JsonGenerator, JsonParser, JsonToken}
 import com.fasterxml.jackson.databind._
 import com.fasterxml.jackson.databind.module.SimpleModule
 
-sealed trait Kind
-case class Spark() extends Kind {
-  override def toString: String = "spark"
+sealed abstract class Kind(val name: String) {
+  override def toString: String = name
 }
 
-case class PySpark() extends Kind {
-  override def toString: String = "pyspark"
-}
+object Spark extends Kind("spark")
 
-case class SparkR() extends Kind {
-  override def toString: String = "sparkr"
-}
+object PySpark extends Kind("pyspark")
 
-case class Shared() extends Kind {
-  override def toString: String = "shared"
-}
+object SparkR extends Kind("sparkr")
+
+object Shared extends Kind("shared")
 
 object Kind {
 
   def apply(kind: String): Kind = kind match {
-    case "spark" | "scala" => Spark()
-    case "pyspark" | "python" => PySpark()
-    case "sparkr" | "r" => SparkR()
-    case "shared" => Shared()
+    case "spark" | "scala" => Spark
+    case "pyspark" | "python" => PySpark
+    case "sparkr" | "r" => SparkR
+    case "shared" => Shared
     case other => throw new IllegalArgumentException(s"Invalid kind: $other")
   }
-
 }
 
 class SessionKindModule extends SimpleModule("SessionKind") {

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/core/src/main/scala/org/apache/livy/sessions/SessionState.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/livy/sessions/SessionState.scala b/core/src/main/scala/org/apache/livy/sessions/SessionState.scala
index fd19321..577a27a 100644
--- a/core/src/main/scala/org/apache/livy/sessions/SessionState.scala
+++ b/core/src/main/scala/org/apache/livy/sessions/SessionState.scala
@@ -17,91 +17,55 @@
 
 package org.apache.livy.sessions
 
-sealed trait SessionState {
-  /** Returns true if the State represents a process that can eventually execute commands */
-  def isActive: Boolean
+sealed abstract class SessionState(val state: String, val isActive: Boolean) {
+  override def toString: String = state
 }
 
-sealed trait FinishedSessionState extends SessionState {
-  /** When session is finished. */
-  def time: Long
-}
+class FinishedSessionState(
+  override val state: String,
+  override val isActive: Boolean,
+  val time: Long
+) extends SessionState(state, isActive)
 
 object SessionState {
 
-  def apply(s: String): SessionState = {
-    s match {
-      case "not_started" => NotStarted()
-      case "starting" => Starting()
-      case "recovering" => Recovering()
-      case "idle" => Idle()
-      case "running" => Running()
-      case "busy" => Busy()
-      case "shutting_down" => ShuttingDown()
-      case "error" => Error()
-      case "dead" => Dead()
-      case "success" => Success()
-      case _ => throw new IllegalArgumentException(s"Illegal session state: $s")
-    }
+  def apply(s: String): SessionState = s match {
+    case "not_started" => NotStarted
+    case "starting" => Starting
+    case "recovering" => Recovering
+    case "idle" => Idle
+    case "running" => Running
+    case "busy" => Busy
+    case "shutting_down" => ShuttingDown
+    case "error" => Error
+    case "dead" => Dead
+    case "success" => Success
+    case _ => throw new IllegalArgumentException(s"Illegal session state: $s")
   }
 
-  case class NotStarted() extends SessionState {
-    override def isActive: Boolean = true
-
-    override def toString: String = "not_started"
-  }
+  object NotStarted extends SessionState("not_started", true)
 
-  case class Starting() extends SessionState {
-    override def isActive: Boolean = true
+  object Starting extends SessionState("starting", true)
 
-    override def toString: String = "starting"
-  }
+  object Recovering extends SessionState("recovering", true)
 
-  case class Recovering() extends SessionState {
-    override def isActive: Boolean = true
+  object Idle extends SessionState("idle", true)
 
-    override def toString: String = "recovering"
-  }
+  object Running extends SessionState("running", true)
 
-  case class Idle() extends SessionState {
-    override def isActive: Boolean = true
+  object Busy extends SessionState("busy", true)
 
-    override def toString: String = "idle"
-  }
+  object ShuttingDown extends SessionState("shutting_down", false)
 
-  case class Running() extends SessionState {
-    override def isActive: Boolean = true
-
-    override def toString: String = "running"
-  }
-
-  case class Busy() extends SessionState {
-    override def isActive: Boolean = true
-
-    override def toString: String = "busy"
-  }
+  case class Error(override val time: Long) extends
+    FinishedSessionState("error", true, time)
+  object Error extends Error(System.nanoTime)
 
-  case class ShuttingDown() extends SessionState {
-    override def isActive: Boolean = false
+  case class Dead(override val time: Long) extends
+    FinishedSessionState("dead", false, time)
+  object Dead extends Dead(System.nanoTime)
 
-    override def toString: String = "shutting_down"
-  }
-
-  case class Error(time: Long = System.nanoTime()) extends FinishedSessionState {
-    override def isActive: Boolean = true
-
-    override def toString: String = "error"
-  }
-
-  case class Dead(time: Long = System.nanoTime()) extends FinishedSessionState {
-    override def isActive: Boolean = false
-
-    override def toString: String = "dead"
-  }
-
-  case class Success(time: Long = System.nanoTime()) extends FinishedSessionState {
-    override def isActive: Boolean = false
-
-    override def toString: String = "success"
-  }
+  case class Success(override val time: Long) extends
+    FinishedSessionState("success", false, time)
+  object Success extends Success(System.nanoTime)
 }

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/integration-test/src/main/scala/org/apache/livy/test/framework/LivyRestClient.scala
----------------------------------------------------------------------
diff --git a/integration-test/src/main/scala/org/apache/livy/test/framework/LivyRestClient.scala b/integration-test/src/main/scala/org/apache/livy/test/framework/LivyRestClient.scala
index eaa023a..941a650 100644
--- a/integration-test/src/main/scala/org/apache/livy/test/framework/LivyRestClient.scala
+++ b/integration-test/src/main/scala/org/apache/livy/test/framework/LivyRestClient.scala
@@ -111,9 +111,9 @@ class LivyRestClient(val httpClient: AsyncHttpClient, val livyEndpoint: String)
   }
 
   class BatchSession(id: Int) extends Session(id, BATCH_TYPE) {
-    def verifySessionDead(): Unit = verifySessionState(SessionState.Dead())
-    def verifySessionRunning(): Unit = verifySessionState(SessionState.Running())
-    def verifySessionSuccess(): Unit = verifySessionState(SessionState.Success())
+    def verifySessionDead(): Unit = verifySessionState(SessionState.Dead)
+    def verifySessionRunning(): Unit = verifySessionState(SessionState.Running)
+    def verifySessionSuccess(): Unit = verifySessionState(SessionState.Success)
   }
 
   class InteractiveSession(id: Int) extends Session(id, INTERACTIVE_TYPE) {
@@ -226,11 +226,11 @@ class LivyRestClient(val httpClient: AsyncHttpClient, val livyEndpoint: String)
         .setBody(mapper.writeValueAsString(requestBody))
         .execute()
 
-      verifySessionState(SessionState.Dead())
+      verifySessionState(SessionState.Dead)
     }
 
     def verifySessionIdle(): Unit = {
-      verifySessionState(SessionState.Idle())
+      verifySessionState(SessionState.Idle)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/integration-test/src/test/scala/org/apache/livy/test/BatchIT.scala
----------------------------------------------------------------------
diff --git a/integration-test/src/test/scala/org/apache/livy/test/BatchIT.scala b/integration-test/src/test/scala/org/apache/livy/test/BatchIT.scala
index 0b3a061..32794e7 100644
--- a/integration-test/src/test/scala/org/apache/livy/test/BatchIT.scala
+++ b/integration-test/src/test/scala/org/apache/livy/test/BatchIT.scala
@@ -87,7 +87,7 @@ class BatchIT extends BaseIntegrationTestSuite with BeforeAndAfterAll {
   test("deleting a session should kill YARN app") {
     val output = newOutputPath()
     withTestLib(classOf[SimpleSparkApp], List(output, "false")) { s =>
-      s.verifySessionState(SessionState.Running())
+      s.verifySessionState(SessionState.Running)
       s.snapshot().appInfo.driverLogUrl.value should include ("containerlogs")
 
       val appId = s.appId()
@@ -102,7 +102,7 @@ class BatchIT extends BaseIntegrationTestSuite with BeforeAndAfterAll {
   test("killing YARN app should change batch state to dead") {
     val output = newOutputPath()
     withTestLib(classOf[SimpleSparkApp], List(output, "false")) { s =>
-      s.verifySessionState(SessionState.Running())
+      s.verifySessionState(SessionState.Running)
       val appId = s.appId()
 
       // Kill the YARN app and check batch state should be KILLED.

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/integration-test/src/test/scala/org/apache/livy/test/InteractiveIT.scala
----------------------------------------------------------------------
diff --git a/integration-test/src/test/scala/org/apache/livy/test/InteractiveIT.scala b/integration-test/src/test/scala/org/apache/livy/test/InteractiveIT.scala
index 0b341e2..22ef739 100644
--- a/integration-test/src/test/scala/org/apache/livy/test/InteractiveIT.scala
+++ b/integration-test/src/test/scala/org/apache/livy/test/InteractiveIT.scala
@@ -32,7 +32,7 @@ import org.apache.livy.test.framework.{BaseIntegrationTestSuite, LivyRestClient}
 
 class InteractiveIT extends BaseIntegrationTestSuite {
   test("basic interactive session") {
-    withNewSession(Spark()) { s =>
+    withNewSession(Spark) { s =>
       s.run("val sparkVersion = sc.version").result().left.foreach(info(_))
       s.run("1+1").verifyResult("res0: Int = 2")
       s.run("""sc.getConf.get("spark.executor.instances")""").verifyResult("res1: String = 1")
@@ -67,7 +67,7 @@ class InteractiveIT extends BaseIntegrationTestSuite {
   }
 
   pytest("pyspark interactive session") {
-    withNewSession(PySpark()) { s =>
+    withNewSession(PySpark) { s =>
       s.run("1+1").verifyResult("2")
       s.run("sqlContext").verifyResult(startsWith("<pyspark.sql.context.HiveContext"))
       s.run("sc.parallelize(range(100)).map(lambda x: x * 2).reduce(lambda x, y: x + y)")
@@ -82,7 +82,7 @@ class InteractiveIT extends BaseIntegrationTestSuite {
   }
 
   rtest("R interactive session") {
-    withNewSession(SparkR()) { s =>
+    withNewSession(SparkR) { s =>
       // R's output sometimes includes the count of statements, which makes it annoying to test
       // things. This helps a bit.
       val curr = new AtomicInteger()
@@ -102,14 +102,14 @@ class InteractiveIT extends BaseIntegrationTestSuite {
   }
 
   test("application kills session") {
-    withNewSession(Spark()) { s =>
+    withNewSession(Spark) { s =>
       s.runFatalStatement("System.exit(0)")
     }
   }
 
   test("should kill RSCDriver if it doesn't respond to end session") {
     val testConfName = s"${RSCConf.LIVY_SPARK_PREFIX}${RSCConf.Entry.TEST_STUCK_END_SESSION.key()}"
-    withNewSession(Spark(), Map(testConfName -> "true")) { s =>
+    withNewSession(Spark, Map(testConfName -> "true")) { s =>
       val appId = s.appId()
       s.stop()
       val appReport = cluster.yarnClient.getApplicationReport(appId)
@@ -120,7 +120,7 @@ class InteractiveIT extends BaseIntegrationTestSuite {
   test("should kill RSCDriver if it didn't register itself in time") {
     val testConfName =
       s"${RSCConf.LIVY_SPARK_PREFIX}${RSCConf.Entry.TEST_STUCK_START_DRIVER.key()}"
-    withNewSession(Spark(), Map(testConfName -> "true"), false) { s =>
+    withNewSession(Spark, Map(testConfName -> "true"), false) { s =>
       eventually(timeout(2 minutes), interval(5 seconds)) {
         val appId = s.appId()
         appId should not be null
@@ -133,7 +133,7 @@ class InteractiveIT extends BaseIntegrationTestSuite {
   test("user jars are properly imported in Scala interactive sessions") {
     // Include a popular Java library to test importing user jars.
     val sparkConf = Map("spark.jars.packages" -> "org.codehaus.plexus:plexus-utils:3.0.24")
-    withNewSession(Spark(), sparkConf) { s =>
+    withNewSession(Spark, sparkConf) { s =>
       // Check is the library loaded in JVM in the proper class loader.
       s.run("Thread.currentThread.getContextClassLoader.loadClass" +
           """("org.codehaus.plexus.util.FileUtils")""")
@@ -153,7 +153,7 @@ class InteractiveIT extends BaseIntegrationTestSuite {
   test("heartbeat should kill expired session") {
     // Set it to 2s because verifySessionIdle() is calling GET every second.
     val heartbeatTimeout = Duration.create("2s")
-    withNewSession(Spark(), Map.empty, true, heartbeatTimeout.toSeconds.toInt) { s =>
+    withNewSession(Spark, Map.empty, true, heartbeatTimeout.toSeconds.toInt) { s =>
       // If the test reaches here, that means verifySessionIdle() is successfully keeping the
       // session alive. Now verify heartbeat is killing expired session.
       Thread.sleep(heartbeatTimeout.toMillis * 2)
@@ -162,7 +162,7 @@ class InteractiveIT extends BaseIntegrationTestSuite {
   }
 
   test("recover interactive session") {
-    withNewSession(Spark()) { s =>
+    withNewSession(Spark) { s =>
       val stmt1 = s.run("1")
       stmt1.verifyResult("res0: Int = 1")
 
@@ -182,7 +182,7 @@ class InteractiveIT extends BaseIntegrationTestSuite {
       s.verifySessionDoesNotExist()
 
       // Verify new session doesn't reuse old session id.
-      withNewSession(Spark(), Map.empty, false) { s1 =>
+      withNewSession(Spark, Map.empty, false) { s1 =>
         s1.id should be > s.id
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/repl/src/main/scala/org/apache/livy/repl/ReplDriver.scala
----------------------------------------------------------------------
diff --git a/repl/src/main/scala/org/apache/livy/repl/ReplDriver.scala b/repl/src/main/scala/org/apache/livy/repl/ReplDriver.scala
index ce5b8be..af51e43 100644
--- a/repl/src/main/scala/org/apache/livy/repl/ReplDriver.scala
+++ b/repl/src/main/scala/org/apache/livy/repl/ReplDriver.scala
@@ -93,24 +93,24 @@ class ReplDriver(conf: SparkConf, livyConf: RSCConf)
 
   override protected def createWrapper(msg: BaseProtocol.BypassJobRequest): BypassJobWrapper = {
     Kind(msg.jobType) match {
-      case PySpark() if session.interpreter(PySpark()).isDefined =>
+      case PySpark if session.interpreter(PySpark).isDefined =>
         new BypassJobWrapper(this, msg.id,
           new BypassPySparkJob(msg.serializedJob,
-            session.interpreter(PySpark()).get.asInstanceOf[PythonInterpreter]))
+            session.interpreter(PySpark).get.asInstanceOf[PythonInterpreter]))
       case _ => super.createWrapper(msg)
     }
   }
 
   override protected def addFile(path: String): Unit = {
     if (!ClientConf.TEST_MODE) {
-      session.interpreter(PySpark()).foreach { _.asInstanceOf[PythonInterpreter].addFile(path) }
+      session.interpreter(PySpark).foreach { _.asInstanceOf[PythonInterpreter].addFile(path) }
     }
     super.addFile(path)
   }
 
   override protected def addJarOrPyFile(path: String): Unit = {
     if (!ClientConf.TEST_MODE) {
-      session.interpreter(PySpark())
+      session.interpreter(PySpark)
         .foreach { _.asInstanceOf[PythonInterpreter].addPyFile(this, conf, path) }
     }
     super.addJarOrPyFile(path)

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/repl/src/main/scala/org/apache/livy/repl/Session.scala
----------------------------------------------------------------------
diff --git a/repl/src/main/scala/org/apache/livy/repl/Session.scala b/repl/src/main/scala/org/apache/livy/repl/Session.scala
index fc125ac..5d471cd 100644
--- a/repl/src/main/scala/org/apache/livy/repl/Session.scala
+++ b/repl/src/main/scala/org/apache/livy/repl/Session.scala
@@ -65,7 +65,7 @@ class Session(
 
   private implicit val formats = DefaultFormats
 
-  private var _state: SessionState = SessionState.NotStarted()
+  private var _state: SessionState = SessionState.NotStarted
 
   // Number of statements kept in driver's memory
   private val numRetainedStatements = livyConf.getInt(RSCConf.Entry.RETAINED_STATEMENTS)
@@ -100,11 +100,11 @@ class Session(
           "SparkEntries should not be null when lazily initialize other interpreters.")
 
         val interp = kind match {
-          case Spark() =>
+          case Spark =>
             // This should never be touched here.
             throw new IllegalStateException("SparkInterpreter should not be lazily created.")
-          case PySpark() => PythonInterpreter(sparkConf, entries)
-          case SparkR() => SparkRInterpreter(sparkConf, entries)
+          case PySpark => PythonInterpreter(sparkConf, entries)
+          case SparkR => SparkRInterpreter(sparkConf, entries)
         }
         interp.start()
         interpGroup(kind) = interp
@@ -119,7 +119,7 @@ class Session(
 
   def start(): Future[SparkEntries] = {
     val future = Future {
-      changeState(SessionState.Starting())
+      changeState(SessionState.Starting)
 
       // Always start SparkInterpreter after beginning, because we rely on SparkInterpreter to
       // initialize SparkContext and create SparkEntries.
@@ -129,14 +129,14 @@ class Session(
       entries = sparkInterp.sparkEntries()
       require(entries != null, "SparkEntries object should not be null in Spark Interpreter.")
       interpGroup.synchronized {
-        interpGroup.put(Spark(), sparkInterp)
+        interpGroup.put(Spark, sparkInterp)
       }
 
-      changeState(SessionState.Idle())
+      changeState(SessionState.Idle)
       entries
     }(interpreterExecutor)
 
-    future.onFailure { case _ => changeState(SessionState.Error()) }(interpreterExecutor)
+    future.onFailure { case _ => changeState(SessionState.Error) }(interpreterExecutor)
     future
   }
 
@@ -149,7 +149,7 @@ class Session(
   def execute(code: String, codeType: String = null): Int = {
     val tpe = if (codeType != null) {
       Kind(codeType)
-    } else if (defaultInterpKind != Shared()) {
+    } else if (defaultInterpKind != Shared) {
       defaultInterpKind
     } else {
       throw new IllegalArgumentException(s"Code type should be specified if session kind is shared")
@@ -259,12 +259,12 @@ class Session(
   private def executeCode(interp: Option[Interpreter],
      executionCount: Int,
      code: String): String = {
-    changeState(SessionState.Busy())
+    changeState(SessionState.Busy)
 
     def transitToIdle() = {
       val executingLastStatement = executionCount == newStatementId.intValue() - 1
       if (_statements.isEmpty || executingLastStatement) {
-        changeState(SessionState.Idle())
+        changeState(SessionState.Idle)
       }
     }
 
@@ -297,7 +297,7 @@ class Session(
               (TRACEBACK -> traceback)
 
           case Interpreter.ExecuteAborted(message) =>
-            changeState(SessionState.Error())
+            changeState(SessionState.Error)
 
             (STATUS -> ERROR) ~
               (EXECUTION_COUNT -> executionCount) ~
@@ -332,13 +332,13 @@ class Session(
   private def setJobGroup(codeType: Kind, statementId: Int): String = {
     val jobGroup = statementIdToJobGroup(statementId)
     val cmd = codeType match {
-      case Spark() =>
+      case Spark =>
         // A dummy value to avoid automatic value binding in scala REPL.
         s"""val _livyJobGroup$jobGroup = sc.setJobGroup("$jobGroup",""" +
           s""""Job group for statement $jobGroup")"""
-      case PySpark() =>
+      case PySpark =>
         s"""sc.setJobGroup("$jobGroup", "Job group for statement $jobGroup")"""
-      case SparkR() =>
+      case SparkR =>
         sc.getConf.get("spark.livy.spark_major_version", "1") match {
           case "1" =>
             s"""setJobGroup(sc, "$jobGroup", "Job group for statement $jobGroup", """ +

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/repl/src/test/scala/org/apache/livy/repl/BaseSessionSpec.scala
----------------------------------------------------------------------
diff --git a/repl/src/test/scala/org/apache/livy/repl/BaseSessionSpec.scala b/repl/src/test/scala/org/apache/livy/repl/BaseSessionSpec.scala
index bcea0df..7e23d1f 100644
--- a/repl/src/test/scala/org/apache/livy/repl/BaseSessionSpec.scala
+++ b/repl/src/test/scala/org/apache/livy/repl/BaseSessionSpec.scala
@@ -60,7 +60,7 @@ abstract class BaseSessionSpec(kind: Kind)
       // Session's constructor should fire an initial state change event.
       stateChangedCalled.intValue() shouldBe 1
       Await.ready(session.start(), 30 seconds)
-      assert(session.state === SessionState.Idle())
+      assert(session.state === SessionState.Idle)
       // There should be at least 1 state change event fired when session transits to idle.
       stateChangedCalled.intValue() should (be > 1)
       testCode(session)
@@ -74,14 +74,14 @@ abstract class BaseSessionSpec(kind: Kind)
     val future = session.start()
     try {
       Await.ready(future, 60 seconds)
-      session.state should (equal (SessionState.Starting()) or equal (SessionState.Idle()))
+      session.state should (equal (SessionState.Starting) or equal (SessionState.Idle))
     } finally {
       session.close()
     }
   }
 
   it should "eventually become the idle state" in withSession { session =>
-    session.state should equal (SessionState.Idle())
+    session.state should equal (SessionState.Idle)
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/repl/src/test/scala/org/apache/livy/repl/PythonSessionSpec.scala
----------------------------------------------------------------------
diff --git a/repl/src/test/scala/org/apache/livy/repl/PythonSessionSpec.scala b/repl/src/test/scala/org/apache/livy/repl/PythonSessionSpec.scala
index 69d4131..b54be11 100644
--- a/repl/src/test/scala/org/apache/livy/repl/PythonSessionSpec.scala
+++ b/repl/src/test/scala/org/apache/livy/repl/PythonSessionSpec.scala
@@ -23,7 +23,7 @@ import org.scalatest._
 
 import org.apache.livy.sessions._
 
-abstract class PythonSessionSpec extends BaseSessionSpec(PySpark()) {
+abstract class PythonSessionSpec extends BaseSessionSpec(PySpark) {
 
   it should "execute `1 + 2` == 3" in withSession { session =>
     val statement = execute(session)("1 + 2")

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/repl/src/test/scala/org/apache/livy/repl/ReplDriverSuite.scala
----------------------------------------------------------------------
diff --git a/repl/src/test/scala/org/apache/livy/repl/ReplDriverSuite.scala b/repl/src/test/scala/org/apache/livy/repl/ReplDriverSuite.scala
index 6d7094d..da64577 100644
--- a/repl/src/test/scala/org/apache/livy/repl/ReplDriverSuite.scala
+++ b/repl/src/test/scala/org/apache/livy/repl/ReplDriverSuite.scala
@@ -45,7 +45,7 @@ class ReplDriverSuite extends FunSuite with LivyBaseUnitTestSuite {
       .setConf(RSCConf.Entry.LIVY_JARS.key(), "")
       .setURI(new URI("rsc:/"))
       .setConf(RSCConf.Entry.DRIVER_CLASS.key(), classOf[ReplDriver].getName())
-      .setConf(RSCConf.Entry.SESSION_KIND.key(), Spark().toString)
+      .setConf(RSCConf.Entry.SESSION_KIND.key(), Spark.toString)
       .build()
       .asInstanceOf[RSCClient]
 

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/repl/src/test/scala/org/apache/livy/repl/SharedSessionSpec.scala
----------------------------------------------------------------------
diff --git a/repl/src/test/scala/org/apache/livy/repl/SharedSessionSpec.scala b/repl/src/test/scala/org/apache/livy/repl/SharedSessionSpec.scala
index d63e22f..1edb8a2 100644
--- a/repl/src/test/scala/org/apache/livy/repl/SharedSessionSpec.scala
+++ b/repl/src/test/scala/org/apache/livy/repl/SharedSessionSpec.scala
@@ -28,7 +28,7 @@ import org.scalatest.concurrent.Eventually.{eventually, interval, timeout}
 import org.apache.livy.rsc.driver.{Statement, StatementState}
 import org.apache.livy.sessions._
 
-class SharedSessionSpec extends BaseSessionSpec(Shared()) {
+class SharedSessionSpec extends BaseSessionSpec(Shared) {
 
   private def execute(session: Session, code: String, codeType: String): Statement = {
     val id = session.execute(code, codeType)

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/repl/src/test/scala/org/apache/livy/repl/SparkRSessionSpec.scala
----------------------------------------------------------------------
diff --git a/repl/src/test/scala/org/apache/livy/repl/SparkRSessionSpec.scala b/repl/src/test/scala/org/apache/livy/repl/SparkRSessionSpec.scala
index dc1887d..f1d271a 100644
--- a/repl/src/test/scala/org/apache/livy/repl/SparkRSessionSpec.scala
+++ b/repl/src/test/scala/org/apache/livy/repl/SparkRSessionSpec.scala
@@ -22,7 +22,7 @@ import org.json4s.jackson.JsonMethods.parse
 
 import org.apache.livy.sessions._
 
-class SparkRSessionSpec extends BaseSessionSpec(SparkR()) {
+class SparkRSessionSpec extends BaseSessionSpec(SparkR) {
 
   override protected def withFixture(test: NoArgTest) = {
     assume(!sys.props.getOrElse("skipRTests", "false").toBoolean, "Skipping R tests.")

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/repl/src/test/scala/org/apache/livy/repl/SparkSessionSpec.scala
----------------------------------------------------------------------
diff --git a/repl/src/test/scala/org/apache/livy/repl/SparkSessionSpec.scala b/repl/src/test/scala/org/apache/livy/repl/SparkSessionSpec.scala
index 7e51080..dad817b 100644
--- a/repl/src/test/scala/org/apache/livy/repl/SparkSessionSpec.scala
+++ b/repl/src/test/scala/org/apache/livy/repl/SparkSessionSpec.scala
@@ -28,7 +28,7 @@ import org.scalatest.concurrent.Eventually._
 import org.apache.livy.rsc.driver.StatementState
 import org.apache.livy.sessions._
 
-class SparkSessionSpec extends BaseSessionSpec(Spark()) {
+class SparkSessionSpec extends BaseSessionSpec(Spark) {
 
   it should "execute `1 + 2` == 3" in withSession { session =>
     val statement = execute(session)("1 + 2")

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/server/src/main/scala/org/apache/livy/server/batch/BatchSession.scala
----------------------------------------------------------------------
diff --git a/server/src/main/scala/org/apache/livy/server/batch/BatchSession.scala b/server/src/main/scala/org/apache/livy/server/batch/BatchSession.scala
index 29a8d8c..ae9b212 100644
--- a/server/src/main/scala/org/apache/livy/server/batch/BatchSession.scala
+++ b/server/src/main/scala/org/apache/livy/server/batch/BatchSession.scala
@@ -113,7 +113,7 @@ object BatchSession extends Logging {
     new BatchSession(
       id,
       appTag,
-      SessionState.Starting(),
+      SessionState.Starting,
       livyConf,
       owner,
       proxyUser,
@@ -129,7 +129,7 @@ object BatchSession extends Logging {
     new BatchSession(
       m.id,
       m.appTag,
-      SessionState.Recovering(),
+      SessionState.Recovering,
       livyConf,
       m.owner,
       m.proxyUser,
@@ -175,12 +175,12 @@ class BatchSession(
       debug(s"$this state changed from $oldState to $newState")
       newState match {
         case SparkApp.State.RUNNING =>
-          _state = SessionState.Running()
+          _state = SessionState.Running
           info(s"Batch session $id created [appid: ${appId.orNull}, state: ${state.toString}, " +
             s"info: ${appInfo.asJavaMap}]")
-        case SparkApp.State.FINISHED => _state = SessionState.Success()
+        case SparkApp.State.FINISHED => _state = SessionState.Success
         case SparkApp.State.KILLED | SparkApp.State.FAILED =>
-          _state = SessionState.Dead()
+          _state = SessionState.Dead
         case _ =>
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/server/src/main/scala/org/apache/livy/server/interactive/CreateInteractiveRequest.scala
----------------------------------------------------------------------
diff --git a/server/src/main/scala/org/apache/livy/server/interactive/CreateInteractiveRequest.scala b/server/src/main/scala/org/apache/livy/server/interactive/CreateInteractiveRequest.scala
index 792c59f..b2f34b0 100644
--- a/server/src/main/scala/org/apache/livy/server/interactive/CreateInteractiveRequest.scala
+++ b/server/src/main/scala/org/apache/livy/server/interactive/CreateInteractiveRequest.scala
@@ -20,7 +20,7 @@ package org.apache.livy.server.interactive
 import org.apache.livy.sessions.{Kind, Shared}
 
 class CreateInteractiveRequest {
-  var kind: Kind = Shared()
+  var kind: Kind = Shared
   var proxyUser: Option[String] = None
   var jars: List[String] = List()
   var pyFiles: List[String] = List()

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/server/src/main/scala/org/apache/livy/server/interactive/InteractiveSession.scala
----------------------------------------------------------------------
diff --git a/server/src/main/scala/org/apache/livy/server/interactive/InteractiveSession.scala b/server/src/main/scala/org/apache/livy/server/interactive/InteractiveSession.scala
index d00146b..26c8d93 100644
--- a/server/src/main/scala/org/apache/livy/server/interactive/InteractiveSession.scala
+++ b/server/src/main/scala/org/apache/livy/server/interactive/InteractiveSession.scala
@@ -112,7 +112,7 @@ object InteractiveSession extends Logging {
       None,
       appTag,
       client,
-      SessionState.Starting(),
+      SessionState.Starting,
       request.kind,
       request.heartbeatTimeoutInSecond,
       livyConf,
@@ -138,7 +138,7 @@ object InteractiveSession extends Logging {
       metadata.appId,
       metadata.appTag,
       client,
-      SessionState.Recovering(),
+      SessionState.Recovering,
       metadata.kind,
       metadata.heartbeatTimeoutS,
       livyConf,
@@ -398,7 +398,7 @@ class InteractiveSession(
   }
 
   if (client.isEmpty) {
-    transition(Dead())
+    transition(Dead)
     val msg = s"Cannot recover interactive session $id because its RSCDriver URI is unknown."
     info(msg)
     sessionLog = IndexedSeq(msg)
@@ -424,7 +424,7 @@ class InteractiveSession(
       override def onJobFailed(job: JobHandle[Void], cause: Throwable): Unit = errorOut()
 
       override def onJobSucceeded(job: JobHandle[Void], result: Void): Unit = {
-        transition(SessionState.Running())
+        transition(SessionState.Running)
         info(s"Interactive session $id created [appid: ${appId.orNull}, owner: $owner, proxyUser:" +
           s" $proxyUser, state: ${state.toString}, kind: ${kind.toString}, " +
           s"info: ${appInfo.asJavaMap}]")
@@ -434,8 +434,8 @@ class InteractiveSession(
         // Other code might call stop() to close the RPC channel. When RPC channel is closing,
         // this callback might be triggered. Check and don't call stop() to avoid nested called
         // if the session is already shutting down.
-        if (serverSideState != SessionState.ShuttingDown()) {
-          transition(SessionState.Error())
+        if (serverSideState != SessionState.ShuttingDown) {
+          transition(SessionState.Error)
           stop()
           app.foreach { a =>
             info(s"Failed to ping RSC driver for session $id. Killing application.")
@@ -453,20 +453,18 @@ class InteractiveSession(
       id, appId, appTag, kind, heartbeatTimeout.toSeconds.toInt, owner, proxyUser, rscDriverUri)
 
   override def state: SessionState = {
-    if (serverSideState.isInstanceOf[SessionState.Running]) {
+    if (serverSideState == SessionState.Running) {
       // If session is in running state, return the repl state from RSCClient.
       client
         .flatMap(s => Option(s.getReplState))
         .map(SessionState(_))
-        .getOrElse(SessionState.Busy()) // If repl state is unknown, assume repl is busy.
-    } else {
-      serverSideState
-    }
+        .getOrElse(SessionState.Busy) // If repl state is unknown, assume repl is busy.
+    } else serverSideState
   }
 
   override def stopSession(): Unit = {
     try {
-      transition(SessionState.ShuttingDown())
+      transition(SessionState.ShuttingDown)
       sessionStore.remove(RECOVERY_SESSION_TYPE, id)
       client.foreach { _.stop(true) }
     } catch {
@@ -476,7 +474,7 @@ class InteractiveSession(
           _.kill()
         }
     } finally {
-      transition(SessionState.Dead())
+      transition(SessionState.Dead)
     }
   }
 
@@ -586,7 +584,7 @@ class InteractiveSession(
 
   private def ensureRunning(): Unit = synchronized {
     serverSideState match {
-      case SessionState.Running() =>
+      case SessionState.Running =>
       case _ =>
         throw new IllegalStateException("Session is in state %s" format serverSideState)
     }
@@ -613,7 +611,7 @@ class InteractiveSession(
       debug(s"$this app state changed from $oldState to $newState")
       newState match {
         case SparkApp.State.FINISHED | SparkApp.State.KILLED | SparkApp.State.FAILED =>
-          transition(SessionState.Dead())
+          transition(SessionState.Dead)
         case _ =>
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/server/src/test/scala/org/apache/livy/server/SessionServletSpec.scala
----------------------------------------------------------------------
diff --git a/server/src/test/scala/org/apache/livy/server/SessionServletSpec.scala b/server/src/test/scala/org/apache/livy/server/SessionServletSpec.scala
index 2410fe2..f07c296 100644
--- a/server/src/test/scala/org/apache/livy/server/SessionServletSpec.scala
+++ b/server/src/test/scala/org/apache/livy/server/SessionServletSpec.scala
@@ -40,7 +40,7 @@ object SessionServletSpec {
 
     override def recoveryMetadata: RecoveryMetadata = MockRecoveryMetadata(0)
 
-    override def state: SessionState = SessionState.Idle()
+    override def state: SessionState = SessionState.Idle
 
     override protected def stopSession(): Unit = ()
 

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/server/src/test/scala/org/apache/livy/server/batch/BatchServletSpec.scala
----------------------------------------------------------------------
diff --git a/server/src/test/scala/org/apache/livy/server/batch/BatchServletSpec.scala b/server/src/test/scala/org/apache/livy/server/batch/BatchServletSpec.scala
index 3402cff..2c37c19 100644
--- a/server/src/test/scala/org/apache/livy/server/batch/BatchServletSpec.scala
+++ b/server/src/test/scala/org/apache/livy/server/batch/BatchServletSpec.scala
@@ -123,7 +123,7 @@ class BatchServletSpec extends BaseSessionServletSpec[BatchSession, BatchRecover
 
     it("should show session properties") {
       val id = 0
-      val state = SessionState.Running()
+      val state = SessionState.Running
       val appId = "appid"
       val appInfo = AppInfo(Some("DRIVER LOG URL"), Some("SPARK UI URL"))
       val log = IndexedSeq[String]("log1", "log2")

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/server/src/test/scala/org/apache/livy/server/batch/BatchSessionSpec.scala
----------------------------------------------------------------------
diff --git a/server/src/test/scala/org/apache/livy/server/batch/BatchSessionSpec.scala b/server/src/test/scala/org/apache/livy/server/batch/BatchSessionSpec.scala
index eb80bef..196d328 100644
--- a/server/src/test/scala/org/apache/livy/server/batch/BatchSessionSpec.scala
+++ b/server/src/test/scala/org/apache/livy/server/batch/BatchSessionSpec.scala
@@ -103,7 +103,7 @@ class BatchSessionSpec
       val m = BatchRecoveryMetadata(99, None, "appTag", null, None)
       val batch = BatchSession.recover(m, conf, sessionStore, Some(mockApp))
 
-      batch.state shouldBe a[SessionState.Recovering]
+      batch.state shouldBe (SessionState.Recovering)
 
       batch.appIdKnown("appId")
       verify(sessionStore, atLeastOnce()).save(

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/server/src/test/scala/org/apache/livy/server/interactive/BaseInteractiveServletSpec.scala
----------------------------------------------------------------------
diff --git a/server/src/test/scala/org/apache/livy/server/interactive/BaseInteractiveServletSpec.scala b/server/src/test/scala/org/apache/livy/server/interactive/BaseInteractiveServletSpec.scala
index b401a92..70e8653 100644
--- a/server/src/test/scala/org/apache/livy/server/interactive/BaseInteractiveServletSpec.scala
+++ b/server/src/test/scala/org/apache/livy/server/interactive/BaseInteractiveServletSpec.scala
@@ -57,7 +57,7 @@ abstract class BaseInteractiveServletSpec
   protected def createRequest(
       inProcess: Boolean = true,
       extraConf: Map[String, String] = Map(),
-      kind: Kind = Spark()): CreateInteractiveRequest = {
+      kind: Kind = Spark): CreateInteractiveRequest = {
     val classpath = sys.props("java.class.path")
     val request = new CreateInteractiveRequest()
     request.kind = kind

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/server/src/test/scala/org/apache/livy/server/interactive/CreateInteractiveRequestSpec.scala
----------------------------------------------------------------------
diff --git a/server/src/test/scala/org/apache/livy/server/interactive/CreateInteractiveRequestSpec.scala b/server/src/test/scala/org/apache/livy/server/interactive/CreateInteractiveRequestSpec.scala
index a67c725..b84d98a 100644
--- a/server/src/test/scala/org/apache/livy/server/interactive/CreateInteractiveRequestSpec.scala
+++ b/server/src/test/scala/org/apache/livy/server/interactive/CreateInteractiveRequestSpec.scala
@@ -34,7 +34,7 @@ class CreateInteractiveRequestSpec extends FunSpec with LivyBaseUnitTestSuite {
     it("should have default values for fields after deserialization") {
       val json = """{ "kind" : "pyspark" }"""
       val req = mapper.readValue(json, classOf[CreateInteractiveRequest])
-      assert(req.kind === PySpark())
+      assert(req.kind === PySpark)
       assert(req.proxyUser === None)
       assert(req.jars === List())
       assert(req.pyFiles === List())

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/server/src/test/scala/org/apache/livy/server/interactive/InteractiveSessionServletSpec.scala
----------------------------------------------------------------------
diff --git a/server/src/test/scala/org/apache/livy/server/interactive/InteractiveSessionServletSpec.scala b/server/src/test/scala/org/apache/livy/server/interactive/InteractiveSessionServletSpec.scala
index d42d78c..e946fc0 100644
--- a/server/src/test/scala/org/apache/livy/server/interactive/InteractiveSessionServletSpec.scala
+++ b/server/src/test/scala/org/apache/livy/server/interactive/InteractiveSessionServletSpec.scala
@@ -59,11 +59,11 @@ class InteractiveSessionServletSpec extends BaseInteractiveServletSpec {
       val statementCounter = new AtomicInteger()
 
       val session = mock[InteractiveSession]
-      when(session.kind).thenReturn(Spark())
+      when(session.kind).thenReturn(Spark)
       when(session.appId).thenReturn(None)
       when(session.appInfo).thenReturn(AppInfo())
       when(session.logLines()).thenReturn(IndexedSeq())
-      when(session.state).thenReturn(SessionState.Idle())
+      when(session.state).thenReturn(SessionState.Idle)
       when(session.stop()).thenReturn(Future.successful(()))
       when(session.proxyUser).thenReturn(None)
       when(session.statements).thenAnswer(
@@ -156,8 +156,8 @@ class InteractiveSessionServletSpec extends BaseInteractiveServletSpec {
     val appId = "appid"
     val owner = "owner"
     val proxyUser = "proxyUser"
-    val state = SessionState.Running()
-    val kind = Spark()
+    val state = SessionState.Running
+    val kind = Spark
     val appInfo = AppInfo(Some("DRIVER LOG URL"), Some("SPARK UI URL"))
     val log = IndexedSeq[String]("log1", "log2")
 

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/server/src/test/scala/org/apache/livy/server/interactive/InteractiveSessionSpec.scala
----------------------------------------------------------------------
diff --git a/server/src/test/scala/org/apache/livy/server/interactive/InteractiveSessionSpec.scala b/server/src/test/scala/org/apache/livy/server/interactive/InteractiveSessionSpec.scala
index 9943c00..39790f7 100644
--- a/server/src/test/scala/org/apache/livy/server/interactive/InteractiveSessionSpec.scala
+++ b/server/src/test/scala/org/apache/livy/server/interactive/InteractiveSessionSpec.scala
@@ -58,7 +58,7 @@ class InteractiveSessionSpec extends FunSpec
     assume(sys.env.get("SPARK_HOME").isDefined, "SPARK_HOME is not set.")
 
     val req = new CreateInteractiveRequest()
-    req.kind = PySpark()
+    req.kind = PySpark
     req.driverMemory = Some("512m")
     req.driverCores = Some(1)
     req.executorMemory = Some("512m")
@@ -92,7 +92,7 @@ class InteractiveSessionSpec extends FunSpec
     it(desc) {
       assume(session != null, "No active session.")
       eventually(timeout(30 seconds), interval(100 millis)) {
-        session.state shouldBe a[SessionState.Idle]
+        session.state shouldBe (SessionState.Idle)
       }
       fn(session)
     }
@@ -113,14 +113,14 @@ class InteractiveSessionSpec extends FunSpec
         .set(LivyConf.REPL_JARS, testedJars.mkString(","))
         .set(LivyConf.LIVY_SPARK_VERSION, "1.6.2")
         .set(LivyConf.LIVY_SPARK_SCALA_VERSION, "2.10")
-      val properties = InteractiveSession.prepareBuilderProp(Map.empty, Spark(), livyConf)
+      val properties = InteractiveSession.prepareBuilderProp(Map.empty, Spark, livyConf)
       assert(properties(LivyConf.SPARK_JARS).split(",").toSet === Set("test_2.10-0.1.jar",
         "local://dummy-path/test/test1_2.10-1.0.jar",
         "hdfs:///dummy-path/test/test3.jar",
         "dummy.jar"))
 
       livyConf.set(LivyConf.LIVY_SPARK_SCALA_VERSION, "2.11")
-      val properties1 = InteractiveSession.prepareBuilderProp(Map.empty, Spark(), livyConf)
+      val properties1 = InteractiveSession.prepareBuilderProp(Map.empty, Spark, livyConf)
       assert(properties1(LivyConf.SPARK_JARS).split(",").toSet === Set(
         "file:///dummy-path/test/test2_2.11-1.0-SNAPSHOT.jar",
         "hdfs:///dummy-path/test/test3.jar",
@@ -139,7 +139,7 @@ class InteractiveSessionSpec extends FunSpec
         .set(LivyConf.RSC_JARS, rscJars.mkString(","))
         .set(LivyConf.LIVY_SPARK_VERSION, "1.6.2")
         .set(LivyConf.LIVY_SPARK_SCALA_VERSION, "2.10")
-      val properties = InteractiveSession.prepareBuilderProp(Map.empty, Spark(), livyConf)
+      val properties = InteractiveSession.prepareBuilderProp(Map.empty, Spark, livyConf)
       // if livy.rsc.jars is configured in LivyConf, it should be passed to RSCConf.
       properties(RSCConf.Entry.LIVY_JARS.key()).split(",").toSet === rscJars
 
@@ -149,7 +149,7 @@ class InteractiveSessionSpec extends FunSpec
         "file:///dummy-path/foo2.jar",
         "hdfs:///dummy-path/foo3.jar")
       val properties1 = InteractiveSession.prepareBuilderProp(
-        Map(RSCConf.Entry.LIVY_JARS.key() -> rscJars1.mkString(",")), Spark(), livyConf)
+        Map(RSCConf.Entry.LIVY_JARS.key() -> rscJars1.mkString(",")), Spark, livyConf)
       // if rsc jars are configured both in LivyConf and RSCConf, RSCConf should take precedence.
       properties1(RSCConf.Entry.LIVY_JARS.key()).split(",").toSet === rscJars1
     }
@@ -170,7 +170,7 @@ class InteractiveSessionSpec extends FunSpec
       verify(sessionStore, atLeastOnce()).save(
         MockitoMatchers.eq(InteractiveSession.RECOVERY_SESSION_TYPE), anyObject())
 
-      session.state should (be(a[SessionState.Starting]) or be(a[SessionState.Idle]))
+      session.state should (be(SessionState.Starting) or be(SessionState.Idle))
     }
 
     withSession("should execute `1 + 2` == 3") { session =>
@@ -211,7 +211,7 @@ class InteractiveSessionSpec extends FunSpec
 
       result should equal (expectedResult)
       eventually(timeout(10 seconds), interval(30 millis)) {
-        session.state shouldBe a[SessionState.Idle]
+        session.state shouldBe (SessionState.Idle)
       }
     }
 
@@ -247,10 +247,10 @@ class InteractiveSessionSpec extends FunSpec
       when(mockClient.submit(any(classOf[PingJob]))).thenReturn(mock[JobHandle[Void]])
       val m =
         InteractiveRecoveryMetadata(
-          78, None, "appTag", Spark(), 0, null, None, Some(URI.create("")))
+          78, None, "appTag", Spark, 0, null, None, Some(URI.create("")))
       val s = InteractiveSession.recover(m, conf, sessionStore, None, Some(mockClient))
 
-      s.state shouldBe a[SessionState.Recovering]
+      s.state shouldBe (SessionState.Recovering)
 
       s.appIdKnown("appId")
       verify(sessionStore, atLeastOnce()).save(
@@ -261,7 +261,7 @@ class InteractiveSessionSpec extends FunSpec
       val conf = new LivyConf()
       val sessionStore = mock[SessionStore]
       val m = InteractiveRecoveryMetadata(
-        78, Some("appId"), "appTag", Spark(), 0, null, None, None)
+        78, Some("appId"), "appTag", Spark, 0, null, None, None)
       val s = InteractiveSession.recover(m, conf, sessionStore, None)
 
       s.state shouldBe a[SessionState.Dead]

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/server/src/test/scala/org/apache/livy/server/interactive/JobApiSpec.scala
----------------------------------------------------------------------
diff --git a/server/src/test/scala/org/apache/livy/server/interactive/JobApiSpec.scala b/server/src/test/scala/org/apache/livy/server/interactive/JobApiSpec.scala
index d575b54..36a46ca 100644
--- a/server/src/test/scala/org/apache/livy/server/interactive/JobApiSpec.scala
+++ b/server/src/test/scala/org/apache/livy/server/interactive/JobApiSpec.scala
@@ -169,7 +169,7 @@ class JobApiSpec extends BaseInteractiveServletSpec {
   private def waitForIdle(id: Int): Unit = {
     eventually(timeout(1 minute), interval(100 millis)) {
       jget[SessionInfo](s"/$id") { status =>
-        status.state should be (SessionState.Idle().toString())
+        status.state should be (SessionState.Idle.toString())
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/server/src/test/scala/org/apache/livy/sessions/MockSession.scala
----------------------------------------------------------------------
diff --git a/server/src/test/scala/org/apache/livy/sessions/MockSession.scala b/server/src/test/scala/org/apache/livy/sessions/MockSession.scala
index 3cfbe46..1604f4d 100644
--- a/server/src/test/scala/org/apache/livy/sessions/MockSession.scala
+++ b/server/src/test/scala/org/apache/livy/sessions/MockSession.scala
@@ -28,7 +28,7 @@ class MockSession(id: Int, owner: String, conf: LivyConf) extends Session(id, ow
 
   override def logLines(): IndexedSeq[String] = IndexedSeq()
 
-  override def state: SessionState = SessionState.Idle()
+  override def state: SessionState = SessionState.Idle
 
   override def recoveryMetadata: RecoveryMetadata = RecoveryMetadata(0)
 }

http://git-wip-us.apache.org/repos/asf/incubator-livy/blob/54715447/server/src/test/scala/org/apache/livy/sessions/SessionManagerSpec.scala
----------------------------------------------------------------------
diff --git a/server/src/test/scala/org/apache/livy/sessions/SessionManagerSpec.scala b/server/src/test/scala/org/apache/livy/sessions/SessionManagerSpec.scala
index beffa71..69ec817 100644
--- a/server/src/test/scala/org/apache/livy/sessions/SessionManagerSpec.scala
+++ b/server/src/test/scala/org/apache/livy/sessions/SessionManagerSpec.scala
@@ -88,19 +88,19 @@ class SessionManagerSpec extends FunSpec with Matchers with LivyBaseUnitTestSuit
       }
 
       // Batch session should not be gc-ed when alive
-      for (s <- Seq(SessionState.Running(),
-        SessionState.Idle(),
-        SessionState.Recovering(),
-        SessionState.NotStarted(),
-        SessionState.Busy(),
-        SessionState.ShuttingDown())) {
+      for (s <- Seq(SessionState.Running,
+        SessionState.Idle,
+        SessionState.Recovering,
+        SessionState.NotStarted,
+        SessionState.Busy,
+        SessionState.ShuttingDown)) {
         changeStateAndCheck(s) { sm => sm.get(session.id) should be (Some(session)) }
       }
 
       // Stopped session should be gc-ed after retained timeout
-      for (s <- Seq(SessionState.Error(),
-        SessionState.Success(),
-        SessionState.Dead())) {
+      for (s <- Seq(SessionState.Error,
+        SessionState.Success,
+        SessionState.Dead)) {
         eventually(timeout(30 seconds), interval(100 millis)) {
           changeStateAndCheck(s) { sm => sm.get(session.id) should be (None) }
         }