Browse Source

HUE-2863 [livy] Rewrite repl to eliminate races to the interpreter

Erick Tryzelaar 10 năm trước cách đây
mục cha
commit
6c8cd82d9e
24 tập tin đã thay đổi với 1112 bổ sung883 xóa
  1. 13 1
      apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/sessions/State.scala
  2. 45 30
      apps/spark/java/livy-repl/src/main/resources/fake_R.sh
  3. 22 18
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/Interpreter.scala
  4. 9 7
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/Main.scala
  5. 110 12
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/Session.scala
  6. 1 12
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/WebApp.scala
  7. 30 0
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/package.scala
  8. 73 120
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/process/ProcessInterpreter.scala
  9. 34 18
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/python/PythonInterpreter.scala
  10. 0 59
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/python/PythonSession.scala
  11. 260 0
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/scala/SparkInterpreter.scala
  12. 0 341
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/scala/SparkInterpreter/SparkInterpreter.scala
  13. 0 58
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/scala/SparkSession.scala
  14. 63 68
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/sparkr/SparkRInterpreter.scala
  15. 0 94
      apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/sparkr/SparkRSession.scala
  16. 37 0
      apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/BaseInterpreterSpec.scala
  17. 3 3
      apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/BaseSessionSpec.scala
  18. 131 0
      apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/PythonInterpreterSpec.scala
  19. 3 3
      apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/PythonSessionSpec.scala
  20. 130 0
      apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/ScalaInterpreterSpec.scala
  21. 105 0
      apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/SparkRInterpreterSpec.scala
  22. 2 2
      apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/SparkRSessionSpec.scala
  23. 7 6
      apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/SparkSessionSpec.scala
  24. 34 31
      apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/WebAppSpec.scala

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

@@ -18,40 +18,52 @@
 
 package com.cloudera.hue.livy.sessions
 
-sealed trait State
+sealed trait State {
+  /** Returns true if the State represents a process that can eventually execute commands */
+  def isActive: Boolean
+}
 
 case class NotStarted() extends State {
+  override def isActive = true
   override def toString = "not_started"
 }
 
 case class Starting() extends State {
+  override def isActive = true
   override def toString = "starting"
 }
 
 case class Idle() extends State {
+  override def isActive = true
   override def toString = "idle"
 }
 
 case class Running() extends State {
+  override def isActive = true
   override def toString = "running"
 }
 
 case class Busy() extends State {
+  override def isActive = true
   override def toString = "busy"
 }
 
 case class Error() extends State {
+  override def isActive = true
   override def toString = "error"
 }
 
 case class ShuttingDown() extends State {
+  override def isActive = false
   override def toString = "shutting_down"
 }
 
 case class Dead() extends State {
+  override def isActive = false
   override def toString = "dead"
 }
 
 case class Success() extends State {
+  override def isActive = false
   override def toString = "success"
 }

+ 45 - 30
apps/spark/java/livy-repl/src/main/resources/fake_R.sh

@@ -16,6 +16,7 @@
 
 import ast
 import cStringIO
+import collections
 import datetime
 import decimal
 import json
@@ -28,19 +29,13 @@ LOG = logging.getLogger('fake_shell')
 
 global_dict = {}
 
-execution_count = 0
-
 
 def execute_reply(status, content):
-    global execution_count
-    execution_count += 1
-
     return {
         'msg_type': 'execute_reply',
         'content': dict(
             content,
             status=status,
-            execution_count=execution_count - 1
         )
     }
 
@@ -60,7 +55,6 @@ def execute_reply_error(exc_type, exc_value, tb):
 
 def execute(code):
     try:
-        code = ast.parse(code)
         to_run_exec, to_run_single = code.body[:-1], code.body[-1:]
 
         for node in to_run_exec:
@@ -93,7 +87,23 @@ def execute(code):
 
     return execute_reply_ok({
         'text/plain': output.rstrip(),
-        })
+    })
+
+
+def execute_magic(line):
+    parts = line[1:].split(' ', 1)
+    if len(parts) == 1:
+        magic, rest = parts[0], ()
+    else:
+        magic, rest = parts[0], (parts[1],)
+
+    try:
+        handler = magic_router[magic]
+    except KeyError:
+        exc_type, exc_value, tb = sys.exc_info()
+        return execute_reply_error(exc_type, exc_value, [])
+    else:
+        return handler(*rest)
 
 
 def execute_request(content):
@@ -103,33 +113,38 @@ def execute_request(content):
         exc_type, exc_value, tb = sys.exc_info()
         return execute_reply_error(exc_type, exc_value, [])
 
-    lines = code.split('\n')
+    lines = collections.deque(code.rstrip().split('\n'))
+    last_line = ''
+    result = None
 
-    if lines and lines[-1].startswith('%'):
-        code, magic = lines[:-1], lines[-1]
+    while lines:
+        line = last_line + lines.popleft()
 
-        # Make sure to execute the other lines first.
-        if code:
-            result = execute('\n'.join(code))
-            if result['content']['status'] != 'ok':
-                return result
+        if line.rstrip() == '':
+            continue
 
-        parts = magic[1:].split(' ', 1)
-        if len(parts) == 1:
-            magic, rest = parts[0], ()
+        if line.startswith('%'):
+            result = execute_magic(line)
         else:
-            magic, rest = parts[0], (parts[1],)
+            try:
+                code = ast.parse(line)
+            except SyntaxError:
+                last_line = line + '\n'
+                continue
+            else:
+                result = execute(code)
 
-        try:
-            handler = magic_router[magic]
-        except KeyError:
-            exc_type, exc_value, tb = sys.exc_info()
-            return execute_reply_error(exc_type, exc_value, [])
+        if result['content']['status'] == 'ok':
+            last_line = ''
         else:
-            return handler(*rest)
-    else:
-        return execute(code)
+            return result
 
+    if result is None:
+        return execute_reply_ok({
+            'text/plain': '',
+        })
+    else:
+        return result
 
 def magic_table_convert(value):
     try:
@@ -330,7 +345,6 @@ try:
         except ValueError, e:
             response = json.dumps({
                 'msg_type': 'inspect_reply',
-                'execution_count': execution_count - 1,
                 'content': {
                     'status': 'error',
                     'ename': 'ValueError',
@@ -342,7 +356,8 @@ try:
         print >> sys_stdout, response
         sys_stdout.flush()
 finally:
-    global_dict['sc'].stop()
+    if 'sc' in global_dict:
+        global_dict['sc'].stop()
 
     sys.stdin = sys_stdin
     sys.stdout = sys_stdout

+ 22 - 18
apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/Interpreter.scala

@@ -18,29 +18,33 @@
 
 package com.cloudera.hue.livy.repl
 
-import com.cloudera.hue.livy.Utils
-import com.cloudera.hue.livy.sessions.State
-import org.json4s._
+import org.json4s.JObject
 
-import _root_.scala.concurrent._
-import _root_.scala.concurrent.duration.Duration
+object Interpreter {
+  abstract class ExecuteResponse
+
+  case class ExecuteSuccess(content: JObject) extends ExecuteResponse
+  case class ExecuteError(ename: String,
+                          evalue: String,
+                          traceback: Seq[String] = Seq()) extends ExecuteResponse
+  case class ExecuteIncomplete() extends ExecuteResponse
+  case class ExecuteAborted(message: String) extends ExecuteResponse
+}
 
 trait Interpreter {
-  /** The Interpreter's state */
-  def state: State
+  import Interpreter._
+
+  def kind: String
 
-    /**
-     * Execute the code and return the result as a Future as it may
-     * take some time to execute.
-     */
-  def execute(code: String): Future[JValue]
+  /** Start the Interpreter */
+  def start(): Unit
+
+  /**
+   * Execute the code and return the result as a Future as it may
+   * take some time to execute.
+   */
+  def execute(code: String): ExecuteResponse
 
   /** Shut down the interpreter. */
   def close(): Unit
-
-  @throws(classOf[TimeoutException])
-  @throws(classOf[InterruptedException])
-  final def waitForStateChange(oldState: State, atMost: Duration) = {
-    Utils.waitUntil({ () => state != oldState }, atMost)
-  }
 }

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

@@ -21,9 +21,9 @@ package com.cloudera.hue.livy.repl
 import java.util.concurrent.TimeUnit
 import javax.servlet.ServletContext
 
-import com.cloudera.hue.livy.repl.python.PythonSession
-import com.cloudera.hue.livy.repl.scala.SparkSession
-import com.cloudera.hue.livy.repl.sparkr.SparkRSession
+import com.cloudera.hue.livy.repl.python.PythonInterpreter
+import com.cloudera.hue.livy.repl.scala.SparkInterpreter
+import com.cloudera.hue.livy.repl.sparkr.SparkRInterpreter
 import com.cloudera.hue.livy.sessions.Starting
 import com.cloudera.hue.livy.{Logging, WebServer}
 import dispatch._
@@ -102,12 +102,14 @@ class ScalatraBootstrap extends LifeCycle with Logging {
 
   override def init(context: ServletContext): Unit = {
     try {
-      session = context.getInitParameter(Main.SESSION_KIND) match {
-        case Main.PYSPARK_SESSION => PythonSession.create()
-        case Main.SPARK_SESSION => SparkSession.create()
-        case Main.SPARKR_SESSION => SparkRSession.create()
+      val interpreter = context.getInitParameter(Main.SESSION_KIND) match {
+        case Main.PYSPARK_SESSION => PythonInterpreter()
+        case Main.SPARK_SESSION => SparkInterpreter()
+        case Main.SPARKR_SESSION => SparkRInterpreter()
       }
 
+      session = Session(interpreter)
+
       context.mount(new WebApp(session), "/*")
 
       val callbackUrl = Option(System.getProperty("livy.repl.callback-url"))

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

@@ -18,29 +18,127 @@
 
 package com.cloudera.hue.livy.repl
 
-import com.cloudera.hue.livy.Utils
-import com.cloudera.hue.livy.sessions.{Kind, State}
-import org.json4s.JValue
+import java.util.concurrent.Executors
+
+import com.cloudera.hue.livy.{Utils, Logging}
+import com.cloudera.hue.livy.sessions._
+import org.json4s.JsonDSL._
+import org.json4s.{JValue, DefaultFormats, Extraction}
 
 import _root_.scala.concurrent.duration.Duration
-import _root_.scala.concurrent.{Future, TimeoutException}
+import _root_.scala.concurrent.{TimeoutException, ExecutionContext, Future}
+
+object Session {
+  val STATUS = "status"
+  val OK = "ok"
+  val ERROR = "error"
+  val EXECUTION_COUNT = "execution_count"
+  val DATA = "data"
+  val ENAME = "ename"
+  val EVALUE = "evalue"
+  val TRACEBACK = "traceback"
+
+  def apply(interpreter: Interpreter): Session = new Session(interpreter)
+}
+
+class Session(interpreter: Interpreter)
+  extends Logging
+{
+  import Session._
+
+  private implicit val executor = ExecutionContext.fromExecutorService(Executors.newSingleThreadExecutor())
+  private implicit val formats = DefaultFormats
 
-trait Session {
-  def kind: Kind
+  private var _state: State = NotStarted()
+  private var _history = IndexedSeq[Statement]()
 
-  def state: State
+  Future {
+    _state = Starting()
+    interpreter.start()
+    _state = Idle()
+  }.onFailure { case _ =>
+    _state = Error()
+  }
+
+  def kind: String = interpreter.kind
+
+  def state = _state
 
-  def execute(code: String): Statement
+  def history: IndexedSeq[Statement] = _history
 
-  def history: IndexedSeq[Statement]
+  def execute(code: String): Statement = synchronized {
+    val executionCount = _history.length
+    val statement = Statement(executionCount, Future { executeCode(executionCount, code) })
+    _history :+= statement
+    statement
+  }
+
+  def close(): Unit = {
+    executor.shutdown()
+    interpreter.close()
+  }
 
-  def close(): Unit
+  def clearHistory() = synchronized {
+    _history = IndexedSeq()
+  }
 
   @throws(classOf[TimeoutException])
   @throws(classOf[InterruptedException])
-  final def waitForStateChange(oldState: State, atMost: Duration) = {
+  def waitForStateChange(oldState: State, atMost: Duration) = {
     Utils.waitUntil({ () => state != oldState }, atMost)
   }
+
+  private def executeCode(executionCount: Int, code: String) = {
+    _state = Busy()
+
+    try {
+
+      interpreter.execute(code) match {
+        case Interpreter.ExecuteSuccess(data) =>
+          _state = Idle()
+
+          (STATUS -> OK) ~
+          (EXECUTION_COUNT -> executionCount) ~
+          (DATA -> data)
+        case Interpreter.ExecuteIncomplete() =>
+          _state = Idle()
+
+          (STATUS -> ERROR) ~
+          (EXECUTION_COUNT -> executionCount) ~
+          (ENAME -> "Error") ~
+          (EVALUE -> "incomplete statement") ~
+          (TRACEBACK -> List())
+        case Interpreter.ExecuteError(ename, evalue, traceback) =>
+          _state = Idle()
+
+          (STATUS -> ERROR) ~
+          (EXECUTION_COUNT -> executionCount) ~
+          (ENAME -> ename) ~
+          (EVALUE -> evalue) ~
+          (TRACEBACK -> traceback)
+        case Interpreter.ExecuteAborted(message) =>
+          _state = Error()
+
+          (STATUS -> ERROR) ~
+          (EXECUTION_COUNT -> executionCount) ~
+          (ENAME -> "Error") ~
+          (EVALUE -> f"Interpreter died:\n$message") ~
+          (TRACEBACK -> List())
+      }
+    } catch {
+      case e: Throwable =>
+        error("Exception when executing code", e)
+
+        _state = Idle()
+
+
+        (STATUS -> ERROR) ~
+        (EXECUTION_COUNT -> executionCount) ~
+        (ENAME -> f"Internal Error: ${e.getClass.getName}") ~
+        (EVALUE -> e.getMessage) ~
+        (TRACEBACK -> List())
+    }
+  }
 }
 
-case class Statement(id: Int, result: Future[JValue])
+case class Statement(id: Int, result: Future[JValue])

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

@@ -95,18 +95,7 @@ private object Serializers {
   def Formats: List[CustomSerializer[_]] = List(StatementSerializer)
 
   def serializeSession(session: Session): JValue = {
-    val state = session.state match {
-      case NotStarted() => "not_started"
-      case Starting() => "starting"
-      case Idle() => "idle"
-      case Busy() => "busy"
-      case Running() => "running"
-      case Error() => "error"
-      case ShuttingDown() => "shutting_down"
-      case Dead() => "dead"
-      case Success() => "success"
-    }
-    Map("state" -> state)
+    Map("state" -> session.state.toString)
   }
 
   def serializeHistory(history: IndexedSeq[Statement],

+ 30 - 0
apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/package.scala

@@ -0,0 +1,30 @@
+/*
+ * 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
+
+import org.json4s.JField
+
+package object repl {
+  type MimeTypeMap = List[JField]
+
+  val APPLICATION_JSON = "application/json"
+  val APPLICATION_LIVY_TABLE_JSON = "application/vnd.livy.table.v1+json"
+  val IMAGE_PNG = "image/png"
+  val TEXT_PLAIN = "text/plain"
+}

+ 73 - 120
apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/process/ProcessInterpreter.scala

@@ -19,166 +19,119 @@
 package com.cloudera.hue.livy.repl.process
 
 import java.io.{BufferedReader, IOException, InputStreamReader, PrintWriter}
-import java.util.concurrent.{LinkedBlockingQueue, TimeUnit}
+import java.util.concurrent.locks.ReentrantLock
 
 import com.cloudera.hue.livy.Logging
 import com.cloudera.hue.livy.repl.Interpreter
-import com.cloudera.hue.livy.sessions._
 import org.json4s.JValue
 
-import scala.annotation.tailrec
-import scala.concurrent.duration.Duration
-import scala.concurrent.{Await, ExecutionContext, Future, Promise}
-import scala.util
+import scala.concurrent.Promise
+import scala.io.Source
 
 private sealed trait Request
 private case class ExecuteRequest(code: String, promise: Promise[JValue]) extends Request
 private case class ShutdownRequest(promise: Promise[Unit]) extends Request
 
+/**
+ * Abstract trait that describes an interpreter that is running in a separate process.
+ *
+ * This type is not thread safe, so must be protected by a mutex.
+ *
+ * @param process
+ */
 abstract class ProcessInterpreter(process: Process)
   extends Interpreter
   with Logging
 {
-  implicit val executor: ExecutionContext = ExecutionContext.global
-
-  protected[this] var _state: State = Starting()
-
   protected[this] val stdin = new PrintWriter(process.getOutputStream)
   protected[this] val stdout = new BufferedReader(new InputStreamReader(process.getInputStream), 1)
 
-  private[this] val _queue = new LinkedBlockingQueue[Request]
+  override def start() = {
+    waitUntilReady()
+  }
 
-  override def state: State = _state
+  override def execute(code: String): Interpreter.ExecuteResponse = {
+    try {
+      sendExecuteRequest(code)
+    } catch {
+      case e: Throwable =>
+        Interpreter.ExecuteError(e.getClass.getName, e.getMessage)
+    }
+  }
 
-  override def execute(code: String): Future[JValue] = {
-    _state match {
-      case (Dead() | ShuttingDown() | Error()) =>
-        Future.failed(new IllegalStateException("interpreter is not running"))
-      case _ =>
-        val promise = Promise[JValue]()
-        _queue.add(ExecuteRequest(code, promise))
-        promise.future
+  override def close(): Unit = {
+    if (isProcessAlive) {
+      logger.info("Shutting down process")
+      sendShutdownRequest()
+
+      try {
+        process.getInputStream.close()
+        process.getOutputStream.close()
+      } catch {
+        case _: IOException =>
+      }
+
+      try {
+        process.destroy()
+      } finally {
+        logger.info("process has been shut down")
+      }
     }
   }
 
-  protected def waitUntilReady(): Unit
+  private def isProcessAlive = {
+    try {
+      process.exitValue()
+      false
+    } catch {
+      case _: IllegalThreadStateException =>
+        true
+    }
+  }
 
-  protected def sendExecuteRequest(request: String): Option[JValue]
+  protected def sendExecuteRequest(request: String): Interpreter.ExecuteResponse
 
   protected def sendShutdownRequest(): Unit = {}
 
-  private[this] val thread = new Thread("process interpreter") {
-    override def run() = {
-      waitUntilReady()
-
-      _state = Idle()
+  protected def waitUntilReady(): Unit
 
-      loop()
-    }
+  private[this] val stderrLock = new ReentrantLock()
+  private[this] var stderrLines = Seq[String]()
 
-    @tailrec
-    private def loop(): Unit = {
-      (_state, _queue.take()) match {
-        case (Error(), ExecuteRequest(code, promise)) =>
-          promise.failure(new Exception("session has been terminated"))
-          loop()
-
-        case (state, ExecuteRequest(code, promise)) =>
-          require(state == Idle())
-
-          _state = Busy()
-
-          sendExecuteRequest(code) match {
-            case Some(rep) =>
-              synchronized {
-                _state = Idle()
-              }
-
-              promise.success(rep)
-            case None =>
-              synchronized {
-                _state = Error()
-              }
-
-              promise.failure(new Exception("session has been terminated"))
-          }
-          loop()
-
-        case (_, ShutdownRequest(promise)) =>
-          require(state == Idle() || state == Error())
-
-          synchronized {
-            _state = ShuttingDown()
-          }
-
-          try {
-            sendShutdownRequest()
-
-            try {
-              process.getInputStream.close()
-              process.getOutputStream.close()
-            } catch {
-              case _: IOException =>
-            }
-
-            try {
-              process.destroy()
-            } finally {
-              synchronized {
-                _state = Dead()
-              }
-
-              promise.success(())
-            }
-          }
-      }
+  protected def takeErrorLines(): String = {
+    stderrLock.lock()
+    try {
+      val lines = stderrLines
+      stderrLines = Seq()
+      lines.mkString("\n")
+    } finally {
+      stderrLock.unlock()
     }
   }
 
-  thread.start()
-
-  override def close(): Unit = {
-    val future = synchronized {
-      _state match {
-        case (Dead() | ShuttingDown()) =>
-          Future.successful()
-        case _ =>
-          val promise = Promise[Unit]()
-          _queue.add(ShutdownRequest(promise))
-
-          promise.future.andThen {
-            case util.Success(_) =>
-              thread.join()
-            case util.Failure(_) =>
-              thread.interrupt()
-              thread.join()
-          }
+  private[this] val stderrThread = new Thread("process stderr thread") {
+    override def run() = {
+      val lines = Source.fromInputStream(process.getErrorStream).getLines()
+
+      for (line <- lines) {
+        stderrLock.lock()
+        try {
+          stderrLines :+= line
+        } finally {
+          stderrLock.unlock()
+        }
       }
     }
-
-    // Give ourselves 10 seconds to tear down the process.
-    try {
-      Await.result(future, Duration(10, TimeUnit.SECONDS))
-    } catch {
-      case e: Throwable =>
-        // Make sure if there are any problems we make sure we kill the process.
-        process.destroy()
-        thread.interrupt()
-        throw e
-    }
   }
 
+  stderrThread.setDaemon(true)
+  stderrThread.start()
 
   private[this] val processWatcherThread = new Thread("process watcher thread") {
     override def run() = {
       val exitCode = process.waitFor()
       if (exitCode != 0) {
-        _state = Error()
-
-        // Give livy-server a moment to see that we've died.
-        Thread.sleep(1000)
-
-        System.exit(1)
+        error(f"Process has died with $exitCode")
       }
     }
   }

+ 34 - 18
apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/python/PythonInterpreter.scala

@@ -26,6 +26,7 @@ import com.cloudera.hue.livy.repl.Interpreter
 import com.cloudera.hue.livy.repl.process.ProcessInterpreter
 import com.cloudera.hue.livy.{Logging, Utils}
 import org.apache.spark.SparkContext
+import org.json4s.JsonAST.JObject
 import org.json4s.jackson.JsonMethods._
 import org.json4s.jackson.Serialization.write
 import org.json4s.{DefaultFormats, JValue}
@@ -34,34 +35,32 @@ import py4j.GatewayServer
 import scala.annotation.tailrec
 import scala.collection.JavaConversions._
 import scala.collection.mutable.ArrayBuffer
-import scala.concurrent.ExecutionContext
 
-object PythonInterpreter {
-  def create(): Interpreter = {
+object PythonInterpreter extends Logging {
+  def apply(): Interpreter = {
     val pythonExec = sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python")
 
     val gatewayServer = new GatewayServer(null, 0)
     gatewayServer.start()
 
-    val builder = new ProcessBuilder(Seq(
-      pythonExec,
-      createFakeShell().toString
-    ))
+    val pythonPath = buildPythonPath
+    val builder = new ProcessBuilder(Seq(pythonExec, createFakeShell().toString))
 
     val env = builder.environment()
+
     env.put("PYTHONPATH", pythonPath)
     env.put("PYTHONUNBUFFERED", "YES")
     env.put("PYSPARK_GATEWAY_PORT", "" + gatewayServer.getListeningPort)
     env.put("SPARK_HOME", sys.env.getOrElse("SPARK_HOME", "."))
 
-    builder.redirectError(Redirect.INHERIT)
+    builder.redirectError(Redirect.PIPE)
 
     val process = builder.start()
 
     new PythonInterpreter(process, gatewayServer)
   }
 
-  private def pythonPath = {
+  private def buildPythonPath = {
     val pythonPath = new ArrayBuffer[String]
     for (sparkHome <- sys.env.get("SPARK_HOME")) {
       pythonPath += Seq(sparkHome, "python", "lib", "pyspark.zip").mkString(File.separator)
@@ -122,6 +121,8 @@ private class PythonInterpreter(process: Process, gatewayServer: GatewayServer)
 {
   implicit val formats = DefaultFormats
 
+  override def kind = "pyspark"
+
   override def close(): Unit = {
     try {
       super.close()
@@ -139,14 +140,27 @@ private class PythonInterpreter(process: Process, gatewayServer: GatewayServer)
     }
   }
 
-  override protected def sendExecuteRequest(code: String): Option[JValue] = {
-    val rep = sendRequest(Map("msg_type" -> "execute_request", "content" -> Map("code" -> code)))
-    rep.map { case rep =>
-      assert((rep \ "msg_type").extract[String] == "execute_reply")
-
-      val content: JValue = rep \ "content"
-
-      content
+  override protected def sendExecuteRequest(code: String): Interpreter.ExecuteResponse = {
+    sendRequest(Map("msg_type" -> "execute_request", "content" -> Map("code" -> code))) match {
+      case Some(response) =>
+        assert((response \ "msg_type").extract[String] == "execute_reply")
+
+        val content = response \ "content"
+
+        (content \ "status").extract[String] match {
+          case "ok" =>
+            Interpreter.ExecuteSuccess((content \ "data").extract[JObject])
+          case "error" =>
+            val ename = (content \ "ename").extract[String]
+            val evalue = (content \ "evalue").extract[String]
+            val traceback = (content \ "traceback").extract[Seq[String]]
+
+            Interpreter.ExecuteError(ename, evalue, traceback)
+          case status =>
+            Interpreter.ExecuteError("Internal Error", f"Unknown status $status")
+        }
+      case None =>
+        Interpreter.ExecuteAborted(takeErrorLines())
     }
   }
 
@@ -163,6 +177,8 @@ private class PythonInterpreter(process: Process, gatewayServer: GatewayServer)
     stdin.println(write(request))
     stdin.flush()
 
-    Option(stdout.readLine()).map { case line => parse(line) }
+    Option(stdout.readLine()).map { case line =>
+      parse(line)
+    }
   }
 }

+ 0 - 59
apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/python/PythonSession.scala

@@ -1,59 +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.repl.python
-
-import com.cloudera.hue.livy.Logging
-import com.cloudera.hue.livy.repl.{Interpreter, Session, Statement}
-import com.cloudera.hue.livy.sessions._
-import org.json4s.JValue
-
-import scala.concurrent._
-
-object PythonSession {
-  def create(): Session = {
-    new PythonSession(PythonInterpreter.create())
-  }
-}
-
-private class PythonSession(interpreter: Interpreter) extends Session with Logging {
-  private implicit def executor: ExecutionContext = ExecutionContext.global
-
-  private var _history = IndexedSeq[Statement]()
-
-  override def kind = PySpark()
-
-  override def state = interpreter.state
-
-  override def history: IndexedSeq[Statement] = _history
-
-  override def execute(code: String): Statement = synchronized {
-    val result = interpreter.execute(code)
-    val statement = Statement(_history.length, result)
-    _history :+= statement
-    statement
-  }
-
-  override def close(): Unit = interpreter.close()
-}
-
-private sealed trait Request
-private case class ExecuteRequest(code: String, promise: Promise[JValue]) extends Request
-private case class ShutdownRequest(promise: Promise[Unit]) extends Request
-
-case class ExecuteResponse(content: JValue)

+ 260 - 0
apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/scala/SparkInterpreter.scala

@@ -0,0 +1,260 @@
+/*
+ * 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.repl.scala
+
+import java.io._
+
+import com.cloudera.hue.livy.repl
+import com.cloudera.hue.livy.repl.Interpreter
+import org.apache.spark.rdd.RDD
+import org.apache.spark.repl.SparkIMain
+import org.apache.spark.{SparkConf, SparkContext}
+import org.json4s.JsonAST._
+import org.json4s.JsonDSL._
+import org.json4s.{DefaultFormats, Extraction}
+
+import scala.tools.nsc.Settings
+import scala.tools.nsc.interpreter.{JPrintWriter, Results}
+
+
+object SparkInterpreter {
+  private val MAGIC_REGEX = "^%(\\w+)\\W*(.*)".r
+
+  def apply(): SparkInterpreter = { new SparkInterpreter }
+}
+
+/**
+ * This represents a Spark interpreter. It is not thread safe.
+ */
+class SparkInterpreter extends Interpreter {
+  import SparkInterpreter._
+
+  private implicit def formats = DefaultFormats
+
+  private val outputStream = new ByteArrayOutputStream()
+  private var sparkIMain: SparkIMain = _
+  private var sparkContext: SparkContext = _
+
+  def kind = "spark"
+
+  override def start() = {
+    require(sparkIMain == null && sparkContext == null)
+
+    val settings = new Settings()
+    settings.usejavacp.value = true
+
+    sparkIMain = new SparkIMain(settings, new JPrintWriter(outputStream, true))
+    sparkIMain.initializeSynchronous()
+
+    val sparkConf = new SparkConf(true)
+      .setAppName("Livy Spark shell")
+      .set("spark.repl.class.uri", sparkIMain.classServerUri)
+
+    sparkContext = new SparkContext(sparkConf)
+
+    sparkIMain.beQuietDuring {
+      sparkIMain.bind("sc", "org.apache.spark.SparkContext", sparkContext, List("""@transient"""))
+    }
+  }
+
+  override def execute(code: String): Interpreter.ExecuteResponse = {
+    require(sparkIMain != null && sparkContext != null)
+
+    executeLines(code.trim.split("\n").toList, Interpreter.ExecuteSuccess(JObject(
+      (repl.TEXT_PLAIN, JString(""))
+    )))
+  }
+
+  override def close(): Unit = synchronized {
+    if (sparkContext != null) {
+      sparkContext.stop()
+    }
+
+    if (sparkIMain != null) {
+      sparkIMain.close()
+      sparkIMain = null
+    }
+  }
+
+  private def executeMagic(magic: String, rest: String): Interpreter.ExecuteResponse = {
+    magic match {
+      case "json" => executeJsonMagic(rest)
+      case "table" => executeTableMagic(rest)
+      case _ =>
+        Interpreter.ExecuteError("UnknownMagic", f"Unknown magic command $magic")
+    }
+  }
+
+  private def executeJsonMagic(name: String): Interpreter.ExecuteResponse = {
+    try {
+      val value = sparkIMain.valueOfTerm(name) match {
+        case Some(obj: RDD[_]) => obj.asInstanceOf[RDD[_]].take(10)
+        case Some(obj) => obj
+        case None => return Interpreter.ExecuteError("NameError", f"Value $name does not exist")
+      }
+
+      Interpreter.ExecuteSuccess(JObject(
+        (repl.APPLICATION_JSON, Extraction.decompose(value))
+      ))
+    } catch {
+      case _: Throwable =>
+        Interpreter.ExecuteError("ValueError", "Failed to convert value into a JSON value")
+    }
+  }
+
+  private class TypesDoNotMatch extends Exception
+
+  private def convertTableType(value: JValue): String = {
+    value match {
+      case (JNothing | JNull) => "NULL_TYPE"
+      case JBool(_) => "BOOLEAN_TYPE"
+      case JString(_) => "STRING_TYPE"
+      case JInt(_) => "BIGINT_TYPE"
+      case JDouble(_) => "DOUBLE_TYPE"
+      case JDecimal(_) => "DECIMAL_TYPE"
+      case JArray(arr) =>
+        if (allSameType(arr.iterator)) {
+          "ARRAY_TYPE"
+        } else {
+          throw new TypesDoNotMatch
+        }
+      case JObject(obj) =>
+        if (allSameType(obj.iterator.map(_._2))) {
+          "MAP_TYPE"
+        } else {
+          throw new TypesDoNotMatch
+        }
+    }
+  }
+
+  private def allSameType(values: Iterator[JValue]): Boolean = {
+    if (values.hasNext) {
+      val type_name = convertTableType(values.next())
+      values.forall { case value => type_name.equals(convertTableType(value)) }
+    } else {
+      true
+    }
+  }
+
+  private def executeTableMagic(name: String): Interpreter.ExecuteResponse = {
+    val value = sparkIMain.valueOfTerm(name) match {
+      case Some(obj: RDD[_]) => obj.asInstanceOf[RDD[_]].take(10)
+      case Some(obj) => obj
+      case None => return Interpreter.ExecuteError("NameError", f"Value $name does not exist")
+    }
+
+    extractTableFromJValue(Extraction.decompose(value))
+  }
+
+  private def extractTableFromJValue(value: JValue): Interpreter.ExecuteResponse = {
+    // Convert the value into JSON and map it to a table.
+    val rows: List[JValue] = value match {
+      case JArray(arr) => arr
+      case _ => List(value)
+    }
+
+    try {
+      val headers = scala.collection.mutable.Map[String, Map[String, String]]()
+
+      val data = rows.map { case row =>
+        val cols: List[JField] = row match {
+          case JArray(arr: List[JValue]) =>
+            arr.zipWithIndex.map { case (v, index) => JField(index.toString, v) }
+          case JObject(obj) => obj.sortBy(_._1)
+          case value: JValue => List(JField("0", value))
+        }
+
+        cols.map { case (name, value) =>
+          val typeName = convertTableType(value)
+
+          headers.get(name) match {
+            case Some(header) =>
+              if (header.get("type").get != typeName) {
+                throw new TypesDoNotMatch
+              }
+            case None =>
+              headers.put(name, Map(
+                "type" -> typeName,
+                "name" -> name
+              ))
+          }
+
+          value
+        }
+      }
+
+      Interpreter.ExecuteSuccess(
+        repl.APPLICATION_LIVY_TABLE_JSON -> (
+          ("headers" -> headers.toSeq.sortBy(_._1).map(_._2)) ~ ("data" -> data)
+        ))
+    } catch {
+      case _: TypesDoNotMatch =>
+        Interpreter.ExecuteError("TypeError", "table rows have different types")
+    }
+  }
+
+  private def executeLines(lines: List[String], result: Interpreter.ExecuteResponse): Interpreter.ExecuteResponse = {
+    lines match {
+      case Nil => result
+      case head :: tail =>
+        val result = executeLine(head)
+
+        result match {
+          case Interpreter.ExecuteIncomplete() =>
+            tail match {
+              case Nil =>
+                result
+
+              case next :: nextTail =>
+                executeLines(head + "\n" + next :: nextTail, result)
+            }
+          case Interpreter.ExecuteError(_, _, _) =>
+            result
+
+          case _ =>
+            executeLines(tail, result)
+        }
+    }
+  }
+
+  private def executeLine(code: String): Interpreter.ExecuteResponse = {
+    code match {
+      case MAGIC_REGEX(magic, rest) =>
+        executeMagic(magic, rest)
+      case _ =>
+        scala.Console.withOut(outputStream) {
+          sparkIMain.interpret(code) match {
+            case Results.Success =>
+              Interpreter.ExecuteSuccess(
+                repl.TEXT_PLAIN -> readStdout()
+              )
+            case Results.Incomplete => Interpreter.ExecuteIncomplete()
+            case Results.Error => Interpreter.ExecuteError("Error", readStdout())
+          }
+        }
+    }
+  }
+
+  private def readStdout() = {
+    val output = outputStream.toString("UTF-8").trim
+    outputStream.reset()
+
+    output
+  }
+}

+ 0 - 341
apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/scala/SparkInterpreter/SparkInterpreter.scala

@@ -1,341 +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.repl.scala.SparkInterpreter
-
-import java.io._
-
-import com.cloudera.hue.livy.repl.Interpreter
-import com.cloudera.hue.livy.sessions._
-import org.apache.spark.rdd.RDD
-import org.apache.spark.repl.SparkIMain
-import org.apache.spark.{SparkConf, SparkContext}
-import org.json4s.JsonAST._
-import org.json4s.{DefaultFormats, Extraction}
-
-import scala.concurrent.{Future, ExecutionContext}
-import scala.tools.nsc.Settings
-import scala.tools.nsc.interpreter.{JPrintWriter, Results}
-
-
-object SparkInterpreter {
-  private val MAGIC_REGEX = "^%(\\w+)\\W*(.*)".r
-}
-
-sealed abstract class ExecuteResponse(executeCount: Int)
-case class ExecuteComplete(executeCount: Int, output: String) extends ExecuteResponse(executeCount)
-case class ExecuteIncomplete(executeCount: Int, output: String) extends ExecuteResponse(executeCount)
-case class ExecuteError(executeCount: Int, output: String) extends ExecuteResponse(executeCount)
-case class ExecuteMagic(executeCount: Int, content: JValue) extends ExecuteResponse(executeCount)
-
-class SparkInterpreter extends Interpreter {
-  import SparkInterpreter._
-
-  private implicit def executor: ExecutionContext = ExecutionContext.global
-  private implicit def formats = DefaultFormats
-
-  private var _state: State = NotStarted()
-  private val outputStream = new ByteArrayOutputStream()
-  private var sparkIMain: SparkIMain = _
-  private var sparkContext: SparkContext = _
-  private var executeCount = 0
-
-  @Override
-  def state = _state
-
-  def start() = {
-    require(_state == NotStarted() && sparkIMain == null)
-
-    _state = Starting()
-
-    class InterpreterClassLoader(classLoader: ClassLoader) extends ClassLoader(classLoader) {}
-    val classLoader = new InterpreterClassLoader(classOf[SparkInterpreter].getClassLoader)
-
-    val settings = new Settings()
-    settings.usejavacp.value = true
-
-    sparkIMain = createSparkIMain(classLoader, settings)
-    sparkIMain.initializeSynchronous()
-
-    val sparkConf = new SparkConf(true)
-      .setAppName("Livy Spark shell")
-      .set("spark.repl.class.uri", sparkIMain.classServerUri)
-
-    sparkContext = new SparkContext(sparkConf)
-
-    sparkIMain.beQuietDuring {
-      sparkIMain.bind("sc", "org.apache.spark.SparkContext", sparkContext, List("""@transient"""))
-    }
-
-    _state = Idle()
-  }
-
-  private def getMaster(): String = {
-    sys.props.get("spark.master").getOrElse("local[*]")
-  }
-
-  private def createSparkIMain(classLoader: ClassLoader, settings: Settings) = {
-    val out = new JPrintWriter(outputStream, true)
-    val cls = classLoader.loadClass(classOf[SparkIMain].getName)
-    val constructor = cls.getConstructor(classOf[Settings], classOf[JPrintWriter], java.lang.Boolean.TYPE)
-    constructor.newInstance(settings, out, false: java.lang.Boolean).asInstanceOf[SparkIMain]
-  }
-
-  private def executeMagic(magic: String, rest: String): ExecuteResponse = {
-    magic match {
-      case "json" => executeJsonMagic(rest)
-      case "table" => executeTableMagic(rest)
-      case _ =>
-        ExecuteError(executeCount, f"Unknown magic command $magic")
-    }
-  }
-
-  private def executeJsonMagic(name: String): ExecuteResponse = {
-    sparkIMain.valueOfTerm(name) match {
-      case Some(value: RDD[_]) =>
-        ExecuteMagic(
-          executeCount,
-          Extraction.decompose(Map(
-            "application/json" -> value.asInstanceOf[RDD[_]].take(10)
-          ))
-        )
-      case Some(value) =>
-        ExecuteMagic(
-          executeCount,
-          Extraction.decompose(Map(
-            "application/json" -> value
-          ))
-        )
-      case None =>
-        ExecuteError(executeCount, f"Value $name does not exist")
-    }
-  }
-
-  private class TypesDoNotMatch extends Exception
-
-  private def convertTableType(value: JValue): String = {
-    value match {
-      case (JNothing | JNull) => "NULL_TYPE"
-      case JBool(_) => "BOOLEAN_TYPE"
-      case JString(_) => "STRING_TYPE"
-      case JInt(_) => "BIGINT_TYPE"
-      case JDouble(_) => "DOUBLE_TYPE"
-      case JDecimal(_) => "DECIMAL_TYPE"
-      case JArray(arr) =>
-        if (allSameType(arr.iterator)) {
-          "ARRAY_TYPE"
-        } else {
-          throw new TypesDoNotMatch
-        }
-      case JObject(obj) =>
-        if (allSameType(obj.iterator.map(_._2))) {
-          "MAP_TYPE"
-        } else {
-          throw new TypesDoNotMatch
-        }
-    }
-  }
-
-  private def allSameType(values: Iterator[JValue]): Boolean = {
-    if (values.hasNext) {
-      val type_name = convertTableType(values.next())
-      values.forall { case value => type_name.equals(convertTableType(value)) }
-    } else {
-      true
-    }
-  }
-
-  private def executeTableMagic(name: String): ExecuteResponse = {
-    try {
-      sparkIMain.valueOfTerm(name) match {
-        case None =>
-          ExecuteError(executeCount, f"Value $name does not exist")
-        case Some(obj: RDD[_]) =>
-          extractTableFromJValue(Extraction.decompose(
-            obj.asInstanceOf[RDD[_]].take(10)))
-        case Some(obj) =>
-          extractTableFromJValue(Extraction.decompose(obj))
-      }
-    } catch {
-      case _: Throwable =>
-        ExecuteError(executeCount, "Failed to convert value into a table")
-    }
-  }
-
-  private def extractTableFromJValue(value: JValue) = {
-    // Convert the value into JSON and map it to a table.
-    val rows: List[JValue] = value match {
-      case JArray(arr) => arr
-      case _ => List(value)
-    }
-
-    try {
-      val headers = scala.collection.mutable.Map[String, Map[String, String]]()
-
-      val data = rows.map { case row =>
-        val cols: List[JField] = row match {
-          case JArray(arr: List[JValue]) =>
-            arr.zipWithIndex.map { case (v, index) => JField(index.toString, v) }
-          case JObject(obj) => obj.sortBy(_._1)
-          case value: JValue => List(JField("0", value))
-        }
-
-        cols.map { case (name, value) =>
-          val typeName = convertTableType(value)
-
-          headers.get(name) match {
-            case Some(header) =>
-              if (header.get("type").get != typeName) {
-                throw new TypesDoNotMatch
-              }
-            case None =>
-              headers.put(name, Map(
-                "name" -> name,
-                "type" -> typeName
-              ))
-          }
-
-          value
-        }
-      }
-
-      ExecuteMagic(
-        executeCount,
-        Extraction.decompose(Map(
-          "application/vnd.livy.table.v1+json" -> Map(
-            "headers" -> headers.toSeq.sortBy(_._1).map(_._2),
-            "data" -> data
-          )
-        ))
-      )
-    } catch {
-      case _: TypesDoNotMatch =>
-        ExecuteError(
-          executeCount,
-          "table rows have different types"
-        )
-    }
-  }
-
-  @Override
-  def execute(code: String): Future[JValue] = {
-    executeCount += 1
-
-    synchronized {
-      _state = Busy()
-    }
-
-    Future {
-      val result = executeLines(code.trim.split("\n").toList, ExecuteComplete(executeCount, ""))
-
-      val response = result match {
-        case ExecuteComplete(executeCount, output) =>
-          Map(
-            "status" -> "ok",
-            "execution_count" -> executeCount,
-            "data" -> Map(
-              "text/plain" -> output
-            )
-          )
-        case ExecuteMagic(executeCount, content) =>
-          Map(
-            "status" -> "ok",
-            "execution_count" -> executeCount,
-            "data" -> content
-          )
-        case ExecuteIncomplete(executeCount, output) =>
-          Map(
-            "status" -> "error",
-            "execution_count" -> executeCount,
-            "ename" -> "Error",
-            "evalue" -> output
-          )
-        case ExecuteError(executeCount, output) =>
-          Map(
-            "status" -> "error",
-            "execution_count" -> executeCount,
-            "ename" -> "Error",
-            "evalue" -> output
-          )
-      }
-
-      _state = Idle()
-
-      Extraction.decompose(response)
-    }
-  }
-
-  private def executeLines(lines: List[String], result: ExecuteResponse): ExecuteResponse = {
-    lines match {
-      case Nil => result
-      case head :: tail =>
-        val result = executeLine(head)
-
-        result match {
-          case ExecuteIncomplete(_, _) =>
-            tail match {
-              case Nil => result
-              case next :: nextTail => executeLines(head + "\n" + next :: nextTail, result)
-            }
-          case ExecuteError(_, _) =>
-            result
-
-          case _ =>
-            executeLines(tail, result)
-        }
-    }
-  }
-
-  private def executeLine(code: String) = {
-    code match {
-      case MAGIC_REGEX(magic, rest) =>
-        executeMagic(magic, rest)
-      case _ =>
-        scala.Console.withOut(outputStream) {
-          sparkIMain.interpret(code) match {
-            case Results.Success => ExecuteComplete(executeCount - 1, readStdout())
-            case Results.Incomplete => ExecuteIncomplete(executeCount - 1, readStdout())
-            case Results.Error => ExecuteError(executeCount - 1, readStdout())
-          }
-        }
-    }
-  }
-
-  private def readStdout() = {
-    val output = outputStream.toString("UTF-8").trim
-    outputStream.reset()
-
-    output
-  }
-
-  @Override
-  def close(): Unit = synchronized {
-    _state = ShuttingDown()
-
-    if (sparkContext != null) {
-      sparkContext.stop()
-    }
-
-    if (sparkIMain != null) {
-      sparkIMain.close()
-      sparkIMain = null
-    }
-
-    _state = Dead()
-  }
-}

+ 0 - 58
apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/scala/SparkSession.scala

@@ -1,58 +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.repl.scala
-
-import com.cloudera.hue.livy.repl.scala.SparkInterpreter._
-import com.cloudera.hue.livy.repl.{Session, Statement}
-import com.cloudera.hue.livy.sessions._
-import org.json4s._
-
-import scala.concurrent.ExecutionContext
-
-object SparkSession {
-  def create(): Session = new SparkSession()
-}
-
-private class SparkSession extends Session {
-  private implicit def executor: ExecutionContext = ExecutionContext.global
-
-  implicit val formats = DefaultFormats
-
-  private var _history = IndexedSeq[Statement]()
-  private val interpreter = new SparkInterpreter()
-  interpreter.start()
-
-  override def kind: Kind = Spark()
-
-  override def state: State = interpreter.state
-
-  override def history: IndexedSeq[Statement] = _history
-
-  override def execute(code: String): Statement = synchronized {
-    val result = interpreter.execute(code)
-
-    val statement = Statement(_history.length, result)
-    _history :+= statement
-    statement
-  }
-
-  override def close(): Unit = {
-    interpreter.close()
-  }
-}

+ 63 - 68
apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/sparkr/SparkRInterpreter.scala

@@ -18,24 +18,26 @@
 
 package com.cloudera.hue.livy.repl.sparkr
 
+import java.io.{File, FileOutputStream}
+import java.lang.ProcessBuilder.Redirect
 import java.nio.file.Files
-import java.util.concurrent.locks.ReentrantLock
 
+import com.cloudera.hue.livy.repl
+import com.cloudera.hue.livy.repl.Interpreter
 import com.cloudera.hue.livy.repl.process.ProcessInterpreter
 import org.apache.commons.codec.binary.Base64
-import org.json4s.jackson.JsonMethods._
-import org.json4s.jackson.Serialization.write
-import org.json4s.{JValue, _}
+import org.json4s.JsonDSL._
+import org.json4s._
 
 import scala.annotation.tailrec
-import scala.io.Source
+import scala.collection.JavaConversions._
 
-private object SparkRInterpreter {
-  val LIVY_END_MARKER = "----LIVY_END_OF_COMMAND----"
-  val PRINT_MARKER = f"""print("$LIVY_END_MARKER")"""
-  val EXPECTED_OUTPUT = f"""[1] "$LIVY_END_MARKER""""
+object SparkRInterpreter {
+  private val LIVY_END_MARKER = "----LIVY_END_OF_COMMAND----"
+  private val PRINT_MARKER = f"""print("$LIVY_END_MARKER")"""
+  private val EXPECTED_OUTPUT = f"""[1] "$LIVY_END_MARKER""""
 
-  val PLOT_REGEX = (
+  private val PLOT_REGEX = (
     "(" +
       "(?:bagplot)|" +
       "(?:barplot)|" +
@@ -56,9 +58,50 @@ private object SparkRInterpreter {
       "(?:vioplot)" +
     ")"
     ).r.unanchored
+
+  def apply(): SparkRInterpreter = {
+    val sparkrExec = sys.env.getOrElse("SPARKR_DRIVER_R", "sparkR")
+
+    val builder = new ProcessBuilder(Seq(
+      sparkrExec
+    ))
+
+    val env = builder.environment()
+    env.put("SPARK_HOME", sys.env.getOrElse("SPARK_HOME", "."))
+    env.put("SPARKR_DRIVER_R", createFakeShell().toString)
+
+    builder.redirectError(Redirect.PIPE)
+
+    val process = builder.start()
+
+    new SparkRInterpreter(process)
+  }
+
+  private def createFakeShell(): File = {
+    val source = getClass.getClassLoader.getResourceAsStream("fake_R.sh")
+
+    val file = Files.createTempFile("", "").toFile
+    file.deleteOnExit()
+
+    val sink = new FileOutputStream(file)
+    val buf = new Array[Byte](1024)
+    var n = source.read(buf)
+
+    while (n > 0) {
+      sink.write(buf, 0, n)
+      n = source.read(buf)
+    }
+
+    source.close()
+    sink.close()
+
+    file.setExecutable(true)
+
+    file
+  }
 }
 
-private class SparkRInterpreter(process: Process)
+class SparkRInterpreter(process: Process)
   extends ProcessInterpreter(process)
 {
   import SparkRInterpreter._
@@ -67,13 +110,15 @@ private class SparkRInterpreter(process: Process)
 
   private[this] var executionCount = 0
 
+  override def kind = "sparkR"
+
   final override protected def waitUntilReady(): Unit = {
     // Set the option to catch and ignore errors instead of halting.
     sendExecuteRequest("options(error = dump.frames)")
     executionCount = 0
   }
 
-  override protected def sendExecuteRequest(command: String): Option[JValue] = synchronized {
+  override protected def sendExecuteRequest(command: String): Interpreter.ExecuteResponse = {
     var code = command
 
     // Create a image file if this command is trying to plot.
@@ -87,40 +132,24 @@ private class SparkRInterpreter(process: Process)
     }
 
     try {
-      executionCount += 1
-
-      var content = Map(
-        "text/plain" -> (sendRequest(code) + takeErrorLines())
-      )
+      var content: JObject = repl.TEXT_PLAIN -> (sendRequest(code) + takeErrorLines())
 
       // If we rendered anything, pass along the last image.
       tempFile.foreach { case file =>
         val bytes = Files.readAllBytes(file)
         if (bytes.nonEmpty) {
           val image = Base64.encodeBase64String(bytes)
-          content = content + (("image/png", image))
+          content = content ~ (repl.IMAGE_PNG -> image)
         }
       }
 
-      Some(parse(write(
-        Map(
-          "status" -> "ok",
-          "execution_count" -> (executionCount - 1),
-          "data" -> content
-        ))))
+      Interpreter.ExecuteSuccess(content)
     } catch {
       case e: Error =>
-        Some(parse(write(
-        Map(
-          "status" -> "error",
-          "ename" -> "Error",
-          "evalue" -> e.output,
-          "data" -> Map(
-            "text/plain" -> takeErrorLines()
-          )
-        ))))
+        val message = Seq(e.output, takeErrorLines()).mkString("\n")
+        Interpreter.ExecuteError("Error", message)
       case e: Exited =>
-        None
+        Interpreter.ExecuteAborted(takeErrorLines())
     } finally {
       tempFile.foreach(Files.delete)
     }
@@ -201,38 +230,4 @@ private class SparkRInterpreter(process: Process)
 
   private class Exited(val output: String) extends Exception {}
   private class Error(val output: String) extends Exception {}
-
-  private[this] val _lock = new ReentrantLock()
-  private[this] var stderrLines = Seq[String]()
-
-  private def takeErrorLines(): String = {
-    var lines: Seq[String] = null
-    _lock.lock()
-    try {
-      lines = stderrLines
-      stderrLines = Seq[String]()
-    } finally {
-      _lock.unlock()
-    }
-
-    lines.mkString("\n")
-  }
-
-  private[this] val stderrThread = new Thread("sparkr stderr thread") {
-    override def run() = {
-      val lines = Source.fromInputStream(process.getErrorStream).getLines()
-
-      for (line <- lines) {
-        _lock.lock()
-        try {
-          stderrLines :+= line
-        } finally {
-          _lock.unlock()
-        }
-      }
-    }
-  }
-
-  stderrThread.setDaemon(true)
-  stderrThread.start()
 }

+ 0 - 94
apps/spark/java/livy-repl/src/main/scala/com/cloudera/hue/livy/repl/sparkr/SparkRSession.scala

@@ -1,94 +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.repl.sparkr
-
-import java.io.{File, FileOutputStream}
-import java.lang.ProcessBuilder.Redirect
-import java.nio.file.Files
-
-import com.cloudera.hue.livy.repl.{Session, Statement}
-import com.cloudera.hue.livy.sessions._
-
-import scala.collection.JavaConversions._
-
-object SparkRSession {
-  def create(): Session = {
-    val sparkrExec = sys.env.getOrElse("SPARKR_DRIVER_R", "sparkR")
-
-    val builder = new ProcessBuilder(Seq(
-      sparkrExec
-    ))
-
-    val env = builder.environment()
-    env.put("SPARK_HOME", sys.env.getOrElse("SPARK_HOME", "."))
-    env.put("SPARKR_DRIVER_R", createFakeShell().toString)
-
-    builder.redirectError(Redirect.PIPE)
-
-    val process = builder.start()
-
-    val interpreter = new SparkRInterpreter(process)
-
-    new SparkRSession(interpreter)
-  }
-
-  private def createFakeShell(): File = {
-    val source = getClass.getClassLoader.getResourceAsStream("fake_R.sh")
-
-    val file = Files.createTempFile("", "").toFile
-    file.deleteOnExit()
-
-    val sink = new FileOutputStream(file)
-    val buf = new Array[Byte](1024)
-    var n = source.read(buf)
-
-    while (n > 0) {
-      sink.write(buf, 0, n)
-      n = source.read(buf)
-    }
-
-    source.close()
-    sink.close()
-
-    file.setExecutable(true)
-
-    file
-  }
-}
-
-private class SparkRSession(interpreter: SparkRInterpreter) extends Session {
-  private var _history = IndexedSeq[Statement]()
-
-  override def kind: Kind = SparkR()
-
-  override def state: State = interpreter.state
-
-  override def execute(code: String): Statement = {
-    val result = interpreter.execute(code)
-    val statement = Statement(_history.length, result)
-    _history :+= statement
-    statement
-  }
-
-  override def close(): Unit = interpreter.close()
-
-  override def history: IndexedSeq[Statement] = _history
-}
-
-

+ 37 - 0
apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/BaseInterpreterSpec.scala

@@ -0,0 +1,37 @@
+/*
+ * 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.repl
+
+import org.scalatest.{BeforeAndAfter, Matchers, FunSpec}
+
+abstract class BaseInterpreterSpec extends FunSpec with Matchers with BeforeAndAfter {
+
+  def createInterpreter(): Interpreter
+
+  var interpreter: Interpreter = null
+
+  before {
+    interpreter = createInterpreter()
+    interpreter.start()
+  }
+
+  after {
+    interpreter.close()
+  }
+}

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

@@ -30,12 +30,12 @@ abstract class BaseSessionSpec extends FunSpec with Matchers with BeforeAndAfter
 
   implicit val formats = DefaultFormats
 
-  def createSession(): Session
+  def createInterpreter(): Interpreter
 
   var session: Session = null
 
   before {
-    session = createSession()
+    session = Session(createInterpreter())
   }
 
   after {
@@ -48,7 +48,7 @@ abstract class BaseSessionSpec extends FunSpec with Matchers with BeforeAndAfter
     }
 
     it("should eventually become the idle state") {
-      session.waitForStateChange(Starting(), Duration(10, TimeUnit.SECONDS))
+      session.waitForStateChange(Starting(), Duration(30, TimeUnit.SECONDS))
       session.state should equal (Idle())
     }
   }

+ 131 - 0
apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/PythonInterpreterSpec.scala

@@ -0,0 +1,131 @@
+/*
+ * 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.repl
+
+import com.cloudera.hue.livy.repl
+import com.cloudera.hue.livy.repl.python.PythonInterpreter
+import org.json4s.JsonDSL._
+import org.json4s.{DefaultFormats, JValue}
+
+class PythonInterpreterSpec extends BaseInterpreterSpec {
+
+  implicit val formats = DefaultFormats
+
+  override def createInterpreter() = PythonInterpreter()
+
+  describe("A python interpreter") {
+    it("should execute `1 + 2` == 3") {
+      val response = interpreter.execute("1 + 2")
+      response should equal (Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "3"
+      ))
+    }
+
+    it("should execute multiple statements") {
+      var response = interpreter.execute("x = 1")
+      response should equal (Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> ""
+      ))
+
+      response = interpreter.execute("y = 2")
+      response should equal (Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> ""
+      ))
+
+      response = interpreter.execute("x + y")
+      response should equal (Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "3"
+      ))
+    }
+
+    it("should execute multiple statements in one block") {
+      val response = interpreter.execute(
+        """
+          |x = 1
+          |
+          |y = 2
+          |
+          |x + y
+        """.stripMargin)
+      response should equal(Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "3"
+      ))
+    }
+
+    it("should do table magic") {
+      val response = interpreter.execute(
+        """x = [[1, 'a'], [3, 'b']]
+          |%table x
+        """.stripMargin)
+
+      response should equal(Interpreter.ExecuteSuccess(
+        repl.APPLICATION_LIVY_TABLE_JSON -> (
+          ("headers" -> List(
+            ("type" -> "INT_TYPE") ~ ("name" -> "0"),
+            ("type" -> "STRING_TYPE") ~ ("name" -> "1")
+          )) ~
+          ("data" -> List(
+            List[JValue](1, "a"),
+            List[JValue](3, "b")
+          ))
+        )
+      ))
+    }
+
+    it("should allow magic inside statements") {
+      val response = interpreter.execute(
+        """x = [[1, 'a'], [3, 'b']]
+          |%table x
+          |1 + 2
+        """.stripMargin)
+
+      response should equal(Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "3"
+      ))
+    }
+
+    it("should capture stdout") {
+      val response = interpreter.execute("print 'Hello World'")
+      response should equal(Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "Hello World"
+      ))
+    }
+
+    it("should report an error if accessing an unknown variable") {
+      val response = interpreter.execute("x")
+      response should equal(Interpreter.ExecuteError(
+        "NameError",
+        "name 'x' is not defined",
+        List(
+          "Traceback (most recent call last):\n",
+          "NameError: name 'x' is not defined\n"
+        )
+      ))
+    }
+
+    it("should execute spark commands") {
+      val response = interpreter.execute(
+        """sc.parallelize(xrange(0, 2)).map(lambda i: i + 1).collect()""")
+
+      response should equal(Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "[1, 2]"
+      ))
+    }
+  }
+}

+ 3 - 3
apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/PythonSessionSpec.scala

@@ -18,7 +18,7 @@
 
 package com.cloudera.hue.livy.repl
 
-import com.cloudera.hue.livy.repl.python.PythonSession
+import com.cloudera.hue.livy.repl.python.PythonInterpreter
 import org.json4s.Extraction
 import org.json4s.JsonAST.JValue
 
@@ -27,7 +27,7 @@ import _root_.scala.concurrent.duration.Duration
 
 class PythonSessionSpec extends BaseSessionSpec {
 
-  override def createSession() = PythonSession.create()
+  override def createInterpreter() = PythonInterpreter()
 
   describe("A python session") {
     it("should execute `1 + 2` == 3") {
@@ -97,7 +97,7 @@ class PythonSessionSpec extends BaseSessionSpec {
       val result = Await.result(statement.result, Duration.Inf)
       val expectedResult = Extraction.decompose(Map(
         "status" -> "ok",
-        "execution_count" -> 1,
+        "execution_count" -> 0,
         "data" -> Map(
           "application/vnd.livy.table.v1+json" -> Map(
             "headers" -> List(

+ 130 - 0
apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/ScalaInterpreterSpec.scala

@@ -0,0 +1,130 @@
+/*
+ * 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.repl
+
+import com.cloudera.hue.livy.repl
+import com.cloudera.hue.livy.repl.scala.SparkInterpreter
+import org.json4s.JsonDSL._
+import org.json4s.{DefaultFormats, JValue}
+
+class ScalaInterpreterSpec extends BaseInterpreterSpec {
+
+  implicit val formats = DefaultFormats
+
+  override def createInterpreter() = SparkInterpreter()
+
+  describe("A spark interpreter") {
+    it("should execute `1 + 2` == 3") {
+      val response = interpreter.execute("1 + 2")
+      response should equal (Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "res0: Int = 3"
+      ))
+    }
+
+    it("should execute multiple statements") {
+      var response = interpreter.execute("val x = 1")
+      response should equal (Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "x: Int = 1"
+      ))
+
+      response = interpreter.execute("val y = 2")
+      response should equal (Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "y: Int = 2"
+      ))
+
+      response = interpreter.execute("x + y")
+      response should equal (Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "res0: Int = 3"
+      ))
+    }
+
+    it("should execute multiple statements in one block") {
+      val response = interpreter.execute(
+        """
+          |val x = 1
+          |
+          |val y = 2
+          |
+          |x + y
+        """.stripMargin)
+      response should equal(Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "res2: Int = 3"
+      ))
+    }
+
+    it("should do table magic") {
+      val response = interpreter.execute(
+        """val x = List(List(1, "a"), List(3, "b"))
+          |%table x
+        """.stripMargin)
+
+      response should equal(Interpreter.ExecuteSuccess(
+        repl.APPLICATION_LIVY_TABLE_JSON -> (
+          ("headers" -> List(
+            ("type" -> "BIGINT_TYPE") ~ ("name" -> "0"),
+            ("type" -> "STRING_TYPE") ~ ("name" -> "1")
+          )) ~
+            ("data" -> List(
+              List[JValue](1, "a"),
+              List[JValue](3, "b")
+            ))
+          )
+      ))
+    }
+
+    it("should allow magic inside statements") {
+      val response = interpreter.execute(
+        """val x = List(List(1, "a"), List(3, "b"))
+          |%table x
+          |1 + 2
+        """.stripMargin)
+
+      response should equal(Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "res0: Int = 3"
+      ))
+    }
+
+    it("should capture stdout") {
+      val response = interpreter.execute("println(\"Hello World\")")
+      response should equal(Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "Hello World"
+      ))
+    }
+
+    it("should report an error if accessing an unknown variable") {
+      val response = interpreter.execute("x")
+      response should equal(Interpreter.ExecuteError(
+        "Error",
+        """<console>:8: error: not found: value x
+          |              x
+          |              ^""".stripMargin,
+        List()
+      ))
+    }
+
+    it("should execute spark commands") {
+      val response = interpreter.execute(
+        """sc.parallelize(0 to 1).map { i => i+1 }.collect""".stripMargin)
+
+      response should equal(Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "res0: Array[Int] = Array(1, 2)"
+      ))
+    }
+  }
+}

+ 105 - 0
apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/SparkRInterpreterSpec.scala

@@ -0,0 +1,105 @@
+/*
+ * 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.repl
+
+import com.cloudera.hue.livy.repl
+import com.cloudera.hue.livy.repl.sparkr.SparkRInterpreter
+import org.json4s.JsonDSL._
+import org.json4s.{JObject, DefaultFormats, JValue}
+
+class SparkRInterpreterSpec extends BaseInterpreterSpec {
+
+  implicit val formats = DefaultFormats
+
+  override def createInterpreter() = SparkRInterpreter()
+
+  describe("A python interpreter") {
+    it("should execute `1 + 2` == 3") {
+      val response = interpreter.execute("1 + 2")
+      response should equal (Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "[1] 3"
+      ))
+    }
+
+    it("should execute multiple statements") {
+      var response = interpreter.execute("x = 1")
+      response should equal (Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> ""
+      ))
+
+      response = interpreter.execute("y = 2")
+      response should equal (Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> ""
+      ))
+
+      response = interpreter.execute("x + y")
+      response should equal (Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "[1] 3"
+      ))
+    }
+
+    it("should execute multiple statements in one block") {
+      val response = interpreter.execute(
+        """
+          |x = 1
+          |
+          |y = 2
+          |
+          |x + y
+        """.stripMargin)
+      response should equal(Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "[1] 3"
+      ))
+    }
+
+    it("should capture stdout") {
+      val response = interpreter.execute("cat(3)")
+      response should equal(Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "3"
+      ))
+    }
+
+    it("should report an error if accessing an unknown variable") {
+      val response = interpreter.execute("x")
+      response should equal(Interpreter.ExecuteSuccess(
+        repl.TEXT_PLAIN -> "Error: object 'x' not found"
+      ))
+    }
+
+    it("should execute spark commands") {
+      val response = interpreter.execute(
+        """head(createDataFrame(sqlContext, faithful))""")
+
+      response match {
+        case Interpreter.ExecuteSuccess(map: JValue) =>
+          (map \ "text/plain").extract[String] should include (
+            """  eruptions waiting
+              |1     3.600      79
+              |2     1.800      54
+              |3     3.333      74
+              |4     2.283      62
+              |5     4.533      85
+              |6     2.883      55""".stripMargin)
+        case _ =>
+          throw new Exception("response is not a success")
+      }
+
+    }
+  }
+}

+ 2 - 2
apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/SparkRSessionSpec.scala

@@ -18,7 +18,7 @@
 
 package com.cloudera.hue.livy.repl
 
-import com.cloudera.hue.livy.repl.sparkr.SparkRSession
+import com.cloudera.hue.livy.repl.sparkr.SparkRInterpreter
 import org.json4s.Extraction
 import org.json4s.JsonAST.JValue
 
@@ -27,7 +27,7 @@ import _root_.scala.concurrent.duration.Duration
 
 class SparkRSessionSpec extends BaseSessionSpec {
 
-  override def createSession() = SparkRSession.create()
+  override def createInterpreter() = SparkRInterpreter()
 
   describe("A sparkr session") {
     it("should execute `1 + 2` == 3") {

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

@@ -18,16 +18,16 @@
 
 package com.cloudera.hue.livy.repl
 
-import com.cloudera.hue.livy.repl.scala.SparkSession
+import com.cloudera.hue.livy.repl.scala.SparkInterpreter
 import org.json4s.Extraction
-import org.json4s.JsonAST.JValue
+import org.json4s.JsonAST.{JArray, JValue}
 
 import _root_.scala.concurrent.Await
 import _root_.scala.concurrent.duration.Duration
 
 class SparkSessionSpec extends BaseSessionSpec {
 
-  override def createSession() = SparkSession.create()
+  override def createInterpreter() = SparkInterpreter()
 
   describe("A spark session") {
     it("should execute `1 + 2` == 3") {
@@ -118,7 +118,8 @@ class SparkSessionSpec extends BaseSessionSpec {
         "evalue" ->
           """<console>:8: error: not found: value x
             |              x
-            |              ^""".stripMargin
+            |              ^""".stripMargin,
+        "traceback" -> List()
       ))
 
       result should equal (expectedResult)
@@ -136,7 +137,7 @@ class SparkSessionSpec extends BaseSessionSpec {
       resultMap("execution_count").extract[Int] should equal (0)
       resultMap("ename").extract[String] should equal ("Error")
       resultMap("evalue").extract[String] should include ("java.lang.Exception")
-      resultMap.get("traceback") should equal (None)
+      resultMap("traceback").extract[List[_]] should equal (List())
     }
 
     it("should access the spark context") {
@@ -181,7 +182,7 @@ class SparkSessionSpec extends BaseSessionSpec {
 
       val expectedResult = Extraction.decompose(Map(
         "status" -> "ok",
-        "execution_count" -> 1,
+        "execution_count" -> 0,
         "data" -> Map(
           "application/vnd.livy.table.v1+json" -> Map(
             "headers" -> List(

+ 34 - 31
apps/spark/java/livy-repl/src/test/scala/com/cloudera/hue/livy/repl/WebAppSpec.scala

@@ -18,43 +18,40 @@
 
 package com.cloudera.hue.livy.repl
 
+import java.util.concurrent.TimeUnit
+
+import com.cloudera.hue.livy.repl
 import com.cloudera.hue.livy.sessions._
 import org.json4s.JsonAST.{JArray, JString}
 import org.json4s.JsonDSL._
 import org.json4s.jackson.JsonMethods._
-import org.json4s.{DefaultFormats, Extraction}
+import org.json4s.{JValue, DefaultFormats, Extraction}
 import org.scalatest.{BeforeAndAfter, FunSpecLike}
 import org.scalatra.test.scalatest.ScalatraSuite
 
-import _root_.scala.concurrent.Future
+import _root_.scala.concurrent.duration.Duration
+import _root_.scala.concurrent.{Await, Future}
 
 class WebAppSpec extends ScalatraSuite with FunSpecLike with BeforeAndAfter {
 
   implicit val formats = DefaultFormats
 
-  class MockSession extends Session {
-    var _state: State = Idle()
-    var _history = IndexedSeq[Statement]()
-
-    override def kind: Kind = Spark()
+  class MockInterpreter extends Interpreter {
+    override def kind: String = "mock"
 
-    override def state = _state
-
-    override def execute(code: String): Statement = {
-      val rep = render(Map("hi" -> "there"))
-      val statement = Statement(0, Future.successful(rep))
-      _history :+= statement
-      statement
-    }
+    override def start() = {}
 
-    override def close(): Unit = {
-      _state = Dead()
+    override def execute(code: String) = {
+      Thread.sleep(1000)
+      Interpreter.ExecuteSuccess(repl.TEXT_PLAIN -> "1")
     }
 
-    override def history: IndexedSeq[Statement] = _history
+    override def close() = {}
   }
 
-  val session = new MockSession
+  val interpreter = new MockInterpreter()
+  val session = new Session(new MockInterpreter())
+
   val servlet = new WebApp(session)
 
   addServlet(servlet, "/*")
@@ -68,7 +65,7 @@ class WebAppSpec extends ScalatraSuite with FunSpecLike with BeforeAndAfter {
         parsedBody \ "state" should equal (JString("idle"))
       }
 
-      session._state = Busy()
+      session.execute("")
 
       get("/") {
         status should equal (200)
@@ -90,24 +87,30 @@ class WebAppSpec extends ScalatraSuite with FunSpecLike with BeforeAndAfter {
     }
 
     it("GET /history with history should return something") {
-      val history = Extraction.decompose(Map(
-          "data" -> Map("text/plain" -> "1")
-      ))
-      session._history = IndexedSeq(Statement(0, Future.successful(history)))
+      Await.ready(session.execute("").result, Duration(10, TimeUnit.SECONDS))
 
       get("/history") {
         status should equal (200)
         header("Content-Type") should include("application/json")
         parse(body) should equal (
-          ("from", 0) ~
-            ("total", 1) ~
-            ("statements", JArray(List( ("id", 0) ~ ("result", history)))))
+          ("from" -> 0) ~
+          ("total" -> 1) ~
+          (
+            "statements" -> List[JValue](
+              ("id" -> 0) ~
+              ("result" ->
+                ("status" -> "ok") ~
+                ("execution_count" -> 0) ~
+                ("data" -> (repl.TEXT_PLAIN -> "1"))
+              )
+            )
+          )
+        )
       }
     }
-  }
 
-  after {
-    session._state = Idle()
-    session._history = IndexedSeq()
+    after {
+      session.clearHistory()
+    }
   }
 }