Просмотр исходного кода

HUE-2588 [livy] Start unifying factories, managers, and servlets

Erick Tryzelaar 10 лет назад
Родитель
Сommit
32472be
16 измененных файлов с 205 добавлено и 123 удалено
  1. 2 2
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/Main.scala
  2. 28 0
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/SessionFactory.scala
  3. 71 0
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/SessionManager.scala
  4. 41 0
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/SessionServlet.scala
  5. 5 2
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchSessionFactory.scala
  6. 4 33
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchSessionManager.scala
  7. 7 3
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchSessionProcessFactory.scala
  8. 8 11
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchSessionServlet.scala
  9. 4 2
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchSessionYarnFactory.scala
  10. 4 5
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionFactory.scala
  11. 17 42
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionManager.scala
  12. 1 1
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionProcessFactory.scala
  13. 5 14
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionServlet.scala
  14. 1 1
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionYarnFactory.scala
  15. 5 5
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/batches/BatchServletSpec.scala
  16. 2 2
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionServletSpec.scala

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

@@ -118,7 +118,7 @@ object Main {
 
 class ScalatraBootstrap extends LifeCycle with Logging {
 
-  var sessionManager: SessionManager = null
+  var sessionManager: InteractiveSessionManager = null
   var batchManager: BatchManager = null
 
   override def init(context: ServletContext): Unit = {
@@ -141,7 +141,7 @@ class ScalatraBootstrap extends LifeCycle with Logging {
         (new InteractiveSessionYarnFactory(livyConf), new BatchSessionYarnFactory(livyConf))
     }
 
-    sessionManager = new SessionManager(sessionFactory)
+    sessionManager = new InteractiveSessionManager(sessionFactory)
     batchManager = new BatchManager(batchFactory)
 
     context.mount(new InteractiveSessionServlet(sessionManager), "/sessions/*")

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

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

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

@@ -0,0 +1,71 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.cloudera.hue.livy.server
+
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicInteger
+
+import com.cloudera.hue.livy.Logging
+
+import scala.collection.JavaConversions._
+import scala.concurrent.{ExecutionContext, Future}
+
+abstract class SessionManager[S <: Session, C](factory: SessionFactory[S, C])
+  extends Logging
+{
+  private implicit def executor: ExecutionContext = ExecutionContext.global
+
+  protected[this] val _idCounter = new AtomicInteger()
+  protected[this] val _sessions = new ConcurrentHashMap[Int, S]()
+
+  def create(createRequest: C): Future[S] = {
+    val id = _idCounter.getAndIncrement
+    val session: Future[S] = factory.create(id, createRequest)
+
+    session.map({ case(session) =>
+      info("created session %s" format session.id)
+      _sessions.put(session.id, session)
+      session
+    })
+  }
+
+  def get(id: Int): Option[S] = Option(_sessions.get(id))
+
+  def all(): Seq[S] = _sessions.values().toSeq
+
+  def delete(id: Int): Future[Unit] = {
+    get(id) match {
+      case Some(session) => delete(session)
+      case None => Future.successful(())
+    }
+  }
+
+  def delete(session: S): Future[Unit] = {
+    session.stop().map { case _ =>
+      _sessions.remove(session.id)
+        Unit
+    }
+  }
+
+  def remove(id: Int): Option[S] = {
+    Option(_sessions.remove(id))
+  }
+
+  def shutdown(): Unit = {}
+}

+ 41 - 0
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/SessionServlet.scala

@@ -0,0 +1,41 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.cloudera.hue.livy.server
+
+import org.json4s.{DefaultFormats, Formats}
+import org.scalatra.json.JacksonJsonSupport
+import org.scalatra.{FutureSupport, MethodOverride, ScalatraServlet, UrlGeneratorSupport}
+
+import scala.concurrent.ExecutionContext
+
+abstract class SessionServlet[S <: Session, C](sessionManager: SessionManager[S, C])
+  extends ScalatraServlet
+  with FutureSupport
+  with MethodOverride
+  with JacksonJsonSupport
+  with UrlGeneratorSupport
+{
+  override protected implicit def executor: ExecutionContext = ExecutionContext.global
+
+  override protected implicit def jsonFormats: Formats = DefaultFormats
+
+  before() {
+    contentType = formats("json")
+  }
+}

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

@@ -18,6 +18,9 @@
 
 package com.cloudera.hue.livy.server.batch
 
-abstract class BatchSessionFactory {
-  def create(id: Int, createBatchRequest: CreateBatchRequest): BatchSession
+import com.cloudera.hue.livy.server.SessionFactory
+
+abstract class BatchSessionFactory
+  extends SessionFactory[BatchSession, CreateBatchRequest]
+{
 }

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

@@ -18,40 +18,11 @@
 
 package com.cloudera.hue.livy.server.batch
 
-import java.util.concurrent.ConcurrentHashMap
-import java.util.concurrent.atomic.AtomicInteger
+import com.cloudera.hue.livy.server.SessionManager
 
-import scala.collection.JavaConversions._
-import scala.concurrent.Future
-
-class BatchManager(batchFactory: BatchSessionFactory) {
-  private[this] val _idCounter = new AtomicInteger()
-  private[this] val _batches = new ConcurrentHashMap[Int, BatchSession]
-
-  def getBatch(id: Int): Option[BatchSession] = Option(_batches.get(id))
-
-  def getBatches: Array[BatchSession] = _batches.values().iterator().toArray
-
-  def createBatch(createBatchRequest: CreateBatchRequest): BatchSession = {
-    val id = _idCounter.getAndIncrement
-    val batch = batchFactory.create(id, createBatchRequest)
-    _batches.put(id, batch)
-
-    batch
-  }
-
-  def remove(id: Int): Option[BatchSession] = {
-    Option(_batches.remove(id))
-  }
-
-  def delete(batch: BatchSession): Future[Unit] = {
-    _batches.remove(batch.id)
-    batch.stop()
-  }
-
-  def shutdown() = {
-
-  }
+class BatchManager(batchFactory: BatchSessionFactory)
+  extends SessionManager[BatchSession, CreateBatchRequest](batchFactory)
+{
 }
 
 case class CreateBatchRequest(file: String,

+ 7 - 3
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchSessionProcessFactory.scala

@@ -20,7 +20,11 @@ package com.cloudera.hue.livy.server.batch
 
 import com.cloudera.hue.livy.LivyConf
 
-class BatchSessionProcessFactory(livyConf: LivyConf) extends BatchSessionFactory {
-  def create(id: Int, createBatchRequest: CreateBatchRequest): BatchSession =
-    BatchSessionProcess(livyConf, id, createBatchRequest)
+import scala.concurrent.Future
+
+class BatchSessionProcessFactory(livyConf: LivyConf)
+  extends BatchSessionFactory
+{
+  override def create(id: Int, createBatchRequest: CreateBatchRequest): Future[BatchSession] =
+    Future.successful(BatchSessionProcess(livyConf, id, createBatchRequest))
 }

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

@@ -45,7 +45,7 @@ class BatchSessionServlet(batchManager: BatchManager)
 
   get("/") {
     Map(
-      "batches" -> batchManager.getBatches
+      "batches" -> batchManager.all()
     )
   }
 
@@ -53,21 +53,18 @@ class BatchSessionServlet(batchManager: BatchManager)
     val createBatchRequest = parsedBody.extract[CreateBatchRequest]
 
     new AsyncResult {
-      val is = Future {
-        val batch = batchManager.createBatch(createBatchRequest)
-        Created(batch,
-          headers = Map(
-            "Location" -> url(getBatch, "id" -> batch.id.toString)
-          )
+      val is = for {
+        batch <- batchManager.create(createBatchRequest)
+      } yield Created(batch,
+          headers = Map("Location" -> url(getBatch, "id" -> batch.id.toString))
         )
-      }
     }
   }
 
   val getBatch = get("/:id") {
     val id = params("id").toInt
 
-    batchManager.getBatch(id) match {
+    batchManager.get(id) match {
       case None => NotFound("batch not found")
       case Some(batch) => Serializers.serializeBatch(batch)
     }
@@ -76,7 +73,7 @@ class BatchSessionServlet(batchManager: BatchManager)
   get("/:id/state") {
     val id = params("id").toInt
 
-    batchManager.getBatch(id) match {
+    batchManager.get(id) match {
       case None => NotFound("batch not found")
       case Some(batch) =>
         ("id", batch.id) ~ ("state", batch.state.toString)
@@ -86,7 +83,7 @@ class BatchSessionServlet(batchManager: BatchManager)
   get("/:id/log") {
     val id = params("id").toInt
 
-    batchManager.getBatch(id) match {
+    batchManager.get(id) match {
       case None => NotFound("batch not found")
       case Some(batch) =>
         val from = params.get("from").map(_.toInt)

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

@@ -21,10 +21,12 @@ package com.cloudera.hue.livy.server.batch
 import com.cloudera.hue.livy.LivyConf
 import com.cloudera.hue.livy.yarn.Client
 
+import scala.concurrent.Future
+
 class BatchSessionYarnFactory(livyConf: LivyConf) extends BatchSessionFactory {
 
   val client = new Client(livyConf)
 
-  def create(id: Int, createBatchRequest: CreateBatchRequest): BatchSession =
-    BatchSessionYarn(livyConf, client, id, createBatchRequest)
+  def create(id: Int, createBatchRequest: CreateBatchRequest): Future[BatchSession] =
+    Future.successful(BatchSessionYarn(livyConf, client, id, createBatchRequest))
 }

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

@@ -18,10 +18,9 @@
 
 package com.cloudera.hue.livy.server.interactive
 
-import scala.concurrent.Future
+import com.cloudera.hue.livy.server.SessionFactory
 
-trait InteractiveSessionFactory {
-  def createSession(id: Int, createInteractiveRequest: CreateInteractiveRequest): Future[InteractiveSession]
-
-  def close(): Unit = {}
+trait InteractiveSessionFactory
+  extends SessionFactory[InteractiveSession, CreateInteractiveRequest]
+{
 }

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

@@ -18,17 +18,15 @@
 
 package com.cloudera.hue.livy.server.interactive
 
-import java.util.concurrent.ConcurrentHashMap
-import java.util.concurrent.atomic.AtomicInteger
-
 import com.cloudera.hue.livy.Logging
+import com.cloudera.hue.livy.server.SessionManager
 import com.cloudera.hue.livy.sessions.Kind
 
 import scala.collection.JavaConversions._
 import scala.concurrent.duration.Duration
 import scala.concurrent.{Await, ExecutionContext, ExecutionContextExecutor, Future}
 
-object SessionManager {
+object InteractiveSessionManager {
   // Time in milliseconds; TODO: make configurable
   val TIMEOUT = 60000
 
@@ -36,64 +34,41 @@ object SessionManager {
   val GC_PERIOD = 1000 * 60 * 60
 }
 
-class SessionManager(factory: InteractiveSessionFactory) extends Logging {
+class InteractiveSessionManager(factory: InteractiveSessionFactory)
+  extends SessionManager[InteractiveSession, CreateInteractiveRequest](factory)
+  with Logging
+{
+  import InteractiveSessionManager._
 
   private implicit def executor: ExecutionContextExecutor = ExecutionContext.global
 
-  private[this] val _idCounter = new AtomicInteger()
-  private[this] val sessions = new ConcurrentHashMap[Int, InteractiveSession]()
-
   private val garbageCollector = new GarbageCollector(this)
   garbageCollector.start()
 
-  def get(sessionId: Int): Option[InteractiveSession] = {
-    Option(sessions.get(sessionId))
-  }
-
-  def getSessions = {
-    sessions.values
-  }
-
   def getSessionIds = {
-    sessions.keys
+    _sessions.keys
   }
 
-  def createSession(createInteractiveRequest: CreateInteractiveRequest): Future[InteractiveSession] = {
-    val id = _idCounter.getAndIncrement
-    val session = factory.createSession(id, createInteractiveRequest)
-
-    session.map({ case(session: InteractiveSession) =>
-      info("created session %s" format session.id)
-      sessions.put(session.id, session)
-      session
-    })
-  }
-
-  def shutdown(): Unit = {
-    Await.result(Future.sequence(sessions.values.map(delete)), Duration.Inf)
+  override def shutdown(): Unit = {
+    Await.result(Future.sequence(all().map(delete)), Duration.Inf)
     garbageCollector.shutdown()
   }
 
-  def delete(sessionId: Int): Future[Unit] = {
+  /*
+  override def delete(sessionId: Int): Future[Unit] = {
     get(sessionId) match {
       case Some(session) => delete(session)
       case None => Future.successful(Unit)
     }
   }
-
-  def delete(session: InteractiveSession): Future[Unit] = {
-    session.stop().map { case _ =>
-        sessions.remove(session.id)
-        Unit
-    }
-  }
+  */
 
   def collectGarbage() = {
     def expired(session: InteractiveSession): Boolean = {
-      System.currentTimeMillis() - session.lastActivity > SessionManager.TIMEOUT
+      System.currentTimeMillis() - session.lastActivity > TIMEOUT
     }
 
-    sessions.values.filter(expired).foreach(delete)
+    all().filter(expired).foreach(delete)
   }
 }
 
@@ -111,14 +86,14 @@ case class CreateInteractiveRequest(kind: Kind,
 
 class SessionNotFound extends Exception
 
-private class GarbageCollector(sessionManager: SessionManager) extends Thread {
+private class GarbageCollector(sessionManager: InteractiveSessionManager) extends Thread {
 
   private var finished = false
 
   override def run(): Unit = {
     while (!finished) {
       sessionManager.collectGarbage()
-      Thread.sleep(SessionManager.GC_PERIOD)
+      Thread.sleep(InteractiveSessionManager.GC_PERIOD)
     }
   }
 

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

@@ -26,7 +26,7 @@ class InteractiveSessionProcessFactory(livyConf: LivyConf) extends InteractiveSe
 
    implicit def executor: ExecutionContext = ExecutionContext.global
 
-   override def createSession(id: Int, createInteractiveRequest: CreateInteractiveRequest): Future[InteractiveSession] = {
+   override def create(id: Int, createInteractiveRequest: CreateInteractiveRequest): Future[InteractiveSession] = {
      Future {
        InteractiveSessionProcess.create(livyConf, id, createInteractiveRequest)
      }

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

@@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit
 
 import com.cloudera.hue.livy.Logging
 import com.cloudera.hue.livy.msgs.ExecuteRequest
+import com.cloudera.hue.livy.server.SessionServlet
 import com.cloudera.hue.livy.server.interactive.InteractiveSession.SessionFailedToStart
 import com.cloudera.hue.livy.sessions._
 import com.fasterxml.jackson.core.JsonParseException
@@ -30,30 +31,20 @@ import org.json4s.JsonAST.JString
 import org.json4s.JsonDSL._
 import org.json4s._
 import org.scalatra._
-import org.scalatra.json.JacksonJsonSupport
 
 import scala.concurrent._
 import scala.concurrent.duration._
 
 object InteractiveSessionServlet extends Logging
 
-class InteractiveSessionServlet(sessionManager: SessionManager)
-  extends ScalatraServlet
-  with FutureSupport
-  with MethodOverride
-  with JacksonJsonSupport
-  with UrlGeneratorSupport
+class InteractiveSessionServlet(sessionManager: InteractiveSessionManager)
+  extends SessionServlet(sessionManager)
 {
-  override protected implicit def executor: ExecutionContextExecutor = ExecutionContext.global
   override protected implicit def jsonFormats: Formats = DefaultFormats ++ Serializers.Formats
 
-  before() {
-    contentType = formats("json")
-  }
-
   get("/") {
     Map(
-      "sessions" -> sessionManager.getSessions
+      "sessions" -> sessionManager.all
     )
   }
 
@@ -71,7 +62,7 @@ class InteractiveSessionServlet(sessionManager: SessionManager)
 
     new AsyncResult {
       val is = {
-        val sessionFuture = sessionManager.createSession(createInteractiveRequest)
+        val sessionFuture = sessionManager.create(createInteractiveRequest)
 
         sessionFuture.map { case session =>
           Created(session,

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

@@ -29,7 +29,7 @@ class InteractiveSessionYarnFactory(livyConf: LivyConf) extends InteractiveSessi
 
    val client = new Client(livyConf)
 
-   override def createSession(id: Int, createInteractiveRequest: CreateInteractiveRequest): Future[InteractiveSession] = {
+   override def create(id: Int, createInteractiveRequest: CreateInteractiveRequest): Future[InteractiveSession] = {
      Future {
        InteractiveSessionYarn.create(livyConf, client, id, createInteractiveRequest)
      }

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

@@ -83,13 +83,13 @@ class BatchServletSpec extends ScalatraSuite with FunSpecLike with BeforeAndAfte
         val parsedBody = parse(body)
         parsedBody \ "id" should equal (JInt(0))
 
-        val batch = batchManager.getBatch(0)
+        val batch = batchManager.get(0)
         batch should be (defined)
       }
 
       // Wait for the process to finish.
       {
-        val batch: BatchSession = batchManager.getBatch(0).get
+        val batch: BatchSession = batchManager.get(0).get
         Utils.waitUntil({ () =>
           batch.state == Success()
         }, Duration(10, TimeUnit.SECONDS))
@@ -102,7 +102,7 @@ class BatchServletSpec extends ScalatraSuite with FunSpecLike with BeforeAndAfte
         parsedBody \ "id" should equal (JInt(0))
         parsedBody \ "state" should equal (JString("success"))
 
-        val batch = batchManager.getBatch(0)
+        val batch = batchManager.get(0)
         batch should be (defined)
       }
 
@@ -113,7 +113,7 @@ class BatchServletSpec extends ScalatraSuite with FunSpecLike with BeforeAndAfte
         parsedBody \ "id" should equal (JInt(0))
         (parsedBody \ "log").extract[List[String]] should contain ("hello world")
 
-        val batch = batchManager.getBatch(0)
+        val batch = batchManager.get(0)
         batch should be (defined)
       }
 
@@ -123,7 +123,7 @@ class BatchServletSpec extends ScalatraSuite with FunSpecLike with BeforeAndAfte
         val parsedBody = parse(body)
         parsedBody should equal (JObject(("msg", JString("deleted"))))
 
-        val batch = batchManager.getBatch(0)
+        val batch = batchManager.get(0)
         batch should not be defined
       }
     }

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

@@ -76,12 +76,12 @@ class InteractiveSessionServletSpec extends ScalatraSuite with FunSpecLike {
   }
 
   class MockInteractiveSessionFactory() extends InteractiveSessionFactory {
-    override def createSession(id: Int, createInteractiveRequest: CreateInteractiveRequest): Future[InteractiveSession] = {
+    override def create(id: Int, createInteractiveRequest: CreateInteractiveRequest): Future[InteractiveSession] = {
       Future.successful(new MockInteractiveSession(id))
     }
   }
 
-  val sessionManager = new SessionManager(new MockInteractiveSessionFactory())
+  val sessionManager = new InteractiveSessionManager(new MockInteractiveSessionFactory())
   val servlet = new InteractiveSessionServlet(sessionManager)
 
   addServlet(servlet, "/*")