Эх сурвалжийг харах

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

Erick Tryzelaar 10 жил өмнө
parent
commit
6c8cd82d9e
24 өөрчлөгдсөн 1112 нэмэгдсэн , 883 устгасан
  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()
+    }
   }
 }