Browse Source

[livy] Rename server.interactive.*Session* to *InteractiveSession*

Erick Tryzelaar 10 years ago
parent
commit
ec9ceff
18 changed files with 205 additions and 135 deletions
  1. 5 5
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/Main.scala
  2. 2 2
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSession.scala
  3. 31 0
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionFactory.scala
  4. 7 7
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionManager.scala
  5. 5 5
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionProcess.scala
  6. 35 0
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionProcessFactory.scala
  7. 8 8
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionServlet.scala
  8. 5 5
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionThread.scala
  9. 35 0
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionThreadFactory.scala
  10. 5 5
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionYarn.scala
  11. 42 0
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionYarnFactory.scala
  12. 2 2
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveWebSession.scala
  13. 0 70
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/SessionFactory.scala
  14. 4 5
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/interactive/BaseSessionSpec.scala
  15. 4 4
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionProcessSpec.scala
  16. 8 9
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionServletSpec.scala
  17. 4 4
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionThreadSpec.scala
  18. 3 4
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/interactive/StatementSpec.scala

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

@@ -20,7 +20,7 @@ package com.cloudera.hue.livy.server
 
 import javax.servlet.ServletContext
 
-import com.cloudera.hue.livy.server.batch.{BatchYarnFactory, BatchProcessFactory, BatchServlet, BatchManager}
+import com.cloudera.hue.livy.server.batch.{BatchServlet, BatchManager, BatchYarnFactory, BatchProcessFactory}
 import com.cloudera.hue.livy.server.interactive._
 import com.cloudera.hue.livy.{Utils, Logging, LivyConf, WebServer}
 import org.scalatra._
@@ -80,17 +80,17 @@ class ScalatraBootstrap extends LifeCycle with Logging {
 
     val (sessionFactory, batchFactory) = sessionFactoryKind match {
       case LivyConf.Thread() =>
-        (new ThreadSessionFactory(livyConf), new BatchProcessFactory(livyConf) )
+        (new InteractiveSessionThreadFactory(livyConf), new BatchProcessFactory(livyConf) )
       case LivyConf.Process() =>
-        (new ProcessSessionFactory(livyConf), new BatchProcessFactory(livyConf))
+        (new InteractiveSessionProcessFactory(livyConf), new BatchProcessFactory(livyConf))
       case LivyConf.Yarn() =>
-        (new YarnSessionFactory(livyConf), new BatchYarnFactory(livyConf))
+        (new InteractiveSessionYarnFactory(livyConf), new BatchYarnFactory(livyConf))
     }
 
     sessionManager = new SessionManager(sessionFactory)
     batchManager = new BatchManager(batchFactory)
 
-    context.mount(new SessionServlet(sessionManager), "/sessions/*")
+    context.mount(new InteractiveSessionServlet(sessionManager), "/sessions/*")
     context.mount(new BatchServlet(batchManager), "/batches/*")
   }
 

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

@@ -28,13 +28,13 @@ import com.cloudera.hue.livy.sessions.{Kind, State}
 import scala.concurrent._
 import scala.concurrent.duration.Duration
 
-object Session {
+object InteractiveSession {
   class SessionFailedToStart(msg: String) extends Exception(msg)
 
   class StatementNotFound extends Exception
 }
 
-trait Session {
+trait InteractiveSession {
   def id: Int
 
   def kind: Kind

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

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

+ 7 - 7
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/SessionManager.scala → apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionManager.scala

@@ -36,17 +36,17 @@ object SessionManager {
   val GC_PERIOD = 1000 * 60 * 60
 }
 
-class SessionManager(factory: SessionFactory) extends Logging {
+class SessionManager(factory: InteractiveSessionFactory) extends Logging {
 
   private implicit def executor: ExecutionContextExecutor = ExecutionContext.global
 
   private[this] val _idCounter = new AtomicInteger()
-  private[this] val sessions = new ConcurrentHashMap[Int, Session]()
+  private[this] val sessions = new ConcurrentHashMap[Int, InteractiveSession]()
 
   private val garbageCollector = new GarbageCollector(this)
   garbageCollector.start()
 
-  def get(sessionId: Int): Option[Session] = {
+  def get(sessionId: Int): Option[InteractiveSession] = {
     Option(sessions.get(sessionId))
   }
 
@@ -58,11 +58,11 @@ class SessionManager(factory: SessionFactory) extends Logging {
     sessions.keys
   }
 
-  def createSession(kind: Kind, proxyUser: Option[String] = None): Future[Session] = {
+  def createSession(kind: Kind, proxyUser: Option[String] = None): Future[InteractiveSession] = {
     val id = _idCounter.getAndIncrement
     val session = factory.createSession(id, kind, proxyUser = proxyUser)
 
-    session.map({ case(session: Session) =>
+    session.map({ case(session: InteractiveSession) =>
       info("created session %s" format session.id)
       sessions.put(session.id, session)
       session
@@ -81,7 +81,7 @@ class SessionManager(factory: SessionFactory) extends Logging {
     }
   }
 
-  def delete(session: Session): Future[Unit] = {
+  def delete(session: InteractiveSession): Future[Unit] = {
     session.stop().map { case _ =>
         sessions.remove(session.id)
         Unit
@@ -89,7 +89,7 @@ class SessionManager(factory: SessionFactory) extends Logging {
   }
 
   def collectGarbage() = {
-    def expired(session: Session): Boolean = {
+    def expired(session: InteractiveSession): Boolean = {
       System.currentTimeMillis() - session.lastActivity > SessionManager.TIMEOUT
     }
 

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

@@ -32,15 +32,15 @@ import scala.collection.mutable.ArrayBuffer
 import scala.concurrent.Future
 import scala.io.Source
 
-object ProcessSession extends Logging {
+object InteractiveSessionProcess extends Logging {
 
   val CONF_LIVY_REPL_JAR = "livy.repl.jar"
   val CONF_LIVY_REPL_CALLBACK_URL = "livy.repl.callback-url"
   val CONF_LIVY_REPL_DRIVER_CLASS_PATH = "livy.repl.driverClassPath"
 
-  def create(livyConf: LivyConf, id: Int, kind: Kind, proxyUser: Option[String] = None): Session = {
+  def create(livyConf: LivyConf, id: Int, kind: Kind, proxyUser: Option[String] = None): InteractiveSession = {
     val process = startProcess(livyConf, id, kind, proxyUser)
-    new ProcessSession(id, kind, proxyUser, process)
+    new InteractiveSessionProcess(id, kind, proxyUser, process)
   }
 
   // Loop until we've started a process with a valid port.
@@ -73,10 +73,10 @@ object ProcessSession extends Logging {
   }
 }
 
-private class ProcessSession(id: Int,
+private class InteractiveSessionProcess(id: Int,
                              kind: Kind,
                              proxyUser: Option[String],
-                             process: Process) extends WebSession(id, kind, proxyUser) {
+                             process: Process) extends InteractiveWebSession(id, kind, proxyUser) {
 
   val stdoutThread = new Thread {
     override def run() = {

+ 35 - 0
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionProcessFactory.scala

@@ -0,0 +1,35 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.cloudera.hue.livy.server.interactive
+
+import com.cloudera.hue.livy.LivyConf
+import com.cloudera.hue.livy.sessions.Kind
+
+import scala.concurrent.{Future, ExecutionContext}
+
+class InteractiveSessionProcessFactory(livyConf: LivyConf) extends InteractiveSessionFactory {
+
+   implicit def executor: ExecutionContext = ExecutionContext.global
+
+   override def createSession(id: Int, kind: Kind, proxyUser: Option[String] = None): Future[InteractiveSession] = {
+     Future {
+       InteractiveSessionProcess.create(livyConf, id, kind, proxyUser)
+     }
+   }
+ }

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

@@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit
 
 import com.cloudera.hue.livy.Logging
 import com.cloudera.hue.livy.msgs.ExecuteRequest
-import Session.SessionFailedToStart
+import InteractiveSession.SessionFailedToStart
 import com.cloudera.hue.livy.sessions._
 import com.fasterxml.jackson.core.JsonParseException
 import org.json4s.JsonAST.JString
@@ -34,9 +34,9 @@ import org.scalatra.json.JacksonJsonSupport
 import scala.concurrent._
 import scala.concurrent.duration._
 
-object SessionServlet extends Logging
+object InteractiveSessionServlet extends Logging
 
-class SessionServlet(sessionManager: SessionManager)
+class InteractiveSessionServlet(sessionManager: SessionManager)
   extends ScalatraServlet
   with FutureSupport
   with MethodOverride
@@ -139,7 +139,7 @@ class SessionServlet(sessionManager: SessionManager)
 
     sessionManager.get(sessionId) match {
       case None => NotFound("Session not found")
-      case Some(session: Session) =>
+      case Some(session: InteractiveSession) =>
         val from = params.get("from").map(_.toInt).getOrElse(0)
         val size = params.get("size").map(_.toInt).getOrElse(session.statements.length)
 
@@ -191,7 +191,7 @@ class SessionServlet(sessionManager: SessionManager)
     case e: SessionFailedToStart => InternalServerError(e.getMessage)
     case e: dispatch.StatusCode => ActionResult(ResponseStatus(e.code), e.getMessage, Map.empty)
     case e =>
-      SessionServlet.error("internal error", e)
+      InteractiveSessionServlet.error("internal error", e)
       InternalServerError(e.toString)
   }
 }
@@ -212,7 +212,7 @@ private object Serializers {
 
   private def serializeStatementState(state: Statement.State) = JString(state.toString)
 
-  def serializeSession(session: Session): JValue = {
+  def serializeSession(session: InteractiveSession): JValue = {
     ("id", session.id) ~
       ("state", serializeSessionState(session.state)) ~
       ("kind", serializeSessionKind(session.kind)) ~
@@ -232,11 +232,11 @@ private object Serializers {
       ("output" -> output)
   }
 
-  case object SessionSerializer extends CustomSerializer[Session](implicit formats => ( {
+  case object SessionSerializer extends CustomSerializer[InteractiveSession](implicit formats => ( {
     // We don't support deserialization.
     PartialFunction.empty
   }, {
-    case session: Session =>
+    case session: InteractiveSession =>
       serializeSession(session)
   }
     )

+ 5 - 5
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/ThreadSession.scala → apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionThread.scala

@@ -28,24 +28,24 @@ import com.cloudera.hue.livy.sessions.{Kind, PySpark, Spark, State}
 import scala.collection.mutable.ArrayBuffer
 import scala.concurrent.{ExecutionContext, ExecutionContextExecutor, Future}
 
-object ThreadSession {
+object InteractiveSessionThread {
   val LIVY_HOME = System.getenv("LIVY_HOME")
   val LIVY_REPL = LIVY_HOME + "/bin/livy-repl"
 
-  def create(id: Int, kind: Kind): Session = {
+  def create(id: Int, kind: Kind): InteractiveSession = {
     val session = kind match {
       case Spark() =>
         SparkSession.create()
       case PySpark() =>
         PythonSession.createPySpark()
     }
-    new ThreadSession(id, kind, session)
+    new InteractiveSessionThread(id, kind, session)
   }
 }
 
-private class ThreadSession(val id: Int,
+private class InteractiveSessionThread(val id: Int,
                             val kind: Kind,
-                            session: com.cloudera.hue.livy.repl.Session) extends Session {
+                            session: com.cloudera.hue.livy.repl.Session) extends InteractiveSession {
 
   protected implicit def executor: ExecutionContextExecutor = ExecutionContext.global
 

+ 35 - 0
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionThreadFactory.scala

@@ -0,0 +1,35 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.cloudera.hue.livy.server.interactive
+
+import com.cloudera.hue.livy.LivyConf
+import com.cloudera.hue.livy.sessions.Kind
+
+import scala.concurrent.{Future, ExecutionContext}
+
+class InteractiveSessionThreadFactory(livyConf: LivyConf) extends InteractiveSessionFactory {
+
+   implicit def executor: ExecutionContext = ExecutionContext.global
+
+   override def createSession(id: Int, kind: Kind, proxyUser: Option[String] = None): Future[InteractiveSession] = {
+     Future {
+       InteractiveSessionThread.create(id, kind)
+     }
+   }
+ }

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

@@ -30,13 +30,13 @@ import com.cloudera.hue.livy.yarn.{Client, Job}
 import scala.concurrent.{Await, ExecutionContext, ExecutionContextExecutor, Future}
 import scala.concurrent.duration._
 
-object YarnSession {
+object InteractiveSessionYarn {
   protected implicit def executor: ExecutionContextExecutor = ExecutionContext.global
 
   private val CONF_LIVY_JAR = "livy.yarn.jar"
   private lazy val regex = """Application report for (\w+)""".r.unanchored
 
-  def create(livyConf: LivyConf, client: Client, id: Int, kind: Kind, proxyUser: Option[String] = None): Session = {
+  def create(livyConf: LivyConf, client: Client, id: Int, kind: Kind, proxyUser: Option[String] = None): InteractiveSession = {
     val callbackUrl = System.getProperty("livy.server.callback-url")
     val url = f"$callbackUrl/sessions/$id/callback"
 
@@ -62,7 +62,7 @@ object YarnSession {
       job
     }
 
-    new YarnSession(id, kind, proxyUser, job)
+    new InteractiveSessionYarn(id, kind, proxyUser, job)
   }
 
   private def livyJar(livyConf: LivyConf) = {
@@ -74,10 +74,10 @@ object YarnSession {
   }
 }
 
-private class YarnSession(id: Int,
+private class InteractiveSessionYarn(id: Int,
                           kind: Kind,
                           proxyUser: Option[String],
-                          job: Future[Job]) extends WebSession(id, kind, proxyUser) {
+                          job: Future[Job]) extends InteractiveWebSession(id, kind, proxyUser) {
   job.onFailure { case _ =>
     _state = Error()
   }

+ 42 - 0
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionYarnFactory.scala

@@ -0,0 +1,42 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.cloudera.hue.livy.server.interactive
+
+import com.cloudera.hue.livy.LivyConf
+import com.cloudera.hue.livy.sessions.Kind
+import com.cloudera.hue.livy.yarn.Client
+
+import scala.concurrent.{Future, ExecutionContext}
+
+class InteractiveSessionYarnFactory(livyConf: LivyConf) extends InteractiveSessionFactory {
+
+   implicit def executor: ExecutionContext = ExecutionContext.global
+
+   val client = new Client(livyConf)
+
+   override def createSession(id: Int, kind: Kind, proxyUser: Option[String] = None): Future[InteractiveSession] = {
+     Future {
+       InteractiveSessionYarn.create(livyConf, client, id, kind, proxyUser)
+     }
+   }
+
+   override def close(): Unit = {
+     client.close()
+   }
+ }

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

@@ -32,9 +32,9 @@ import scala.collection.mutable.ArrayBuffer
 import scala.concurrent.duration.Duration
 import scala.concurrent.{Future, _}
 
-class WebSession(val id: Int,
+class InteractiveWebSession(val id: Int,
                  val kind: Kind,
-                 val proxyUser: Option[String]) extends Session with Logging {
+                 val proxyUser: Option[String]) extends InteractiveSession with Logging {
 
   protected implicit def executor: ExecutionContextExecutor = ExecutionContext.global
   protected implicit def jsonFormats: Formats = DefaultFormats

+ 0 - 70
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/SessionFactory.scala

@@ -1,70 +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.server.interactive
-
-import com.cloudera.hue.livy.LivyConf
-import com.cloudera.hue.livy.sessions.Kind
-import com.cloudera.hue.livy.yarn.Client
-
-import scala.concurrent.{ExecutionContext, Future}
-
-trait SessionFactory {
-  def createSession(id: Int, kind: Kind, proxyUser: Option[String] = None): Future[Session]
-
-  def close(): Unit = {}
-}
-
-class ThreadSessionFactory(livyConf: LivyConf) extends SessionFactory {
-
-  implicit def executor: ExecutionContext = ExecutionContext.global
-
-  override def createSession(id: Int, kind: Kind, proxyUser: Option[String] = None): Future[Session] = {
-    Future {
-      ThreadSession.create(id, kind)
-    }
-  }
-}
-
-class ProcessSessionFactory(livyConf: LivyConf) extends SessionFactory {
-
-  implicit def executor: ExecutionContext = ExecutionContext.global
-
-  override def createSession(id: Int, kind: Kind, proxyUser: Option[String] = None): Future[Session] = {
-    Future {
-      ProcessSession.create(livyConf, id, kind, proxyUser)
-    }
-  }
-}
-
-class YarnSessionFactory(livyConf: LivyConf) extends SessionFactory {
-
-  implicit def executor: ExecutionContext = ExecutionContext.global
-
-  val client = new Client(livyConf)
-
-  override def createSession(id: Int, kind: Kind, proxyUser: Option[String] = None): Future[Session] = {
-    Future {
-      YarnSession.create(livyConf, client, id, kind, proxyUser)
-    }
-  }
-
-  override def close(): Unit = {
-    client.close()
-  }
-}

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

@@ -16,13 +16,12 @@
  * limitations under the License.
  */
 
-package com.cloudera.hue.livy.server
+package com.cloudera.hue.livy.server.interactive
 
 import java.util.concurrent.TimeUnit
 
 import com.cloudera.hue.livy.msgs.ExecuteRequest
-import com.cloudera.hue.livy.server.interactive.Session
-import com.cloudera.hue.livy.sessions.{Starting, Idle}
+import com.cloudera.hue.livy.sessions.{Idle, Starting}
 import org.json4s.{DefaultFormats, Extraction}
 import org.scalatest.{BeforeAndAfter, FunSpec, Matchers}
 
@@ -33,9 +32,9 @@ abstract class BaseSessionSpec extends FunSpec with Matchers with BeforeAndAfter
 
   implicit val formats = DefaultFormats
 
-  var session: Session = null
+  var session: InteractiveSession = null
 
-  def createSession(): Session
+  def createSession(): InteractiveSession
 
   before {
     session = createSession()

+ 4 - 4
apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/ProcessSessionSpec.scala → apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionProcessSpec.scala

@@ -16,17 +16,17 @@
  * limitations under the License.
  */
 
-package com.cloudera.hue.livy.server
+package com.cloudera.hue.livy.server.interactive
 
 import com.cloudera.hue.livy.LivyConf
-import com.cloudera.hue.livy.server.interactive.ProcessSession
+import com.cloudera.hue.livy.server.interactive.InteractiveSessionProcess
 import com.cloudera.hue.livy.sessions.Spark
 import org.scalatest.{BeforeAndAfter, FunSpecLike, Matchers}
 
-class ProcessSessionSpec extends BaseSessionSpec with FunSpecLike with Matchers with BeforeAndAfter {
+class InteractiveSessionProcessSpec extends BaseSessionSpec with FunSpecLike with Matchers with BeforeAndAfter {
 
   val livyConf = new LivyConf()
   livyConf.set("livy.repl.driverClassPath", sys.props("java.class.path"))
 
-  def createSession() = ProcessSession.create(livyConf, 0, Spark())
+  def createSession() = InteractiveSessionProcess.create(livyConf, 0, Spark())
 }

+ 8 - 9
apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/SessionServletSpec.scala → apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionServletSpec.scala

@@ -16,13 +16,12 @@
  * limitations under the License.
  */
 
-package com.cloudera.hue.livy.server
+package com.cloudera.hue.livy.server.interactive
 
 import java.net.URL
 import java.util.concurrent.atomic.AtomicInteger
 
 import com.cloudera.hue.livy.msgs.ExecuteRequest
-import com.cloudera.hue.livy.server.interactive._
 import com.cloudera.hue.livy.sessions._
 import org.json4s.JsonAST.{JArray, JObject}
 import org.json4s.jackson.JsonMethods._
@@ -31,9 +30,9 @@ import org.scalatra.test.scalatest.ScalatraSuite
 
 import scala.concurrent.Future
 
-class SessionServletSpec extends ScalatraSuite with FunSpecLike {
+class InteractiveSessionServletSpec extends ScalatraSuite with FunSpecLike {
 
-  class MockSession(val id: Int) extends Session {
+  class MockInteractiveSession(val id: Int) extends InteractiveSession {
     var _state: State = Idle()
 
     var _idCounter = new AtomicInteger()
@@ -70,14 +69,14 @@ class SessionServletSpec extends ScalatraSuite with FunSpecLike {
     override def interrupt(): Future[Unit] = ???
   }
 
-  class MockSessionFactory() extends SessionFactory {
-    override def createSession(id: Int, kind: Kind, proxyUser: Option[String]): Future[Session] = {
-      Future.successful(new MockSession(id))
+  class MockInteractiveSessionFactory() extends InteractiveSessionFactory {
+    override def createSession(id: Int, kind: Kind, proxyUser: Option[String]): Future[InteractiveSession] = {
+      Future.successful(new MockInteractiveSession(id))
     }
   }
 
-  val sessionManager = new SessionManager(new MockSessionFactory())
-  val servlet = new SessionServlet(sessionManager)
+  val sessionManager = new SessionManager(new MockInteractiveSessionFactory())
+  val servlet = new InteractiveSessionServlet(sessionManager)
 
   addServlet(servlet, "/*")
 

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

@@ -16,13 +16,13 @@
  * limitations under the License.
  */
 
-package com.cloudera.hue.livy.server
+package com.cloudera.hue.livy.server.interactive
 
-import com.cloudera.hue.livy.server.interactive.ThreadSession
+import com.cloudera.hue.livy.server.interactive.BaseSessionSpec
 import com.cloudera.hue.livy.sessions.Spark
 import org.scalatest.{BeforeAndAfter, FunSpecLike, Matchers}
 
-class ThreadSessionSpec extends BaseSessionSpec with FunSpecLike with Matchers with BeforeAndAfter {
+class InteractiveSessionThreadSpec extends BaseSessionSpec with FunSpecLike with Matchers with BeforeAndAfter {
 
-  def createSession() = ThreadSession.create(0, Spark())
+  def createSession() = InteractiveSessionThread.create(0, Spark())
 }

+ 3 - 4
apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/StatementSpec.scala → apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/interactive/StatementSpec.scala

@@ -18,13 +18,12 @@
  * limitations under the License.
  */
 
-package com.cloudera.hue.livy.server
+package com.cloudera.hue.livy.server.interactive
 
 import com.cloudera.hue.livy.msgs.ExecuteRequest
-import com.cloudera.hue.livy.server.interactive.Statement
-import org.json4s.JsonAST.{JArray, JString}
+import org.json4s.JsonAST.JString
 import org.json4s.{DefaultFormats, Extraction}
-import org.scalatest.{Matchers, FunSpec}
+import org.scalatest.{FunSpec, Matchers}
 
 import scala.concurrent.duration.Duration
 import scala.concurrent.{Await, Future}