Procházet zdrojové kódy

HUE-2588 [livy] Merge SessionManagers

Erick Tryzelaar před 10 roky
rodič
revize
44b5fed101
13 změnil soubory, kde provedl 131 přidání a 164 odebrání
  1. 6 7
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/Main.scala
  2. 2 0
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/Session.scala
  3. 45 5
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/SessionManager.scala
  4. 0 40
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchSessionManager.scala
  5. 2 1
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchSessionServlet.scala
  6. 34 0
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/CreateBatchRequest.scala
  7. 34 0
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/CreateInteractiveRequest.scala
  8. 1 1
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSession.scala
  9. 0 103
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionManager.scala
  10. 2 2
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionServlet.scala
  11. 1 1
      apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/interactive/InteractiveWebSession.scala
  12. 2 1
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/batches/BatchServletSpec.scala
  13. 2 3
      apps/spark/java/livy-server/src/test/scala/com/cloudera/hue/livy/server/interactive/InteractiveSessionServletSpec.scala

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

@@ -19,12 +19,11 @@
 package com.cloudera.hue.livy.server
 package com.cloudera.hue.livy.server
 
 
 import java.io.IOException
 import java.io.IOException
-import java.nio.file.Path
 import javax.servlet.ServletContext
 import javax.servlet.ServletContext
 
 
-import com.cloudera.hue.livy.server.batch.{BatchSessionProcessFactory, BatchSessionServlet, BatchManager, BatchSessionYarnFactory}
-import com.cloudera.hue.livy.server.interactive._
 import com.cloudera.hue.livy._
 import com.cloudera.hue.livy._
+import com.cloudera.hue.livy.server.batch._
+import com.cloudera.hue.livy.server.interactive._
 import org.scalatra._
 import org.scalatra._
 import org.scalatra.servlet.ScalatraListener
 import org.scalatra.servlet.ScalatraListener
 import org.slf4j.LoggerFactory
 import org.slf4j.LoggerFactory
@@ -118,8 +117,8 @@ object Main {
 
 
 class ScalatraBootstrap extends LifeCycle with Logging {
 class ScalatraBootstrap extends LifeCycle with Logging {
 
 
-  var sessionManager: InteractiveSessionManager = null
-  var batchManager: BatchManager = null
+  var sessionManager: SessionManager[InteractiveSession, CreateInteractiveRequest] = null
+  var batchManager: SessionManager[BatchSession, CreateBatchRequest] = null
 
 
   override def init(context: ServletContext): Unit = {
   override def init(context: ServletContext): Unit = {
     val livyConf = new LivyConf()
     val livyConf = new LivyConf()
@@ -141,8 +140,8 @@ class ScalatraBootstrap extends LifeCycle with Logging {
         (new InteractiveSessionYarnFactory(livyConf), new BatchSessionYarnFactory(livyConf))
         (new InteractiveSessionYarnFactory(livyConf), new BatchSessionYarnFactory(livyConf))
     }
     }
 
 
-    sessionManager = new InteractiveSessionManager(sessionFactory)
-    batchManager = new BatchManager(batchFactory)
+    sessionManager = new SessionManager(sessionFactory)
+    batchManager = new SessionManager(batchFactory)
 
 
     context.mount(new InteractiveSessionServlet(sessionManager), "/sessions/*")
     context.mount(new InteractiveSessionServlet(sessionManager), "/sessions/*")
     context.mount(new BatchSessionServlet(batchManager), "/batches/*")
     context.mount(new BatchSessionServlet(batchManager), "/batches/*")

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

@@ -25,6 +25,8 @@ import scala.concurrent.Future
 trait Session {
 trait Session {
   def id: Int
   def id: Int
 
 
+  def lastActivity: Option[Long] = None
+
   def state: State
   def state: State
 
 
   def stop(): Future[Unit]
   def stop(): Future[Unit]

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

@@ -26,19 +26,32 @@ import com.cloudera.hue.livy.Logging
 import scala.collection.JavaConversions._
 import scala.collection.JavaConversions._
 import scala.concurrent.{ExecutionContext, Future}
 import scala.concurrent.{ExecutionContext, Future}
 
 
-abstract class SessionManager[S <: Session, C](factory: SessionFactory[S, C])
-  extends Logging
-{
+object SessionManager {
+  // Time in milliseconds; TODO: make configurable
+  val TIMEOUT = 60000
+
+  // Time in milliseconds; TODO: make configurable
+  val GC_PERIOD = 1000 * 60 * 60
+}
+
+class SessionManager[S <: Session, C](factory: SessionFactory[S, C])
+  extends Logging {
+
+  import SessionManager._
+
   private implicit def executor: ExecutionContext = ExecutionContext.global
   private implicit def executor: ExecutionContext = ExecutionContext.global
 
 
   protected[this] val _idCounter = new AtomicInteger()
   protected[this] val _idCounter = new AtomicInteger()
   protected[this] val _sessions = new ConcurrentHashMap[Int, S]()
   protected[this] val _sessions = new ConcurrentHashMap[Int, S]()
 
 
+  private val garbageCollector = new GarbageCollector
+  garbageCollector.start()
+
   def create(createRequest: C): Future[S] = {
   def create(createRequest: C): Future[S] = {
     val id = _idCounter.getAndIncrement
     val id = _idCounter.getAndIncrement
     val session: Future[S] = factory.create(id, createRequest)
     val session: Future[S] = factory.create(id, createRequest)
 
 
-    session.map({ case(session) =>
+    session.map({ case (session) =>
       info("created session %s" format session.id)
       info("created session %s" format session.id)
       _sessions.put(session.id, session)
       _sessions.put(session.id, session)
       session
       session
@@ -59,7 +72,7 @@ abstract class SessionManager[S <: Session, C](factory: SessionFactory[S, C])
   def delete(session: S): Future[Unit] = {
   def delete(session: S): Future[Unit] = {
     session.stop().map { case _ =>
     session.stop().map { case _ =>
       _sessions.remove(session.id)
       _sessions.remove(session.id)
-        Unit
+      Unit
     }
     }
   }
   }
 
 
@@ -68,4 +81,31 @@ abstract class SessionManager[S <: Session, C](factory: SessionFactory[S, C])
   }
   }
 
 
   def shutdown(): Unit = {}
   def shutdown(): Unit = {}
+
+  def collectGarbage() = {
+    def expired(session: Session): Boolean = {
+      session.lastActivity match {
+        case Some(lastActivity) => System.currentTimeMillis() - lastActivity > TIMEOUT
+        case None => false
+      }
+    }
+
+    all().filter(expired).foreach(delete)
+  }
+
+  private class GarbageCollector extends Thread {
+
+    private var finished = false
+
+    override def run(): Unit = {
+      while (!finished) {
+        collectGarbage()
+        Thread.sleep(SessionManager.GC_PERIOD)
+      }
+    }
+
+    def shutdown(): Unit = {
+      finished = true
+    }
+  }
 }
 }

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

@@ -1,40 +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.batch
-
-import com.cloudera.hue.livy.server.SessionManager
-
-class BatchManager(batchFactory: BatchSessionFactory)
-  extends SessionManager[BatchSession, CreateBatchRequest](batchFactory)
-{
-}
-
-case class CreateBatchRequest(file: String,
-                              proxyUser: Option[String] = None,
-                              args: List[String] = List(),
-                              className: Option[String] = None,
-                              jars: List[String] = List(),
-                              pyFiles: List[String] = List(),
-                              files: List[String] = List(),
-                              driverMemory: Option[String] = None,
-                              driverCores: Option[Int] = None,
-                              executorMemory: Option[String] = None,
-                              executorCores: Option[Int] = None,
-                              numExecutors: Option[Int] = None,
-                              archives: List[String] = List())

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

@@ -19,6 +19,7 @@
 package com.cloudera.hue.livy.server.batch
 package com.cloudera.hue.livy.server.batch
 
 
 import com.cloudera.hue.livy.Logging
 import com.cloudera.hue.livy.Logging
+import com.cloudera.hue.livy.server.SessionManager
 import com.fasterxml.jackson.core.JsonParseException
 import com.fasterxml.jackson.core.JsonParseException
 import org.json4s.JsonDSL._
 import org.json4s.JsonDSL._
 import org.json4s._
 import org.json4s._
@@ -29,7 +30,7 @@ import scala.concurrent.{Future, ExecutionContext, ExecutionContextExecutor}
 
 
 object BatchSessionServlet extends Logging
 object BatchSessionServlet extends Logging
 
 
-class BatchSessionServlet(batchManager: BatchManager)
+class BatchSessionServlet(batchManager: SessionManager[BatchSession, CreateBatchRequest])
   extends ScalatraServlet
   extends ScalatraServlet
   with FutureSupport
   with FutureSupport
   with MethodOverride
   with MethodOverride

+ 34 - 0
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/CreateBatchRequest.scala

@@ -0,0 +1,34 @@
+/*
+ * 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.batch
+
+case class CreateBatchRequest(
+    file: String,
+    proxyUser: Option[String] = None,
+    args: List[String] = List(),
+    className: Option[String] = None,
+    jars: List[String] = List(),
+    pyFiles: List[String] = List(),
+    files: List[String] = List(),
+    driverMemory: Option[String] = None,
+    driverCores: Option[Int] = None,
+    executorMemory: Option[String] = None,
+    executorCores: Option[Int] = None,
+    numExecutors: Option[Int] = None,
+    archives: List[String] = List())

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

@@ -0,0 +1,34 @@
+/*
+ * 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.sessions.Kind
+
+case class CreateInteractiveRequest(
+    kind: Kind,
+    proxyUser: Option[String] = None,
+    jars: List[String] = List(),
+    pyFiles: List[String] = List(),
+    files: List[String] = List(),
+    driverMemory: Option[String] = None,
+    driverCores: Option[Int] = None,
+    executorMemory: Option[String] = None,
+    executorCores: Option[Int] = None,
+    numExecutors: Option[Int] = None,
+    archives: List[String] = List())

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

@@ -40,7 +40,7 @@ trait InteractiveSession extends Session {
 
 
   def proxyUser: Option[String]
   def proxyUser: Option[String]
 
 
-  def lastActivity: Long
+  override def lastActivity: Option[Long]
 
 
   def url: Option[URL]
   def url: Option[URL]
 
 

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

@@ -1,103 +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.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 InteractiveSessionManager {
-  // Time in milliseconds; TODO: make configurable
-  val TIMEOUT = 60000
-
-  // Time in milliseconds; TODO: make configurable
-  val GC_PERIOD = 1000 * 60 * 60
-}
-
-class InteractiveSessionManager(factory: InteractiveSessionFactory)
-  extends SessionManager[InteractiveSession, CreateInteractiveRequest](factory)
-  with Logging
-{
-  import InteractiveSessionManager._
-
-  private implicit def executor: ExecutionContextExecutor = ExecutionContext.global
-
-  private val garbageCollector = new GarbageCollector(this)
-  garbageCollector.start()
-
-  def getSessionIds = {
-    _sessions.keys
-  }
-
-  override def shutdown(): Unit = {
-    Await.result(Future.sequence(all().map(delete)), Duration.Inf)
-    garbageCollector.shutdown()
-  }
-
-  /*
-  override def delete(sessionId: Int): Future[Unit] = {
-    get(sessionId) match {
-      case Some(session) => delete(session)
-      case None => Future.successful(Unit)
-    }
-  }
-  */
-
-  def collectGarbage() = {
-    def expired(session: InteractiveSession): Boolean = {
-      System.currentTimeMillis() - session.lastActivity > TIMEOUT
-    }
-
-    all().filter(expired).foreach(delete)
-  }
-}
-
-case class CreateInteractiveRequest(kind: Kind,
-                                    proxyUser: Option[String] = None,
-                                    jars: List[String] = List(),
-                                    pyFiles: List[String] = List(),
-                                    files: List[String] = List(),
-                                    driverMemory: Option[String] = None,
-                                    driverCores: Option[Int] = None,
-                                    executorMemory: Option[String] = None,
-                                    executorCores: Option[Int] = None,
-                                    numExecutors: Option[Int] = None,
-                                    archives: List[String] = List())
-
-class SessionNotFound extends Exception
-
-private class GarbageCollector(sessionManager: InteractiveSessionManager) extends Thread {
-
-  private var finished = false
-
-  override def run(): Unit = {
-    while (!finished) {
-      sessionManager.collectGarbage()
-      Thread.sleep(InteractiveSessionManager.GC_PERIOD)
-    }
-  }
-
-  def shutdown(): Unit = {
-    finished = true
-  }
-}

+ 2 - 2
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.Logging
 import com.cloudera.hue.livy.msgs.ExecuteRequest
 import com.cloudera.hue.livy.msgs.ExecuteRequest
-import com.cloudera.hue.livy.server.SessionServlet
+import com.cloudera.hue.livy.server.{SessionManager, SessionServlet}
 import com.cloudera.hue.livy.server.interactive.InteractiveSession.SessionFailedToStart
 import com.cloudera.hue.livy.server.interactive.InteractiveSession.SessionFailedToStart
 import com.cloudera.hue.livy.sessions._
 import com.cloudera.hue.livy.sessions._
 import com.fasterxml.jackson.core.JsonParseException
 import com.fasterxml.jackson.core.JsonParseException
@@ -37,7 +37,7 @@ import scala.concurrent.duration._
 
 
 object InteractiveSessionServlet extends Logging
 object InteractiveSessionServlet extends Logging
 
 
-class InteractiveSessionServlet(sessionManager: InteractiveSessionManager)
+class InteractiveSessionServlet(sessionManager: SessionManager[InteractiveSession, CreateInteractiveRequest])
   extends SessionServlet(sessionManager)
   extends SessionServlet(sessionManager)
 {
 {
   override protected implicit def jsonFormats: Formats = DefaultFormats ++ Serializers.Formats
   override protected implicit def jsonFormats: Formats = DefaultFormats ++ Serializers.Formats

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

@@ -64,7 +64,7 @@ abstract class InteractiveWebSession(val id: Int, createInteractiveRequest: Crea
     dispatch.url(url.toString)
     dispatch.url(url.toString)
   }
   }
 
 
-  override def lastActivity: Long = _lastActivity
+  override def lastActivity: Option[Long] = Some(_lastActivity)
 
 
   override def state: State = _state
   override def state: State = _state
 
 

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

@@ -22,6 +22,7 @@ import java.io.FileWriter
 import java.nio.file.{Files, Path}
 import java.nio.file.{Files, Path}
 import java.util.concurrent.TimeUnit
 import java.util.concurrent.TimeUnit
 
 
+import com.cloudera.hue.livy.server.SessionManager
 import com.cloudera.hue.livy.sessions.Success
 import com.cloudera.hue.livy.sessions.Success
 import com.cloudera.hue.livy.{LivyConf, Utils}
 import com.cloudera.hue.livy.{LivyConf, Utils}
 import com.cloudera.hue.livy.server.batch._
 import com.cloudera.hue.livy.server.batch._
@@ -54,7 +55,7 @@ class BatchServletSpec extends ScalatraSuite with FunSpecLike with BeforeAndAfte
   }
   }
 
 
   val batchFactory = new BatchSessionProcessFactory(new LivyConf())
   val batchFactory = new BatchSessionProcessFactory(new LivyConf())
-  val batchManager = new BatchManager(batchFactory)
+  val batchManager = new SessionManager(batchFactory)
   val servlet = new BatchSessionServlet(batchManager)
   val servlet = new BatchSessionServlet(batchManager)
 
 
   addServlet(servlet, "/*")
   addServlet(servlet, "/*")

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

@@ -22,6 +22,7 @@ import java.net.URL
 import java.util.concurrent.atomic.AtomicInteger
 import java.util.concurrent.atomic.AtomicInteger
 
 
 import com.cloudera.hue.livy.msgs.ExecuteRequest
 import com.cloudera.hue.livy.msgs.ExecuteRequest
+import com.cloudera.hue.livy.server.SessionManager
 import com.cloudera.hue.livy.sessions._
 import com.cloudera.hue.livy.sessions._
 import org.json4s.{DefaultFormats, Formats}
 import org.json4s.{DefaultFormats, Formats}
 import org.json4s.JsonAST.{JInt, JArray, JObject, JString}
 import org.json4s.JsonAST.{JInt, JArray, JObject, JString}
@@ -52,8 +53,6 @@ class InteractiveSessionServletSpec extends ScalatraSuite with FunSpecLike {
 
 
     override def url_=(url: URL): Unit = ???
     override def url_=(url: URL): Unit = ???
 
 
-    override def lastActivity: Long = ???
-
     override def executeStatement(executeRequest: ExecuteRequest): Statement = {
     override def executeStatement(executeRequest: ExecuteRequest): Statement = {
       val id = _idCounter.getAndIncrement
       val id = _idCounter.getAndIncrement
       val statement = new Statement(
       val statement = new Statement(
@@ -81,7 +80,7 @@ class InteractiveSessionServletSpec extends ScalatraSuite with FunSpecLike {
     }
     }
   }
   }
 
 
-  val sessionManager = new InteractiveSessionManager(new MockInteractiveSessionFactory())
+  val sessionManager = new SessionManager(new MockInteractiveSessionFactory())
   val servlet = new InteractiveSessionServlet(sessionManager)
   val servlet = new InteractiveSessionServlet(sessionManager)
 
 
   addServlet(servlet, "/*")
   addServlet(servlet, "/*")