You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@toree.apache.org by bl...@apache.org on 2017/06/15 19:19:07 UTC

[2/2] incubator-toree git commit: TOREE-380: Allow interpreters to produce output by MIME type.

TOREE-380: Allow interpreters to produce output by MIME type.

This updates the interpreter API to return a Map of MIME type to content
instead of a String that is rendered as text. This allows interpreters
to show HTML as cell output.

The output of magic functions is now a similar MIME type to content
structure. This is cleaner and no longer requires hacky post-processing
before relaying a cell's output.

Scala now uses Jupyter's jvm-repr API for JVM objects. This allows
libraries to register functions that will produce display
representations of results by MIME type, that is hopefully compatible
with any JVM kernel.

This includes the following squashed commits:

5434046 TOREE-380: Allow interpreters to produce output by MIME type.
a78e9cb TOREE-380: Add support for Jupyter's jvm-repr API.
fc98890 TOREE-380: Fix tests.
9c9c9dc TOREE-380: Add JVMReprSpec.
b4c1510 TOREE-380: Fix Python integration tests.
7a50f20 TOREE-380: Fix AddJar integration test.
1eaf4a0 TOREE-380: Add copyright header to new files.
e7d4004 TOREE-380: Add comment to ScalaDisplayers.ensureLoaded.


Project: http://git-wip-us.apache.org/repos/asf/incubator-toree/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-toree/commit/5cd17270
Tree: http://git-wip-us.apache.org/repos/asf/incubator-toree/tree/5cd17270
Diff: http://git-wip-us.apache.org/repos/asf/incubator-toree/diff/5cd17270

Branch: refs/heads/master
Commit: 5cd1727089eff26f553f990ab9dfe49d2be14fe2
Parents: 5cfbc83
Author: Ryan Blue <bl...@apache.org>
Authored: Sun Feb 5 18:18:36 2017 -0800
Committer: Ryan Blue <bl...@apache.org>
Committed: Thu Jun 15 12:16:12 2017 -0700

----------------------------------------------------------------------
 build.sbt                                       |   3 +-
 .../toree/interpreter/InterpreterTypes.scala    |   2 +-
 .../interpreter/broker/BrokerTransformer.scala  |   2 +-
 .../org/apache/toree/interpreter/package.scala  |   2 +-
 .../apache/toree/kernel/api/KernelLike.scala    |   3 +-
 .../org/apache/toree/magic/MagicManager.scala   |  42 ++--
 .../org/apache/toree/magic/MagicOutput.scala    |  25 +++
 .../scala/org/apache/toree/magic/package.scala  |  18 +-
 .../broker/BrokerTransformerSpec.scala          |   6 +-
 .../apache/toree/magic/MagicManagerSpec.scala   |  12 +-
 .../boot/layer/HandlerInitialization.scala      |   5 +-
 .../org/apache/toree/kernel/api/Kernel.scala    |  16 +-
 .../protocol/v5/magic/PostProcessor.scala       |  56 -----
 .../protocol/v5/relay/ExecuteRequestRelay.scala |  12 +-
 .../InterpreterActorSpecForIntegration.scala    |  16 +-
 .../PostProcessorSpecForIntegration.scala       |  63 ------
 .../apache/toree/kernel/api/KernelSpec.scala    |  10 +-
 .../tasks/ExecuteRequestTaskActorSpec.scala     |   2 +-
 .../protocol/v5/magic/PostProcessorSpec.scala   | 140 -------------
 .../v5/relay/ExecuteRequestRelaySpec.scala      |  21 +-
 .../v5/stream/KernelInputStreamSpec.scala       |   3 +
 .../toree/magic/builtin/AddDepsSpec.scala       |   3 +-
 .../toree/magic/builtin/DataFrameSpec.scala     |  28 +--
 .../apache/toree/magic/builtin/PySpark.scala    |   7 +-
 scala-interpreter/build.sbt                     |   1 +
 .../scala/ScalaInterpreterSpecific.scala        |  90 +++-----
 .../interpreter/scala/ScalaDisplayers.scala     | 209 +++++++++++++++++++
 .../interpreter/scala/ScalaInterpreter.scala    | 183 ++++++++--------
 .../org/apache/toree/magic/builtin/Scala.scala  |   7 +-
 .../scala-2.11/scala/ScalaInterpreterSpec.scala |  52 +++--
 .../AddExternalJarMagicSpecForIntegration.scala |  12 +-
 .../interpreter/scala/JVMReprSpec.scala         |  97 +++++++++
 .../org/apache/toree/magic/builtin/SparkR.scala |   7 +-
 .../org/apache/toree/magic/builtin/Sql.scala    |   7 +-
 test_toree.py                                   |  16 +-
 35 files changed, 625 insertions(+), 553 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/build.sbt
----------------------------------------------------------------------
diff --git a/build.sbt b/build.sbt
index 43d1a09..c294421 100644
--- a/build.sbt
+++ b/build.sbt
@@ -74,7 +74,8 @@ testOptions in (ThisBuild, Test) += Tests.Argument("-oDF")
 // Build-wide dependencies
 resolvers in ThisBuild  ++= Seq(
   "Apache Snapshots" at "http://repository.apache.org/snapshots/",
-  "Typesafe repository" at "http://repo.typesafe.com/typesafe/releases/"
+  "Typesafe repository" at "http://repo.typesafe.com/typesafe/releases/",
+  "Jitpack" at "https://jitpack.io"
 )
 updateOptions in ThisBuild := updateOptions.value.withCachedResolution(true)
 libraryDependencies in ThisBuild ++= Seq(

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel-api/src/main/scala/org/apache/toree/interpreter/InterpreterTypes.scala
----------------------------------------------------------------------
diff --git a/kernel-api/src/main/scala/org/apache/toree/interpreter/InterpreterTypes.scala b/kernel-api/src/main/scala/org/apache/toree/interpreter/InterpreterTypes.scala
index e112145..971aa12 100644
--- a/kernel-api/src/main/scala/org/apache/toree/interpreter/InterpreterTypes.scala
+++ b/kernel-api/src/main/scala/org/apache/toree/interpreter/InterpreterTypes.scala
@@ -24,5 +24,5 @@ object InterpreterTypes {
   /**
    * Represents the output from an interpret execution.
    */
-  type ExecuteOutput = String
+  type ExecuteOutput = Map[String, String]
 }

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel-api/src/main/scala/org/apache/toree/interpreter/broker/BrokerTransformer.scala
----------------------------------------------------------------------
diff --git a/kernel-api/src/main/scala/org/apache/toree/interpreter/broker/BrokerTransformer.scala b/kernel-api/src/main/scala/org/apache/toree/interpreter/broker/BrokerTransformer.scala
index 2d7a553..0ad2344 100644
--- a/kernel-api/src/main/scala/org/apache/toree/interpreter/broker/BrokerTransformer.scala
+++ b/kernel-api/src/main/scala/org/apache/toree/interpreter/broker/BrokerTransformer.scala
@@ -43,7 +43,7 @@ class BrokerTransformer {
     import scala.concurrent.ExecutionContext.Implicits.global
 
     futureResult
-      .map(results => (Results.Success, Left(results)))
+      .map(results => (Results.Success, Left(Map("text/plain" -> results))))
       .recover({ case ex: BrokerException =>
         (Results.Error, Right(ExecuteError(
           name = ex.getClass.getName,

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel-api/src/main/scala/org/apache/toree/interpreter/package.scala
----------------------------------------------------------------------
diff --git a/kernel-api/src/main/scala/org/apache/toree/interpreter/package.scala b/kernel-api/src/main/scala/org/apache/toree/interpreter/package.scala
index 5aec706..f5c6437 100644
--- a/kernel-api/src/main/scala/org/apache/toree/interpreter/package.scala
+++ b/kernel-api/src/main/scala/org/apache/toree/interpreter/package.scala
@@ -23,6 +23,6 @@ package object interpreter {
   /**
    * Represents the output from an interpret execution.
    */
-  type ExecuteOutput = String
+  type ExecuteOutput = Map[String, String]
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel-api/src/main/scala/org/apache/toree/kernel/api/KernelLike.scala
----------------------------------------------------------------------
diff --git a/kernel-api/src/main/scala/org/apache/toree/kernel/api/KernelLike.scala b/kernel-api/src/main/scala/org/apache/toree/kernel/api/KernelLike.scala
index 4a19d4b..b2f96e9 100644
--- a/kernel-api/src/main/scala/org/apache/toree/kernel/api/KernelLike.scala
+++ b/kernel-api/src/main/scala/org/apache/toree/kernel/api/KernelLike.scala
@@ -23,6 +23,7 @@ import com.typesafe.config.Config
 import org.apache.spark.api.java.JavaSparkContext
 import org.apache.spark.{SparkContext, SparkConf}
 import org.apache.spark.sql.SparkSession
+import org.apache.toree.interpreter.ExecuteOutput
 
 /**
  * Interface for the kernel API. This does not include exposed variables.
@@ -37,7 +38,7 @@ trait KernelLike {
    * @return A tuple containing the result (true/false) and the output as a
    *         string
    */
-  def eval(code: Option[String]): (Boolean, String)
+  def eval(code: Option[String]): (Boolean, ExecuteOutput)
 
   /**
    * Returns a collection of methods that can be used to generate objects

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel-api/src/main/scala/org/apache/toree/magic/MagicManager.scala
----------------------------------------------------------------------
diff --git a/kernel-api/src/main/scala/org/apache/toree/magic/MagicManager.scala b/kernel-api/src/main/scala/org/apache/toree/magic/MagicManager.scala
index 828ec02..08d5880 100644
--- a/kernel-api/src/main/scala/org/apache/toree/magic/MagicManager.scala
+++ b/kernel-api/src/main/scala/org/apache/toree/magic/MagicManager.scala
@@ -75,7 +75,7 @@ class MagicManager(private val pluginManager: PluginManager) extends Dynamic {
   }
 
   @throws[MagicNotFoundException]
-  def applyDynamic(name: String)(args: Any*): Either[CellMagicOutput, LineMagicOutput] = {
+  def applyDynamic(name: String)(args: Any*): MagicOutput = {
     val arg = args.headOption.map(_.toString).getOrElse("")
 
     import org.apache.toree.plugins.Implicits._
@@ -90,25 +90,25 @@ class MagicManager(private val pluginManager: PluginManager) extends Dynamic {
     }
   }
 
-  private def handleMagicResult(name: String, result: Try[Any]) = result match {
-     case Success(magicOutput) => magicOutput match {
-        case null | _: BoxedUnit => Right(LineMagicOutput)
-        case cmo: Map[_, _]
-             if cmo.keys.forall(_.isInstanceOf[String]) &&
-                cmo.values.forall(_.isInstanceOf[String]) =>
-             Left(cmo.asInstanceOf[CellMagicOutput])
-        case unknown =>
-          val message =
-            s"""Magic $name did not return proper magic output
-               |type. Expected ${classOf[CellMagicOutput].getName} or
-               |${classOf[LineMagicOutput].getName}, but found type of
-               |${unknown.getClass.getName}.""".trim.stripMargin
-          logger.warn(message)
-          Left(CellMagicOutput("text/plain" -> message))
-      }
-      case Failure(t) =>
-        val message =  s"Magic $name failed to execute with error: \n${t.getMessage}"
-        logger.warn(message, t)
-        Left(CellMagicOutput("text/plain" -> message))
+  private def handleMagicResult(name: String, result: Try[Any]): MagicOutput = result match {
+    case Success(magicOutput) => magicOutput match {
+      case out: MagicOutput => out
+      case null | _: BoxedUnit => MagicOutput()
+      case cmo: Map[_, _]
+        if cmo.keys.forall(_.isInstanceOf[String]) &&
+          cmo.values.forall(_.isInstanceOf[String]) =>
+        MagicOutput(cmo.asInstanceOf[Map[String, String]].toSeq:_*)
+      case unknown =>
+        val message =
+          s"""Magic $name did not return proper magic output
+             |type. Expected ${classOf[MagicOutput].getName}, but found
+             |type of ${unknown.getClass.getName}.""".trim.stripMargin
+        logger.warn(message)
+        MagicOutput("text/plain" -> message)
+    }
+    case Failure(t) =>
+      val message =  s"Magic $name failed to execute with error: \n${t.getMessage}"
+      logger.warn(message, t)
+      MagicOutput("text/plain" -> message)
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel-api/src/main/scala/org/apache/toree/magic/MagicOutput.scala
----------------------------------------------------------------------
diff --git a/kernel-api/src/main/scala/org/apache/toree/magic/MagicOutput.scala b/kernel-api/src/main/scala/org/apache/toree/magic/MagicOutput.scala
new file mode 100644
index 0000000..5467447
--- /dev/null
+++ b/kernel-api/src/main/scala/org/apache/toree/magic/MagicOutput.scala
@@ -0,0 +1,25 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements.  See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF 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 org.apache.toree.magic
+
+/**
+ * Represents the output of a magic execution.
+ */
+case class MagicOutput(data: (String, String)*) {
+  lazy val asMap: Map[String, String] = Map(data:_*)
+}

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel-api/src/main/scala/org/apache/toree/magic/package.scala
----------------------------------------------------------------------
diff --git a/kernel-api/src/main/scala/org/apache/toree/magic/package.scala b/kernel-api/src/main/scala/org/apache/toree/magic/package.scala
index 7f04364..3e30370 100644
--- a/kernel-api/src/main/scala/org/apache/toree/magic/package.scala
+++ b/kernel-api/src/main/scala/org/apache/toree/magic/package.scala
@@ -18,16 +18,10 @@
 package org.apache.toree
 
 package object magic {
-  /**
-   * Represents the output of a magic execution.
-   */
-  // TODO: This is a duplicate of Data in kernel protocol, needs to be given
-  //       a type/val that can be translated into a specific protocol via
-  //       implicits - or some other transformation - to separate this from
-  //       the protocol type
-  type CellMagicOutput = Map[String, String]
-  val CellMagicOutput = Map
-  
-  type LineMagicOutput = Unit
-  val LineMagicOutput : LineMagicOutput = ()
+  type CellMagicOutput = MagicOutput
+  type LineMagicOutput = MagicOutput
+
+  // support old uses of CellMagicOutput and LineMagicOutput
+  val CellMagicOutput = MagicOutput
+  val LineMagicOutput = MagicOutput()
 }

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel-api/src/test/scala/org/apache/toree/interpreter/broker/BrokerTransformerSpec.scala
----------------------------------------------------------------------
diff --git a/kernel-api/src/test/scala/org/apache/toree/interpreter/broker/BrokerTransformerSpec.scala b/kernel-api/src/test/scala/org/apache/toree/interpreter/broker/BrokerTransformerSpec.scala
index d245f8d..0f1e37e 100644
--- a/kernel-api/src/test/scala/org/apache/toree/interpreter/broker/BrokerTransformerSpec.scala
+++ b/kernel-api/src/test/scala/org/apache/toree/interpreter/broker/BrokerTransformerSpec.scala
@@ -17,9 +17,9 @@
 package org.apache.toree.interpreter.broker
 
 import org.apache.toree.interpreter.{ExecuteError, Results}
-import org.scalatest.concurrent.{Eventually}
+import org.scalatest.concurrent.Eventually
 import scala.concurrent.Promise
-import org.scalatest.{OneInstancePerTest, Matchers, FunSpec}
+import org.scalatest.{FunSpec, Matchers, OneInstancePerTest}
 
 class BrokerTransformerSpec extends FunSpec with Matchers
   with OneInstancePerTest with Eventually
@@ -40,7 +40,7 @@ class BrokerTransformerSpec extends FunSpec with Matchers
 
         eventually {
           val result = transformedFuture.value.get.get
-          result should be((Results.Success, Left(successOutput)))
+          result should be((Results.Success, Left(Map("text/plain" -> successOutput))))
         }
       }
 

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel-api/src/test/scala/org/apache/toree/magic/MagicManagerSpec.scala
----------------------------------------------------------------------
diff --git a/kernel-api/src/test/scala/org/apache/toree/magic/MagicManagerSpec.scala b/kernel-api/src/test/scala/org/apache/toree/magic/MagicManagerSpec.scala
index 5874a86..8d445cf 100644
--- a/kernel-api/src/test/scala/org/apache/toree/magic/MagicManagerSpec.scala
+++ b/kernel-api/src/test/scala/org/apache/toree/magic/MagicManagerSpec.scala
@@ -185,7 +185,7 @@ class MagicManagerSpec
 
         val result = magicManager.applyDynamic("TEST")()
 
-        result.isLeft should be(true)
+        result.asMap.get("text/plain") should not be(empty)
       }
 
       it("should fire an event with the lowercase of the magic name") {
@@ -238,7 +238,7 @@ class MagicManagerSpec
 
       it("should return a Right[LineMagicOutput] if line magic execution is successful and returns null") {
         val pluginName = "TEST"
-        val expected = Right(LineMagicOutput)
+        val expected = LineMagicOutput
 
         doReturn(Some(SuccessPluginMethodResult(
           mock[PluginMethod],
@@ -253,7 +253,7 @@ class MagicManagerSpec
 
       it("should return a Right[LineMagicOutput] if line magic execution is successful and returns BoxedUnit") {
         val pluginName = "TEST"
-        val expected = Right(LineMagicOutput)
+        val expected = LineMagicOutput
 
         doReturn(Some(SuccessPluginMethodResult(
           mock[PluginMethod],
@@ -277,7 +277,7 @@ class MagicManagerSpec
         )
 
         val result = magicManager.applyDynamic(pluginName)(Nil: _*)
-        result.left.get should be(cellMagicOutput)
+        result should be(cellMagicOutput)
       }
 
       it("should return a Left[CellMagicOutput] if is a magic but not a line or cell") {
@@ -291,7 +291,7 @@ class MagicManagerSpec
         )
 
         val result = magicManager.applyDynamic(pluginName)(Nil: _*)
-        result.left.get("text/plain") should not be (empty)
+        result.asMap.get("text/plain") should not be (empty)
 
       }
 
@@ -306,7 +306,7 @@ class MagicManagerSpec
         )
 
         val result = magicManager.applyDynamic(pluginName)(Nil: _*)
-        result.left.get("text/plain") should not be (empty)
+        result.asMap.get("text/plain") should not be (empty)
       }
 
       it("should throw a MagicNotFoundException when a magic cannot be found") {

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel/src/main/scala/org/apache/toree/boot/layer/HandlerInitialization.scala
----------------------------------------------------------------------
diff --git a/kernel/src/main/scala/org/apache/toree/boot/layer/HandlerInitialization.scala b/kernel/src/main/scala/org/apache/toree/boot/layer/HandlerInitialization.scala
index 3da8469..1e7f56c 100644
--- a/kernel/src/main/scala/org/apache/toree/boot/layer/HandlerInitialization.scala
+++ b/kernel/src/main/scala/org/apache/toree/boot/layer/HandlerInitialization.scala
@@ -27,7 +27,7 @@ import org.apache.toree.kernel.protocol.v5.handler._
 import org.apache.toree.kernel.protocol.v5.interpreter.InterpreterActor
 import org.apache.toree.kernel.protocol.v5.interpreter.tasks.InterpreterTaskFactory
 import org.apache.toree.kernel.protocol.v5.kernel.ActorLoader
-import org.apache.toree.kernel.protocol.v5.magic.{MagicParser, PostProcessor}
+import org.apache.toree.kernel.protocol.v5.magic.MagicParser
 import org.apache.toree.kernel.protocol.v5.relay.ExecuteRequestRelay
 import org.apache.toree.kernel.protocol.v5.{MessageType, SocketType, SystemActorType, LanguageInfo}
 import org.apache.toree.magic.MagicManager
@@ -105,11 +105,10 @@ trait StandardHandlerInitialization extends HandlerInitialization {
     )
 
     logger.debug("Creating execute request relay actor")
-    val postProcessor = new PostProcessor(interpreter)
     val magicParser = new MagicParser(magicManager)
     val executeRequestRelayActor = actorSystem.actorOf(
       Props(classOf[ExecuteRequestRelay],
-        actorLoader, pluginManager, magicParser, postProcessor
+        actorLoader, pluginManager, magicParser
       ),
       name = SystemActorType.ExecuteRequestRelay.toString
     )

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel/src/main/scala/org/apache/toree/kernel/api/Kernel.scala
----------------------------------------------------------------------
diff --git a/kernel/src/main/scala/org/apache/toree/kernel/api/Kernel.scala b/kernel/src/main/scala/org/apache/toree/kernel/api/Kernel.scala
index 285db1f..65a8b98 100644
--- a/kernel/src/main/scala/org/apache/toree/kernel/api/Kernel.scala
+++ b/kernel/src/main/scala/org/apache/toree/kernel/api/Kernel.scala
@@ -134,19 +134,19 @@ class Kernel (
    */
   private def handleInterpreterOutput(
     output: (Result, Either[ExecuteOutput, ExecuteFailure])
-  ): (Boolean, String) = {
+  ): (Boolean, ExecuteOutput) = {
     val (success, result) = output
     success match {
       case Results.Success =>
-        (true, result.left.getOrElse("").asInstanceOf[String])
+        (true, result.left.get)
       case Results.Error =>
-        (false, result.right.getOrElse("").toString)
+        (false, Map("text/plain" -> result.right.getOrElse("").toString))
       case Results.Aborted =>
-        (false, "Aborted!")
+        (false, Map("text/plain" -> "Aborted!"))
       case Results.Incomplete =>
         // If we get an incomplete it's most likely a syntax error, so
         // let the user know.
-        (false, "Syntax Error!")
+        (false, Map("text/plain" -> "Syntax Error!"))
     }
   }
 
@@ -161,16 +161,16 @@ class Kernel (
    * @return A tuple containing the result (true/false) and the output as a
    *         string
    */
-  def eval(code: Option[String]): (Boolean, String) = {
+  def eval(code: Option[String]): (Boolean, ExecuteOutput) = {
     code.map(c => {
       magicParser.parse(c) match {
         case Left(parsedCode) =>
           val output = interpreter.interpret(parsedCode)
           handleInterpreterOutput(output)
         case Right(errMsg) =>
-          (false, errMsg)
+          (false, Map("text/plain" -> errMsg))
       }
-    }).getOrElse((false, "Error!"))
+    }).getOrElse((false, Map("text/plain" -> "Error!")))
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel/src/main/scala/org/apache/toree/kernel/protocol/v5/magic/PostProcessor.scala
----------------------------------------------------------------------
diff --git a/kernel/src/main/scala/org/apache/toree/kernel/protocol/v5/magic/PostProcessor.scala b/kernel/src/main/scala/org/apache/toree/kernel/protocol/v5/magic/PostProcessor.scala
deleted file mode 100644
index 569b0dc..0000000
--- a/kernel/src/main/scala/org/apache/toree/kernel/protocol/v5/magic/PostProcessor.scala
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements.  See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF 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 org.apache.toree.kernel.protocol.v5.magic
-
-import org.apache.toree.interpreter.{ExecuteOutput, Interpreter}
-import org.apache.toree.kernel.protocol.v5.{Data, MIMEType}
-import org.apache.toree.magic.{CellMagicOutput, LineMagicOutput}
-import org.apache.toree.utils.LogLike
-
-class PostProcessor(interpreter: Interpreter) extends LogLike {
-  val defaultErr = "Something went wrong in postprocessor!"
-
-  def process(codeOutput: ExecuteOutput): Data = {
-    interpreter.lastExecutionVariableName.flatMap(interpreter.read) match {
-      case Some(l: Left[_, _]) => matchCellMagic(codeOutput, l)
-      case Some(r: Right[_, _]) => matchLineMagic(codeOutput, r)
-      case _ => Data(MIMEType.PlainText -> codeOutput)
-    }
-  }
-
-  protected[magic] def matchCellMagic(code: String, l: Left[_,_]) =
-    l.left.getOrElse(None) match {
-      // NOTE: Hack to get around erasure match issue in Scala 2.11
-      case cmo: Map[_, _]
-        if cmo.keys.forall(_.isInstanceOf[String]) &&
-           cmo.values.forall(_.isInstanceOf[String]) =>
-        cmo.asInstanceOf[CellMagicOutput]
-      case _ => Data(MIMEType.PlainText -> code)
-    }
-
-  protected[magic] def matchLineMagic(code: String, r: Right[_,_]) =
-    r.right.getOrElse(None) match {
-      case lmo: LineMagicOutput => processLineMagic(code)
-      case _ => Data(MIMEType.PlainText -> code)
-    }
-
-  protected[magic] def processLineMagic(code: String): Data = {
-    val parts = code.split("\n")
-    Data(MIMEType.PlainText -> parts.take(parts.size - 1).mkString("\n"))
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel/src/main/scala/org/apache/toree/kernel/protocol/v5/relay/ExecuteRequestRelay.scala
----------------------------------------------------------------------
diff --git a/kernel/src/main/scala/org/apache/toree/kernel/protocol/v5/relay/ExecuteRequestRelay.scala b/kernel/src/main/scala/org/apache/toree/kernel/protocol/v5/relay/ExecuteRequestRelay.scala
index 9105841..4e87cf4 100644
--- a/kernel/src/main/scala/org/apache/toree/kernel/protocol/v5/relay/ExecuteRequestRelay.scala
+++ b/kernel/src/main/scala/org/apache/toree/kernel/protocol/v5/relay/ExecuteRequestRelay.scala
@@ -25,7 +25,7 @@ import org.apache.toree.interpreter.{ExecuteAborted, ExecuteError, ExecuteFailur
 import org.apache.toree.kernel.protocol.v5._
 import org.apache.toree.kernel.protocol.v5.content._
 import org.apache.toree.kernel.protocol.v5.kernel.ActorLoader
-import org.apache.toree.kernel.protocol.v5.magic.{PostProcessor, MagicParser}
+import org.apache.toree.kernel.protocol.v5.magic.MagicParser
 import org.apache.toree.plugins.PluginManager
 import org.apache.toree.utils.LogLike
 import scala.concurrent.Future
@@ -35,8 +35,7 @@ import org.apache.toree.plugins.NewOutputStream
 case class ExecuteRequestRelay(
   actorLoader: ActorLoader,
   pluginManager: PluginManager,
-  magicParser: MagicParser,
-  postProcessor: PostProcessor
+  magicParser: MagicParser
 )
   extends Actor with LogLike
 {
@@ -46,7 +45,7 @@ case class ExecuteRequestRelay(
   /**
    * Takes an ExecuteFailure and (ExecuteReply, ExecuteResult) with contents
    * dictated by the type of failure (either an error or an abort).
- *
+   *
    * @param failure the failure
    * @return (ExecuteReply, ExecuteResult)
    */
@@ -68,7 +67,7 @@ case class ExecuteRequestRelay(
 
   /**
    * Packages the response into an ExecuteReply,ExecuteResult tuple.
- *
+   *
    * @param future The future containing either the output or failure
    * @return The tuple representing the proper response
    */
@@ -76,8 +75,7 @@ case class ExecuteRequestRelay(
     future: Future[Either[ExecuteOutput, ExecuteFailure]]
   ): Future[(ExecuteReply, ExecuteResult)] = future.map { value =>
     if (value.isLeft) {
-      val output = value.left.get
-      val data = postProcessor.process(output)
+      val data = value.left.get
       (
         ExecuteReplyOk(1, Some(Payloads()), Some(UserExpressions())),
         ExecuteResult(1, data, Metadata())

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel/src/test/scala/integration/InterpreterActorSpecForIntegration.scala
----------------------------------------------------------------------
diff --git a/kernel/src/test/scala/integration/InterpreterActorSpecForIntegration.scala b/kernel/src/test/scala/integration/InterpreterActorSpecForIntegration.scala
index cc151a8..8b53abb 100644
--- a/kernel/src/test/scala/integration/InterpreterActorSpecForIntegration.scala
+++ b/kernel/src/test/scala/integration/InterpreterActorSpecForIntegration.scala
@@ -21,19 +21,16 @@ import java.io.{ByteArrayOutputStream, OutputStream}
 
 import akka.actor.{ActorSystem, Props}
 import akka.testkit.{ImplicitSender, TestKit}
-import com.typesafe.config.ConfigFactory
-import org.apache.spark.{SparkConf, SparkContext}
 import org.apache.toree.Main
 import org.apache.toree.interpreter._
-import org.apache.toree.kernel.api.KernelLike
+import org.apache.toree.kernel.api.{DisplayMethodsLike, KernelLike}
 import org.apache.toree.kernel.interpreter.scala.ScalaInterpreter
 import org.apache.toree.kernel.protocol.v5._
 import org.apache.toree.kernel.protocol.v5.content._
 import org.apache.toree.kernel.protocol.v5.interpreter.InterpreterActor
 import org.apache.toree.kernel.protocol.v5.interpreter.tasks.InterpreterTaskFactory
-import org.apache.toree.utils.MultiOutputStream
 import com.typesafe.config.ConfigFactory
-import org.apache.spark.{SparkConf, SparkContext}
+import org.mockito.Mockito._
 import org.scalatest.mock.MockitoSugar
 import org.scalatest.{BeforeAndAfter, FunSpecLike, Matchers}
 import test.utils.UncaughtExceptionSuppression
@@ -60,15 +57,14 @@ class InterpreterActorSpecForIntegration extends TestKit(
     override protected def bindKernelVariable(kernel: KernelLike): Unit = { }
   }
 
-  private val conf = new SparkConf()
-    .setMaster("local[*]")
-    .setAppName("Test Kernel")
-
 
   before {
     output.reset()
     // interpreter.start()
-    interpreter.init(mock[KernelLike])
+    val mockDisplayMethods = mock[DisplayMethodsLike]
+    val mockKernel = mock[KernelLike]
+    doReturn(mockDisplayMethods).when(mockKernel).display
+    interpreter.init(mockKernel)
 
     interpreter.doQuietly({
       //context = new SparkContext(conf) with NoSparkLogging

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel/src/test/scala/integration/PostProcessorSpecForIntegration.scala
----------------------------------------------------------------------
diff --git a/kernel/src/test/scala/integration/PostProcessorSpecForIntegration.scala b/kernel/src/test/scala/integration/PostProcessorSpecForIntegration.scala
deleted file mode 100644
index 5bf70ae..0000000
--- a/kernel/src/test/scala/integration/PostProcessorSpecForIntegration.scala
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements.  See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF 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 integration
-
-import java.io.OutputStream
-
-import org.apache.toree.kernel.api.Kernel
-import org.apache.toree.kernel.interpreter.scala.ScalaInterpreter
-import org.apache.toree.kernel.protocol.v5.magic.PostProcessor
-import org.apache.toree.utils.{MultiOutputStream}
-import org.scalatest.mock.MockitoSugar
-import org.scalatest.{BeforeAndAfter, FunSpec, Matchers}
-
-class PostProcessorSpecForIntegration extends FunSpec with Matchers
-  with BeforeAndAfter with MockitoSugar
-{
-  private var scalaInterpreter: ScalaInterpreter = _
-  private var postProcessor: PostProcessor = _
-
-  before {
-    // TODO: Move instantiation and start of interpreter to a beforeAll
-    //       for performance improvements
-    scalaInterpreter = new ScalaInterpreter
-
-    scalaInterpreter.init(mock[Kernel])
-
-    postProcessor = new PostProcessor(scalaInterpreter)
-  }
-
-  describe("PostProcessor") {
-    describe("#process") {
-      describe("https://github.com/ibm-et/spark-kernel/issues/137") {
-        it(Seq(
-          "should not return a previous execution's result for a",
-          "new execution with no result").mkString(" ")) {
-          val result = scalaInterpreter.interpret("1+1")
-          val postResult = postProcessor.process(result._2.left.get)
-
-          // Imports in Scala do not create a new variable based on execution
-          val result2 = scalaInterpreter.interpret("import java.lang._")
-          val postResult2 = postProcessor.process(result2._2.left.get)
-
-          postResult should not be (postResult2)
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel/src/test/scala/org/apache/toree/kernel/api/KernelSpec.scala
----------------------------------------------------------------------
diff --git a/kernel/src/test/scala/org/apache/toree/kernel/api/KernelSpec.scala b/kernel/src/test/scala/org/apache/toree/kernel/api/KernelSpec.scala
index 799f33a..43ad8a8 100644
--- a/kernel/src/test/scala/org/apache/toree/kernel/api/KernelSpec.scala
+++ b/kernel/src/test/scala/org/apache/toree/kernel/api/KernelSpec.scala
@@ -67,7 +67,7 @@ class KernelSpec extends FunSpec with Matchers with MockitoSugar
     when(mockInterpreter.interpret(BadCode.get))
       .thenReturn((Results.Incomplete, null))
     when(mockInterpreter.interpret(GoodCode.get))
-      .thenReturn((Results.Success, Left(new ExecuteOutput("ok"))))
+      .thenReturn((Results.Success, Left(Map("text/plain" -> "ok"))))
     when(mockInterpreter.interpret(ErrorCode.get))
       .thenReturn((Results.Error, Right(ExecuteError("error","bad", List("1")))))
 
@@ -91,19 +91,19 @@ class KernelSpec extends FunSpec with Matchers with MockitoSugar
   describe("Kernel") {
     describe("#eval") {
       it("should return syntax error") {
-        kernel eval BadCode should be((false, "Syntax Error!"))
+        kernel eval BadCode should be((false, Map("text/plain" -> "Syntax Error!")))
       }
 
       it("should return ok") {
-        kernel eval GoodCode should be((true, "ok"))
+        kernel eval GoodCode should be((true, Map("text/plain" -> "ok")))
       }
 
       it("should return error") {
-        kernel eval ErrorCode should be((false, ErrorMsg))
+        kernel eval ErrorCode should be((false, Map("text/plain" -> ErrorMsg)))
       }
 
       it("should return error on None") {
-        kernel eval None should be ((false, "Error!"))
+        kernel eval None should be ((false, Map("text/plain" -> "Error!")))
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/interpreter/tasks/ExecuteRequestTaskActorSpec.scala
----------------------------------------------------------------------
diff --git a/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/interpreter/tasks/ExecuteRequestTaskActorSpec.scala b/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/interpreter/tasks/ExecuteRequestTaskActorSpec.scala
index 935079c..4f1a032 100644
--- a/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/interpreter/tasks/ExecuteRequestTaskActorSpec.scala
+++ b/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/interpreter/tasks/ExecuteRequestTaskActorSpec.scala
@@ -50,7 +50,7 @@ class ExecuteRequestTaskActorSpec extends TestKit(
     describe("#receive") {
       it("should return an ExecuteReplyOk if the interpreter returns success") {
         val mockInterpreter = mock[Interpreter]
-        doReturn((Results.Success, Left(new ExecuteOutput))).when(mockInterpreter)
+        doReturn((Results.Success, Left(Map()))).when(mockInterpreter)
           .interpret(anyString(), anyBoolean(), anyObject())
 
         val executeRequestTask =

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/magic/PostProcessorSpec.scala
----------------------------------------------------------------------
diff --git a/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/magic/PostProcessorSpec.scala b/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/magic/PostProcessorSpec.scala
deleted file mode 100644
index d5181b0..0000000
--- a/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/magic/PostProcessorSpec.scala
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements.  See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF 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 org.apache.toree.kernel.protocol.v5.magic
-
-import org.apache.toree.interpreter.Interpreter
-import org.apache.toree.kernel.protocol.v5._
-import org.apache.toree.magic.{CellMagicOutput, LineMagicOutput}
-import org.mockito.Matchers._
-import org.mockito.Mockito._
-import org.scalatest.mock.MockitoSugar
-import org.scalatest.{FunSpec, Matchers}
-
-class PostProcessorSpec extends FunSpec with Matchers with MockitoSugar{
-  describe("#matchCellMagic") {
-    it("should return the cell magic output when the Left contains a " +
-       "CellMagicOutput") {
-      val processor = new PostProcessor(mock[Interpreter])
-      val codeOutput = "some output"
-      val cmo = CellMagicOutput()
-      val left = Left(cmo)
-      processor.matchCellMagic(codeOutput, left) should be(cmo)
-    }
-
-    it("should package the original code when the Left does not contain a " +
-       "CellMagicOutput") {
-      val processor = new PostProcessor(mock[Interpreter])
-      val codeOutput = "some output"
-      val left = Left("")
-      val data = Data(MIMEType.PlainText -> codeOutput)
-      processor.matchCellMagic(codeOutput, left) should be(data)
-    }
-  }
-
-  describe("#matchLineMagic") {
-    it("should process the code output when the Right contains a " +
-       "LineMagicOutput") {
-      val processor = spy(new PostProcessor(mock[Interpreter]))
-      val codeOutput = "some output"
-      val lmo = LineMagicOutput
-      val right = Right(lmo)
-      processor.matchLineMagic(codeOutput, right)
-      verify(processor).processLineMagic(codeOutput)
-    }
-
-    it("should package the original code when the Right does not contain a " +
-       "LineMagicOutput") {
-      val processor = new PostProcessor(mock[Interpreter])
-      val codeOutput = "some output"
-      val right = Right("")
-      val data = Data(MIMEType.PlainText -> codeOutput)
-      processor.matchLineMagic(codeOutput, right) should be(data)
-    }
-  }
-
-  describe("#processLineMagic") {
-    it("should remove the result of the magic invocation if it is the last " +
-       "line") {
-      val processor = new PostProcessor(mock[Interpreter])
-      val x = "hello world"
-      val codeOutput = s"$x\nsome other output"
-      val data = Data(MIMEType.PlainText -> x)
-      processor.processLineMagic(codeOutput) should be(data)
-    }
-  }
-
-  describe("#process") {
-    it("should call matchCellMagic when the last variable is a Left") {
-      val intp = mock[Interpreter]
-      val left = Left("")
-      // Need to mock lastExecutionVariableName as it is being chained with
-      // the read method
-      doReturn(Some("")).when(intp).lastExecutionVariableName
-      doReturn(Some(left)).when(intp).read(anyString())
-
-      val processor = spy(new PostProcessor(intp))
-      val codeOutput = "hello"
-      processor.process(codeOutput)
-      verify(processor).matchCellMagic(codeOutput, left)
-    }
-
-    it("should call matchLineMagic when the last variable is a Right") {
-      val intp = mock[Interpreter]
-      val right = Right("")
-      // Need to mock lastExecutionVariableName as it is being chained with
-      // the read method
-      doReturn(Some("")).when(intp).lastExecutionVariableName
-      doReturn(Some(right)).when(intp).read(anyString())
-
-      val processor = spy(new PostProcessor(intp))
-      val codeOutput = "hello"
-      processor.process(codeOutput)
-      verify(processor).matchLineMagic(codeOutput, right)
-    }
-
-    it("should package the original code output when the Left is not a " +
-      "Left[CellMagicOutput, Nothing]") {
-      val intp = mock[Interpreter]
-      val left = Left("")
-      // Need to mock lastExecutionVariableName as it is being chained with
-      // the read method
-      doReturn(Some("")).when(intp).lastExecutionVariableName
-      doReturn(Some(left)).when(intp).read(anyString())
-
-      val processor = spy(new PostProcessor(intp))
-      val codeOutput = "hello"
-      val data = Data(MIMEType.PlainText -> codeOutput)
-      processor.process(codeOutput) should be(data)
-    }
-
-    it("should package the original code output when the Right is not a " +
-       "Right[LineMagicOutput, Nothing]") {
-      val intp = mock[Interpreter]
-      val right = Right("")
-      // Need to mock lastExecutionVariableName as it is being chained with
-      // the read method
-      doReturn(Some("")).when(intp).lastExecutionVariableName
-      doReturn(Some(right)).when(intp).read(anyString())
-
-      val processor = spy(new PostProcessor(intp))
-      val codeOutput = "hello"
-      val data = Data(MIMEType.PlainText -> codeOutput)
-      processor.process(codeOutput) should be(data)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/relay/ExecuteRequestRelaySpec.scala
----------------------------------------------------------------------
diff --git a/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/relay/ExecuteRequestRelaySpec.scala b/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/relay/ExecuteRequestRelaySpec.scala
index 5a84914..95dcb3a 100644
--- a/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/relay/ExecuteRequestRelaySpec.scala
+++ b/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/relay/ExecuteRequestRelaySpec.scala
@@ -18,13 +18,14 @@
 package org.apache.toree.kernel.protocol.v5.relay
 
 import java.io.OutputStream
+import java.util.concurrent.TimeUnit
 import akka.actor._
 import akka.testkit.{ImplicitSender, TestKit, TestProbe}
 import org.apache.toree.interpreter.{ExecuteAborted, ExecuteError}
 import org.apache.toree.kernel.protocol.v5._
 import org.apache.toree.kernel.protocol.v5.content._
 import org.apache.toree.kernel.protocol.v5.kernel.ActorLoader
-import org.apache.toree.kernel.protocol.v5.magic.{MagicParser, PostProcessor}
+import org.apache.toree.kernel.protocol.v5.magic.MagicParser
 import com.typesafe.config.ConfigFactory
 import org.apache.toree.plugins.PluginManager
 import org.apache.toree.plugins.dependencies.DependencyManager
@@ -32,6 +33,7 @@ import org.mockito.Mockito._
 import org.scalatest.mock.MockitoSugar
 import org.scalatest.{BeforeAndAfter, FunSpecLike, Matchers}
 import test.utils.MaxAkkaTestTimeout
+import scala.concurrent.duration.Duration
 
 object ExecuteRequestRelaySpec {
   val config = """
@@ -67,7 +69,6 @@ class ExecuteRequestRelaySpec extends TestKit(
         val executeRequest =
           ExecuteRequest("%myMagic", false, true, UserExpressions(), true)
 
-        val mockPostProcessor = mock[PostProcessor]
         val mockPluginManager = mock[PluginManager]
         val mockDependencyManager = mock[DependencyManager]
         doReturn(mockDependencyManager).when(mockPluginManager).dependencyManager
@@ -78,7 +79,7 @@ class ExecuteRequestRelaySpec extends TestKit(
 
         val executeRequestRelay = system.actorOf(Props(
           classOf[ExecuteRequestRelay], mockActorLoader,
-          mockPluginManager, mockMagicParser, mockPostProcessor
+          mockPluginManager, mockMagicParser
         ))
 
         // Send the message to the ExecuteRequestRelay
@@ -88,7 +89,7 @@ class ExecuteRequestRelaySpec extends TestKit(
         // Expected does not actually match real return of magic, which
         // is a tuple of ExecuteReply and ExecuteResult
         val expected = new ExecuteAborted()
-        interpreterActorProbe.expectMsgClass(
+        interpreterActorProbe.expectMsgClass(max = Duration(5, TimeUnit.SECONDS),
           classOf[(ExecuteRequest, KernelMessage, OutputStream)]
         )
 
@@ -105,7 +106,6 @@ class ExecuteRequestRelaySpec extends TestKit(
         val executeRequest =
           ExecuteRequest("%myMagic", false, true, UserExpressions(), true)
 
-        val mockPostProcessor = mock[PostProcessor]
         val mockPluginManager = mock[PluginManager]
         val mockDependencyManager = mock[DependencyManager]
         doReturn(mockDependencyManager).when(mockPluginManager).dependencyManager
@@ -116,7 +116,7 @@ class ExecuteRequestRelaySpec extends TestKit(
 
         val executeRequestRelay = system.actorOf(Props(
           classOf[ExecuteRequestRelay], mockActorLoader,
-          mockPluginManager, mockMagicParser, mockPostProcessor
+          mockPluginManager, mockMagicParser
         ))
 
         // Send the message to the ExecuteRequestRelay
@@ -145,13 +145,10 @@ class ExecuteRequestRelaySpec extends TestKit(
 
       it("should return an (ExecuteReply, ExecuteResult) on interpreter " +
          "success") {
-        val expected = "SOME OTHER VALUE"
+        val expected = Map(MIMEType.PlainText -> "SOME OTHER VALUE")
         val executeRequest =
           ExecuteRequest("notAMagic", false, true, UserExpressions(), true)
 
-        val mockPostProcessor = mock[PostProcessor]
-        doReturn(Data(MIMEType.PlainText -> expected))
-          .when(mockPostProcessor).process(expected)
         val mockPluginManager = mock[PluginManager]
         val mockDependencyManager = mock[DependencyManager]
         doReturn(mockDependencyManager).when(mockPluginManager).dependencyManager
@@ -162,7 +159,7 @@ class ExecuteRequestRelaySpec extends TestKit(
 
         val executeRequestRelay = system.actorOf(Props(
           classOf[ExecuteRequestRelay], mockActorLoader,
-          mockPluginManager, mockMagicParser, mockPostProcessor
+          mockPluginManager, mockMagicParser
         ))
 
         // Send the message to the ExecuteRequestRelay
@@ -183,7 +180,7 @@ class ExecuteRequestRelaySpec extends TestKit(
           MaxAkkaTestTimeout,
           (
             ExecuteReplyOk(1, Some(Payloads()), Some(UserExpressions())),
-            ExecuteResult(1, Data(MIMEType.PlainText -> expected), Metadata())
+            ExecuteResult(1, expected, Metadata())
           )
         )
       }

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/stream/KernelInputStreamSpec.scala
----------------------------------------------------------------------
diff --git a/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/stream/KernelInputStreamSpec.scala b/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/stream/KernelInputStreamSpec.scala
index 5fdfa50..739828a 100644
--- a/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/stream/KernelInputStreamSpec.scala
+++ b/kernel/src/test/scala/org/apache/toree/kernel/protocol/v5/stream/KernelInputStreamSpec.scala
@@ -65,6 +65,9 @@ class KernelInputStreamSpec
     // set of data
     doReturn(system.actorSelection(fakeInputOutputHandlerActor.path.toString))
       .when(mockActorLoader).load(MessageType.Incoming.InputReply)
+    // Allow time for the actors to start. This avoids read() hanging forever
+    // when running tests in gradle.
+    Thread.sleep(100)
   }
 
   describe("KernelInputStream") {

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel/src/test/scala/org/apache/toree/magic/builtin/AddDepsSpec.scala
----------------------------------------------------------------------
diff --git a/kernel/src/test/scala/org/apache/toree/magic/builtin/AddDepsSpec.scala b/kernel/src/test/scala/org/apache/toree/magic/builtin/AddDepsSpec.scala
index 421d12b..80e2d4b 100644
--- a/kernel/src/test/scala/org/apache/toree/magic/builtin/AddDepsSpec.scala
+++ b/kernel/src/test/scala/org/apache/toree/magic/builtin/AddDepsSpec.scala
@@ -57,7 +57,7 @@ class AddDepsSpec extends FunSpec with Matchers with MockitoSugar
         }
 
         val expected = LineMagicOutput
-        val actual = addDepsMagic.execute("notvalid")
+        addDepsMagic.execute("notvalid")
 
         printHelpWasRun should be (true)
         verify(mockKernel, times(0)).addJars(any())
@@ -65,7 +65,6 @@ class AddDepsSpec extends FunSpec with Matchers with MockitoSugar
           anyString(), anyString(), anyString(), anyBoolean(), anyBoolean(),
           anyBoolean(), any[Seq[(URL, Option[Credentials])]], anyBoolean(), anyBoolean()
         )
-        actual should be (expected)
       }
 
       it("should set the retrievals transitive to true if provided") {

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/kernel/src/test/scala/org/apache/toree/magic/builtin/DataFrameSpec.scala
----------------------------------------------------------------------
diff --git a/kernel/src/test/scala/org/apache/toree/magic/builtin/DataFrameSpec.scala b/kernel/src/test/scala/org/apache/toree/magic/builtin/DataFrameSpec.scala
index d1142b9..38fde38 100644
--- a/kernel/src/test/scala/org/apache/toree/magic/builtin/DataFrameSpec.scala
+++ b/kernel/src/test/scala/org/apache/toree/magic/builtin/DataFrameSpec.scala
@@ -47,7 +47,7 @@ class DataFrameSpec extends FunSpec with Matchers with MockitoSugar with BeforeA
         val message: Either[ExecuteOutput, ExecuteFailure] = Right(mock[ExecuteAborted])
         val code = "code"
         when(interpreter.interpret(code)).thenReturn((Results.Aborted, message))
-        val output = magic.execute(code)
+        val output = magic.execute(code).asMap
         output.contains(MIMEType.PlainText) should be(true)
         output(MIMEType.PlainText) should be(DataFrameResponses.ErrorMessage(
           "html",
@@ -63,7 +63,7 @@ class DataFrameSpec extends FunSpec with Matchers with MockitoSugar with BeforeA
         val message: Either[ExecuteOutput, ExecuteFailure] = Right(mockExecuteError)
         val code = "code"
         when(interpreter.interpret(code)).thenReturn((Results.Error, message))
-        val output = magic.execute(code)
+        val output = magic.execute(code).asMap
         output.contains(MIMEType.PlainText) should be(true)
         output(MIMEType.PlainText) should be(DataFrameResponses.ErrorMessage(
           "html",
@@ -79,7 +79,7 @@ class DataFrameSpec extends FunSpec with Matchers with MockitoSugar with BeforeA
         val message: Either[ExecuteOutput, ExecuteFailure] = Right(mockExecuteError )
         val code = "code"
         when(interpreter.interpret(code)).thenReturn((Results.Error, message))
-        val output = magic.execute(code)
+        val output = magic.execute(code).asMap
         output.contains(MIMEType.PlainText) should be(true)
         output(MIMEType.PlainText) should be(DataFrameResponses.ErrorMessage(
           "html",
@@ -90,7 +90,7 @@ class DataFrameSpec extends FunSpec with Matchers with MockitoSugar with BeforeA
       it("should return a plain text message with help when there are no args"){
         val (magic, _, _) = createMocks
         val code = ""
-        val output = magic.execute(code)
+        val output = magic.execute(code).asMap
         output.contains(MIMEType.PlainText) should be(true)
         output(MIMEType.PlainText).contains(DataFrameResponses.Usage) should be(true)
       }
@@ -98,7 +98,7 @@ class DataFrameSpec extends FunSpec with Matchers with MockitoSugar with BeforeA
       it("should return a json message when json is the selected output"){
         val (magic, interpreter, converter) = createMocks
         val outputText = "test output"
-        val message: Either[ExecuteOutput, ExecuteFailure] = Left(outputText)
+        val message: Either[ExecuteOutput, ExecuteFailure] = Left(Map("text/plain" -> outputText))
         val mockDataFrame = mock[org.apache.spark.sql.DataFrame]
         val variableName = "variable"
         val executeCode =s"""--output=json
@@ -110,7 +110,7 @@ class DataFrameSpec extends FunSpec with Matchers with MockitoSugar with BeforeA
         doReturn(Success(outputText)).when(converter).convert(
           mockDataFrame,"json", 10
         )
-        val output = magic.execute(executeCode)
+        val output = magic.execute(executeCode).asMap
         output.contains(MIMEType.ApplicationJson) should be(true)
         output(MIMEType.ApplicationJson).contains(outputText) should be(true)
       }
@@ -118,7 +118,7 @@ class DataFrameSpec extends FunSpec with Matchers with MockitoSugar with BeforeA
       it("should return an html message when html is the selected output"){
         val (magic, interpreter, converter) = createMocks
         val outputText = "test output"
-        val message: Either[ExecuteOutput, ExecuteFailure] = Left(outputText)
+        val message: Either[ExecuteOutput, ExecuteFailure] = Left(Map("text/plain" -> outputText))
         val mockDataFrame = mock[org.apache.spark.sql.DataFrame]
         val variableName = "variable"
         val executeCode =s"""--output=html
@@ -130,7 +130,7 @@ class DataFrameSpec extends FunSpec with Matchers with MockitoSugar with BeforeA
         doReturn(Success(outputText)).when(converter).convert(
           mockDataFrame,"html", 10
         )
-        val output = magic.execute(executeCode)
+        val output = magic.execute(executeCode).asMap
         output.contains(MIMEType.TextHtml) should be(true)
         output(MIMEType.TextHtml).contains(outputText) should be(true)
       }
@@ -138,7 +138,7 @@ class DataFrameSpec extends FunSpec with Matchers with MockitoSugar with BeforeA
       it("should return a csv message when csv is the selected output"){
         val (magic, interpreter, converter) = createMocks
         val outputText = "test output"
-        val message: Either[ExecuteOutput, ExecuteFailure] = Left(outputText)
+        val message: Either[ExecuteOutput, ExecuteFailure] = Left(Map("text/plain" -> outputText))
         val mockDataFrame = mock[org.apache.spark.sql.DataFrame]
         val variableName = "variable"
         val executeCode =s"""--output=csv
@@ -150,7 +150,7 @@ class DataFrameSpec extends FunSpec with Matchers with MockitoSugar with BeforeA
         doReturn(Success(outputText)).when(converter).convert(
           mockDataFrame,"csv", 10
         )
-        val output = magic.execute(executeCode)
+        val output = magic.execute(executeCode).asMap
         output.contains(MIMEType.PlainText) should be(true)
         output(MIMEType.PlainText).contains(outputText) should be(true)
       }
@@ -158,7 +158,7 @@ class DataFrameSpec extends FunSpec with Matchers with MockitoSugar with BeforeA
       it("should pass the limit argument to the converter"){
         val (magic, interpreter, converter) = createMocks
         val outputText = "test output"
-        val message: Either[ExecuteOutput, ExecuteFailure] = Left(outputText)
+        val message: Either[ExecuteOutput, ExecuteFailure] = Left(Map("text/plain" -> outputText))
         val mockDataFrame = mock[org.apache.spark.sql.DataFrame]
         val variableName = "variable"
         val executeCode =s"""--output=html --limit=3
@@ -177,7 +177,7 @@ class DataFrameSpec extends FunSpec with Matchers with MockitoSugar with BeforeA
       it("should return a plain text message with help when the converter throws an exception"){
         val (magic, interpreter, converter) = createMocks
         val outputText = "test output"
-        val message: Either[ExecuteOutput, ExecuteFailure] = Left(outputText)
+        val message: Either[ExecuteOutput, ExecuteFailure] = Left(Map("text/plain" -> outputText))
         val mockDataFrame = mock[org.apache.spark.sql.DataFrame]
         val code = "variable"
         when(interpreter.interpret(code)).thenReturn((Results.Success, message))
@@ -186,11 +186,11 @@ class DataFrameSpec extends FunSpec with Matchers with MockitoSugar with BeforeA
         doThrow(new RuntimeException()).when(converter).convert(
           mockDataFrame,"html", 10
         )
-        val output = magic.execute(code)
+        val output = magic.execute(code).asMap
         output.contains(MIMEType.PlainText) should be(true)
         output(MIMEType.PlainText).contains(DataFrameResponses.Usage) should be(true)
 
       }
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/pyspark-interpreter/src/main/scala/org/apache/toree/magic/builtin/PySpark.scala
----------------------------------------------------------------------
diff --git a/pyspark-interpreter/src/main/scala/org/apache/toree/magic/builtin/PySpark.scala b/pyspark-interpreter/src/main/scala/org/apache/toree/magic/builtin/PySpark.scala
index 015320d..c6abb53 100644
--- a/pyspark-interpreter/src/main/scala/org/apache/toree/magic/builtin/PySpark.scala
+++ b/pyspark-interpreter/src/main/scala/org/apache/toree/magic/builtin/PySpark.scala
@@ -18,8 +18,7 @@ package org.apache.toree.magic.builtin
 
 import org.apache.toree.interpreter.{ExecuteError, ExecuteAborted}
 import org.apache.toree.kernel.interpreter.pyspark.{PySparkInterpreter, PySparkException}
-import org.apache.toree.kernel.protocol.v5.MIMEType
-import org.apache.toree.magic.{CellMagicOutput, CellMagic}
+import org.apache.toree.magic.{MagicOutput, CellMagic}
 import org.apache.toree.magic.dependencies.IncludeKernel
 import org.apache.toree.plugins.annotations.Event
 
@@ -28,7 +27,7 @@ import org.apache.toree.plugins.annotations.Event
  */
 class PySpark extends CellMagic with IncludeKernel {
   @Event(name = "pyspark")
-  override def execute(code: String): CellMagicOutput = {
+  override def execute(code: String): MagicOutput = {
     val pySpark = kernel.interpreter("PySpark")
 
     if (pySpark.isEmpty || pySpark.get == null)
@@ -39,7 +38,7 @@ class PySpark extends CellMagic with IncludeKernel {
         val (_, output) = pySparkInterpreter.interpret(code)
         output match {
           case Left(executeOutput) =>
-            CellMagicOutput(MIMEType.PlainText -> executeOutput)
+            MagicOutput(executeOutput.toSeq:_*)
           case Right(executeFailure) => executeFailure match {
             case executeAborted: ExecuteAborted =>
               throw new PySparkException("PySpark code was aborted!")

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/scala-interpreter/build.sbt
----------------------------------------------------------------------
diff --git a/scala-interpreter/build.sbt b/scala-interpreter/build.sbt
index c07b032..39dcb02 100644
--- a/scala-interpreter/build.sbt
+++ b/scala-interpreter/build.sbt
@@ -18,3 +18,4 @@ import sbt.Tests.{Group, SubProcess}
  */
 
 libraryDependencies ++= Dependencies.sparkAll.value
+libraryDependencies += "com.github.jupyter" % "jvm-repr" % "0.1.0"

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/scala-interpreter/src/main/scala-2.11/org/apache/toree/kernel/interpreter/scala/ScalaInterpreterSpecific.scala
----------------------------------------------------------------------
diff --git a/scala-interpreter/src/main/scala-2.11/org/apache/toree/kernel/interpreter/scala/ScalaInterpreterSpecific.scala b/scala-interpreter/src/main/scala-2.11/org/apache/toree/kernel/interpreter/scala/ScalaInterpreterSpecific.scala
index 3cf9407..0b838fc 100644
--- a/scala-interpreter/src/main/scala-2.11/org/apache/toree/kernel/interpreter/scala/ScalaInterpreterSpecific.scala
+++ b/scala-interpreter/src/main/scala-2.11/org/apache/toree/kernel/interpreter/scala/ScalaInterpreterSpecific.scala
@@ -19,7 +19,6 @@ package org.apache.toree.kernel.interpreter.scala
 
 import java.io._
 import java.net.URL
-import java.nio.file.Files
 
 import org.apache.toree.global.StreamState
 import org.apache.toree.interpreter.InterpreterTypes.ExecuteOutput
@@ -405,66 +404,43 @@ trait ScalaInterpreterSpecific extends SettingsProducerLike { this: ScalaInterpr
     exceptionHack.lastException = null
   }
 
-  protected def interpretMapToResultAndExecuteInfo(
-    future: Future[(Results.Result, String)]
-  ): Future[(Results.Result, Either[ExecuteOutput, ExecuteFailure])] = {
-    import scala.concurrent.ExecutionContext.Implicits.global
-    future map {
-      case (Results.Success, output)    => (Results.Success, Left(output))
-      case (Results.Incomplete, output) => (Results.Incomplete, Left(output))
-      case (Results.Aborted, output)    => (Results.Aborted, Right(null))
-      case (Results.Error, output)      =>
-        val ex = Some(retrieveLastException)
-        (
-          Results.Error,
-          Right(
-            interpretConstructExecuteError(
-              ex,
-              output
-            )
-          )
-        )
-    }
-  }
+  protected def interpretConstructExecuteError(output: String) = {
+    Option(retrieveLastException) match {
+      // Runtime error
+      case Some(e) =>
+        val ex = e.asInstanceOf[Throwable]
+        clearLastException()
+
+        // The scala REPL does a pretty good job of returning us a stack trace that is free from all the bits that the
+        // interpreter uses before it.
+        //
+        // The REPL emits its message as something like this, so trim off the first and last element
+        //
+        //    java.lang.ArithmeticException: / by zero
+        //    at failure(<console>:17)
+        //    at call_failure(<console>:19)
+        //    ... 40 elided
+
+        val formattedException = output.split("\n")
 
-  protected def interpretConstructExecuteError(
-    value: Option[AnyRef],
-    output: String
-  ) = value match {
-    // Runtime error
-    case Some(e) if e != null =>
-      val ex = e.asInstanceOf[Throwable]
-      clearLastException()
-
-      // The scala REPL does a pretty good job of returning us a stack trace that is free from all the bits that the
-      // interpreter uses before it.
-      //
-      // The REPL emits its message as something like this, so trim off the first and last element
-      //
-      //    java.lang.ArithmeticException: / by zero
-      //    at failure(<console>:17)
-      //    at call_failure(<console>:19)
-      //    ... 40 elided
-
-      val formattedException = output.split("\n")
-
-      ExecuteError(
-        ex.getClass.getName,
-        ex.getLocalizedMessage,
-        formattedException.slice(1, formattedException.size - 1).toList
-      )
-    // Compile time error, need to check internal reporter
-    case _ =>
-      if (iMain.reporter.hasErrors)
-      // TODO: This wrapper is not needed when just getting compile
-      // error that we are not parsing... maybe have it be purely
-      // output and have the error check this?
         ExecuteError(
-          "Compile Error", output, List()
+          ex.getClass.getName,
+          ex.getLocalizedMessage,
+          formattedException.slice(1, formattedException.size - 1).toList
         )
-      else
+      // Compile time error, need to check internal reporter
+      case _ =>
+        if (iMain.reporter.hasErrors)
+        // TODO: This wrapper is not needed when just getting compile
+        // error that we are not parsing... maybe have it be purely
+        // output and have the error check this?
+          ExecuteError(
+            "Compile Error", output, List()
+          )
+        else
         // May as capture the output here.  Could be useful
-        ExecuteError("Unknown Error", output, List())
+          ExecuteError("Unknown Error", output, List())
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/scala-interpreter/src/main/scala/org/apache/toree/kernel/interpreter/scala/ScalaDisplayers.scala
----------------------------------------------------------------------
diff --git a/scala-interpreter/src/main/scala/org/apache/toree/kernel/interpreter/scala/ScalaDisplayers.scala b/scala-interpreter/src/main/scala/org/apache/toree/kernel/interpreter/scala/ScalaDisplayers.scala
new file mode 100644
index 0000000..08a8f07
--- /dev/null
+++ b/scala-interpreter/src/main/scala/org/apache/toree/kernel/interpreter/scala/ScalaDisplayers.scala
@@ -0,0 +1,209 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements.  See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF 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 org.apache.toree.kernel.interpreter.scala
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.util.Try
+import org.apache.spark.SparkContext
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.SparkSession
+import jupyter.Displayer
+import jupyter.Displayers
+import jupyter.MIMETypes
+import org.apache.toree.kernel.protocol.v5.MIMEType
+import org.apache.toree.magic.MagicOutput
+
+object ScalaDisplayers {
+
+  // This method is called to ensure this class is loaded. When loaded, the display methods below
+  // are registered with jupyter.Displayers and will get used to display Scala and Java objects.
+  def ensureLoaded(): Unit = ()
+
+  private def toJava(body: => Map[String, String]): util.Map[String, String] = {
+    body.asJava
+  }
+
+  Displayers.register(classOf[MagicOutput], new Displayer[MagicOutput] {
+    override def display(data: MagicOutput): util.Map[String, String] = toJava {
+      data.asMap
+    }
+  })
+
+  Displayers.register(classOf[SparkContext], new Displayer[SparkContext] {
+    override def display(sc: SparkContext): util.Map[String, String] = toJava {
+      val appId = sc.applicationId
+      val html =
+        s"""<ul>
+           |<li><a href="${sc.uiWebUrl}" target="new_tab">Spark UI: $appId</a></li>
+           |</ul>""".stripMargin
+      val text =
+        s"""Spark $appId: ${sc.uiWebUrl}"""
+
+      Map(
+        MIMEType.PlainText -> text,
+        MIMEType.TextHtml -> html
+      )
+    }
+  })
+
+  Displayers.register(classOf[Array[Row]], new Displayer[Array[Row]] {
+    override def display(arr: Array[Row]): util.Map[String, String] = toJava {
+      val (text, html) = displayRows(arr)
+      Map(MIMEType.PlainText -> text, MIMEType.TextHtml -> html)
+    }
+  })
+
+  Displayers.register(classOf[Option[_]], new Displayer[Option[_]] {
+    override def display(option: Option[_]): util.Map[String, String] = toJava {
+      val result = new mutable.HashMap[String, String]
+
+      option match {
+        case Some(wrapped) =>
+          Displayers.display(wrapped).asScala.foreach {
+            case (mime, text) if mime == MIMETypes.TEXT =>
+              result.put(mime, "Some(" + text + ")")
+            case (mime, value) =>
+              result.put(mime, value)
+          }
+        case None =>
+          result.put(MIMETypes.TEXT, "None")
+      }
+
+      result.toMap
+    }
+  })
+
+  Displayers.register(classOf[SparkSession], new Displayer[SparkSession] {
+    override def display(spark: SparkSession): util.Map[String, String] = {
+      Displayers.display(spark.sparkContext)
+    }
+  })
+
+  // Set the default displayer to call toHtml if present on Scala objects
+  Displayers.registration.setDefault(new Displayer[Object] {
+    override def display(obj: Object): util.Map[String, String] = toJava {
+      if (obj.getClass.isArray) {
+        Map(MIMETypes.TEXT -> obj.asInstanceOf[Array[_]].map(
+          elem => Displayers.display(elem).get(MIMETypes.TEXT)
+        ).mkString("[", ", ", "]"))
+      } else {
+        val objAsString = String.valueOf(obj)
+        Try(callToHTML(obj)).toOption.flatten match {
+          case Some(html) =>
+            Map(
+              MIMETypes.TEXT -> objAsString,
+              MIMETypes.HTML -> html
+            )
+          case None =>
+            Map(MIMETypes.TEXT -> objAsString)
+        }
+      }
+    }
+
+    private def callToHTML(obj: Any): Option[String] = {
+      import scala.reflect.runtime.{universe => ru}
+      val toHtmlMethodName = ru.TermName("toHtml")
+      val classMirror = ru.runtimeMirror(obj.getClass.getClassLoader)
+      val objMirror = classMirror.reflect(obj)
+      val toHtmlSym = objMirror.symbol.toType.member(toHtmlMethodName)
+      if (toHtmlSym.isMethod) {
+        Option(String.valueOf(objMirror.reflectMethod(toHtmlSym.asMethod).apply()))
+      } else {
+        None
+      }
+    }
+  })
+
+  private def displayRows(
+                   rows: Array[Row],
+                   fields: Option[Seq[String]] = None,
+                   isTruncated: Boolean = false): (String, String) = {
+    if (rows.length < 1) {
+      return ("", "")
+    }
+
+    val lengths = Array.fill(rows(0).length)(3)
+    val cells = rows.map { row =>
+      row.toSeq.zipWithIndex.map {
+        case (value, pos) =>
+          val repr = value match {
+            case null => "NULL"
+            case binary: Array[Byte] =>
+              binary.map("%02X".format(_)).mkString("[", " ", "]")
+            case arr: Array[_] =>
+              arr.mkString("[", ", ", "]")
+            case seq: Seq[_] =>
+              seq.mkString("[", ", ", "]")
+            case map: Map[_, _] =>
+              map.map {
+                case (k: Any, v: Any) => s"$k -> $v"
+              }.mkString("{", ", ", "}")
+            case _ =>
+              value.toString
+          }
+          lengths(pos) = Math.max(lengths(pos), repr.length)
+          repr
+      }
+    }
+
+    fields match {
+      case Some(names) =>
+        names.zipWithIndex.foreach {
+          case (name, pos) =>
+            lengths(pos) = Math.max(lengths(pos), name.length)
+        }
+      case _ =>
+    }
+
+    var htmlLines = new mutable.ArrayBuffer[String]()
+    htmlLines
+    htmlLines += "<table>"
+
+    var lines = new mutable.ArrayBuffer[String]()
+    val divider = lengths.map(l => "-" * (l + 2)).mkString("+", "+", "+")
+    val format = lengths.map(l => s" %-${l}s ").mkString("|", "|", "|")
+    lines += divider
+
+    fields match {
+      case Some(names) =>
+        htmlLines += names.mkString("<tr><th>", "</th><th>", "</th></tr>")
+        lines += String.format(format, names:_*)
+        lines += divider
+      case _ =>
+    }
+
+    cells.foreach { row =>
+      htmlLines += row.mkString("<tr><td>", "</td><td>", "</td></tr>")
+      lines += String.format(format, row: _*)
+    }
+
+    if (isTruncated) {
+      val dots = Array.fill(lengths.length)("...")
+      htmlLines += dots.mkString("<tr><td>", "</td><td>", "</td></tr>")
+      lines += String.format(format, dots: _*)
+    }
+
+    htmlLines += "</table>"
+    lines += divider
+
+    (lines.mkString("\n"), htmlLines.mkString("\n"))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/scala-interpreter/src/main/scala/org/apache/toree/kernel/interpreter/scala/ScalaInterpreter.scala
----------------------------------------------------------------------
diff --git a/scala-interpreter/src/main/scala/org/apache/toree/kernel/interpreter/scala/ScalaInterpreter.scala b/scala-interpreter/src/main/scala/org/apache/toree/kernel/interpreter/scala/ScalaInterpreter.scala
index debf638..50e8c0a 100644
--- a/scala-interpreter/src/main/scala/org/apache/toree/kernel/interpreter/scala/ScalaInterpreter.scala
+++ b/scala-interpreter/src/main/scala/org/apache/toree/kernel/interpreter/scala/ScalaInterpreter.scala
@@ -18,30 +18,34 @@
 package org.apache.toree.kernel.interpreter.scala
 
 import java.io.ByteArrayOutputStream
-import java.net.{URL, URLClassLoader}
-import java.nio.charset.Charset
 import java.util.concurrent.ExecutionException
-
 import com.typesafe.config.{Config, ConfigFactory}
+import jupyter.Displayers
 import org.apache.spark.SparkContext
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.repl.Main
-
 import org.apache.toree.interpreter._
-import org.apache.toree.kernel.api.{KernelLike, KernelOptions}
+import org.apache.toree.kernel.api.KernelLike
 import org.apache.toree.utils.TaskManager
 import org.slf4j.LoggerFactory
 import org.apache.toree.kernel.BuildInfo
-
+import org.apache.toree.kernel.protocol.v5.MIMEType
 import scala.annotation.tailrec
+import scala.collection.JavaConverters._
 import scala.concurrent.{Await, Future}
 import scala.language.reflectiveCalls
 import scala.tools.nsc.Settings
 import scala.tools.nsc.interpreter.{IR, OutputStream}
 import scala.tools.nsc.util.ClassPath
-import scala.util.{Try => UtilTry}
+import scala.util.matching.Regex
 
 class ScalaInterpreter(private val config:Config = ConfigFactory.load) extends Interpreter with ScalaInterpreterSpecific {
+  import ScalaInterpreter._
+
+  ScalaDisplayers.ensureLoaded()
+
+   private var kernel: KernelLike = _
+
    protected val logger = LoggerFactory.getLogger(this.getClass.getName)
 
    protected val _thisClassloader = this.getClass.getClassLoader
@@ -79,6 +83,7 @@ class ScalaInterpreter(private val config:Config = ConfigFactory.load) extends I
     * @return The newly initialized interpreter
     */
    override def init(kernel: KernelLike): Interpreter = {
+     this.kernel = kernel
      val args = interpreterArgs(kernel)
      settings = newSettings(args)
      settings = appendClassPath(settings)
@@ -157,93 +162,83 @@ class ScalaInterpreter(private val config:Config = ConfigFactory.load) extends I
 
    override def interpret(code: String, silent: Boolean = false, output: Option[OutputStream]):
     (Results.Result, Either[ExecuteOutput, ExecuteFailure]) = {
-      val starting = (Results.Success, Left(""))
-      interpretRec(code.trim.split("\n").toList, false, starting)
-   }
-
-   def truncateResult(result:String, showType:Boolean =false, noTruncate: Boolean = false): String = {
-     val resultRX="""(?s)(res\d+):\s+(.+)\s+=\s+(.*)""".r
-
-     result match {
-       case resultRX(varName,varType,resString) => {
-           var returnStr=resString
-           if (noTruncate)
-           {
-             val r=read(varName)
-             returnStr=r.getOrElse("").toString
-           }
-
-           if (showType)
-             returnStr=varType+" = "+returnStr
-
-         returnStr
-
-       }
-       case _ => ""
-     }
-
-
+     interpretBlock(code, silent)
    }
 
-   protected def interpretRec(lines: List[String], silent: Boolean = false, results: (Results.Result, Either[ExecuteOutput, ExecuteFailure])): (Results.Result, Either[ExecuteOutput, ExecuteFailure]) = {
-     lines match {
-       case Nil => results
-       case x :: xs =>
-         val output = interpretLine(x)
-
-         output._1 match {
-           // if success, keep interpreting and aggregate ExecuteOutputs
-           case Results.Success =>
-             val result = for {
-               originalResult <- output._2.left
-             } yield(truncateResult(originalResult, KernelOptions.showTypes,KernelOptions.noTruncation))
-             interpretRec(xs, silent, (output._1, result))
-
-           // if incomplete, keep combining incomplete statements
-           case Results.Incomplete =>
-             xs match {
-               case Nil => interpretRec(Nil, silent, (Results.Incomplete, results._2))
-               case _ => interpretRec(x + "\n" + xs.head :: xs.tail, silent, results)
-             }
-
-           //
-           case Results.Aborted =>
-             output
-              //interpretRec(Nil, silent, output)
-
-           // if failure, stop interpreting and return the error
-           case Results.Error =>
-             val result = for {
-               curr <- output._2.right
-             } yield curr
-             interpretRec(Nil, silent, (output._1, result))
-         }
-     }
-   }
+  def prepareResult(interpreterOutput: String,
+                    showType: Boolean = false,
+                    noTruncate: Boolean = false
+                   ): (Option[AnyRef], Option[String], Option[String]) = {
+    if (interpreterOutput.isEmpty) {
+      return (None, None, None)
+    }
+
+    var lastResult = Option.empty[AnyRef]
+    var lastResultAsString = ""
+    val definitions = new StringBuilder
+    val text = new StringBuilder
+
+    interpreterOutput.split("\n").foreach {
+      case NamedResult(name, vtype, value) if read(name).nonEmpty =>
+        val result = read(name)
+
+        lastResultAsString = result.map(String.valueOf(_)).getOrElse("")
+        lastResult = result
+
+        val defLine = (showType, noTruncate) match {
+          case (true, true) =>
+            s"$name: $vtype = $lastResultAsString\n"
+          case (true, false) =>
+            s"$name: $vtype = $value\n"
+          case (false, true) =>
+            s"$name = $lastResultAsString\n"
+          case (false, false) =>
+            s"$name = $value\n"
+        }
+
+        // suppress interpreter-defined values
+        if (!name.matches("res\\d+")) {
+          definitions.append(defLine)
+        }
+
+      case Definition(defType, name) =>
+        lastResultAsString = ""
+        definitions.append(s"defined $defType $name\n")
+
+      case Import(name) =>
+        // do nothing with the line
+
+      case line if lastResultAsString.contains(line) =>
+        // do nothing with the line
+
+      case line =>
+        text.append(line).append("\n")
+    }
+
+    (lastResult,
+     if (definitions.nonEmpty) Some(definitions.toString) else None,
+     if (text.nonEmpty) Some(text.toString) else None)
+  }
 
+  protected def interpretBlock(code: String, silent: Boolean = false):
+    (Results.Result, Either[ExecuteOutput, ExecuteFailure]) = {
 
-   protected def interpretLine(line: String, silent: Boolean = false):
-     (Results.Result, Either[ExecuteOutput, ExecuteFailure]) =
-   {
-     logger.trace(s"Interpreting line: $line")
+     logger.trace(s"Interpreting line: $code")
 
-     val futureResult = interpretAddTask(line, silent)
+     val futureResult = interpretAddTask(code, silent)
 
      // Map the old result types to our new types
      val mappedFutureResult = interpretMapToCustomResult(futureResult)
 
      // Determine whether to provide an error or output
-     val futureResultAndOutput = interpretMapToResultAndOutput(mappedFutureResult)
-
-     val futureResultAndExecuteInfo =
-       interpretMapToResultAndExecuteInfo(futureResultAndOutput)
+     val futureResultAndExecuteInfo = interpretMapToResultAndOutput(mappedFutureResult)
 
      // Block indefinitely until our result has arrived
      import scala.concurrent.duration._
      Await.result(futureResultAndExecuteInfo, Duration.Inf)
    }
 
-   protected def interpretMapToCustomResult(future: Future[IR.Result]) = {
+   protected def interpretMapToCustomResult(future: Future[IR.Result]): Future[Results.Result] = {
      import scala.concurrent.ExecutionContext.Implicits.global
      future map {
        case IR.Success             => Results.Success
@@ -254,14 +249,32 @@ class ScalaInterpreter(private val config:Config = ConfigFactory.load) extends I
      }
    }
 
-   protected def interpretMapToResultAndOutput(future: Future[Results.Result]) = {
+   protected def interpretMapToResultAndOutput(future: Future[Results.Result]):
+      Future[(Results.Result, Either[Map[String, String], ExecuteError])] = {
      import scala.concurrent.ExecutionContext.Implicits.global
+
      future map {
-       result =>
-         val output =
-           lastResultOut.toString(Charset.forName("UTF-8").name()).trim
+       case result @ (Results.Success | Results.Incomplete) =>
+         val lastOutput = lastResultOut.toString("UTF-8").trim
+         lastResultOut.reset()
+
+         val (obj, defStr, text) = prepareResult(lastOutput)
+         defStr.foreach(kernel.display.content(MIMEType.PlainText, _))
+         text.foreach(kernel.display.content(MIMEType.PlainText, _))
+         val output = obj.map(Displayers.display(_).asScala.toMap).getOrElse(Map.empty)
+         (result, Left(output))
+
+       case Results.Error =>
+         val lastOutput = lastResultOut.toString("UTF-8").trim
          lastResultOut.reset()
-         (result, output)
+
+         val (obj, defStr, text) = prepareResult(lastOutput)
+         defStr.foreach(kernel.display.content(MIMEType.PlainText, _))
+         val output = interpretConstructExecuteError(text.get)
+         (Results.Error, Right(output))
+
+       case Results.Aborted =>
+         (Results.Aborted, Right(null))
      }
    }
 
@@ -348,6 +361,10 @@ class ScalaInterpreter(private val config:Config = ConfigFactory.load) extends I
 
 object ScalaInterpreter {
 
+  val NamedResult: Regex = """(\w+):\s+([^=]+)\s+=\s*(.*)""".r
+  val Definition: Regex = """defined\s+(\w+)\s+(.+)""".r
+  val Import: Regex = """import\s+([\w\.,\{\}\s]+)""".r
+
   /**
     * Utility method to ensure that a temporary directory for the REPL exists for testing purposes.
     */

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/scala-interpreter/src/main/scala/org/apache/toree/magic/builtin/Scala.scala
----------------------------------------------------------------------
diff --git a/scala-interpreter/src/main/scala/org/apache/toree/magic/builtin/Scala.scala b/scala-interpreter/src/main/scala/org/apache/toree/magic/builtin/Scala.scala
index ceeda00..5e8ad00 100644
--- a/scala-interpreter/src/main/scala/org/apache/toree/magic/builtin/Scala.scala
+++ b/scala-interpreter/src/main/scala/org/apache/toree/magic/builtin/Scala.scala
@@ -18,9 +18,8 @@ package org.apache.toree.magic.builtin
 
 import org.apache.toree.interpreter.{ExecuteAborted, ExecuteError}
 import org.apache.toree.kernel.interpreter.scala.{ScalaException, ScalaInterpreter}
-import org.apache.toree.kernel.protocol.v5.MIMEType
 import org.apache.toree.magic.dependencies.IncludeKernel
-import org.apache.toree.magic.{CellMagic, CellMagicOutput}
+import org.apache.toree.magic.{CellMagic, MagicOutput}
 import org.apache.toree.plugins.annotations.Event
 /**
  * Represents the magic interface to use the Scala interpreter.
@@ -28,7 +27,7 @@ import org.apache.toree.plugins.annotations.Event
 class Scala extends CellMagic with IncludeKernel {
 
   @Event(name = "scala")
-  override def execute(code: String): CellMagicOutput = {
+  override def execute(code: String): MagicOutput = {
     val scala = kernel.interpreter("Scala")
 
     if (scala.isEmpty || scala.get == null)
@@ -39,7 +38,7 @@ class Scala extends CellMagic with IncludeKernel {
         val (_, output) = scalaInterpreter.interpret(code)
         output match {
           case Left(executeOutput) =>
-            CellMagicOutput(MIMEType.PlainText -> executeOutput)
+            MagicOutput(executeOutput.toSeq:_*)
           case Right(executeFailure) => executeFailure match {
             case executeAborted: ExecuteAborted =>
               throw new ScalaException("Scala code was aborted!")

http://git-wip-us.apache.org/repos/asf/incubator-toree/blob/5cd17270/scala-interpreter/src/test/scala-2.11/scala/ScalaInterpreterSpec.scala
----------------------------------------------------------------------
diff --git a/scala-interpreter/src/test/scala-2.11/scala/ScalaInterpreterSpec.scala b/scala-interpreter/src/test/scala-2.11/scala/ScalaInterpreterSpec.scala
index 93e7874..f95c518 100644
--- a/scala-interpreter/src/test/scala-2.11/scala/ScalaInterpreterSpec.scala
+++ b/scala-interpreter/src/test/scala-2.11/scala/ScalaInterpreterSpec.scala
@@ -51,8 +51,8 @@ class ScalaInterpreterSpec extends FunSpec
   }
 
   trait SingleLineInterpretLineRec extends StubbedStartInterpreter {
-    override protected def interpretRec(lines: List[String], silent: Boolean, results: (Result, Either[ExecuteOutput, ExecuteFailure])): (Result, Either[ExecuteOutput, ExecuteFailure]) =
-      interpretLine(lines.mkString("\n"))
+    protected def interpretRec(lines: List[String], silent: Boolean, results: (Result, Either[ExecuteOutput, ExecuteFailure])): (Result, Either[ExecuteOutput, ExecuteFailure]) =
+      interpretBlock(lines.mkString("\n"))
   }
 
   trait StubbedInterpretAddTask extends StubbedStartInterpreter {
@@ -67,11 +67,11 @@ class ScalaInterpreterSpec extends FunSpec
 
   trait StubbedInterpretMapToResultAndOutput extends StubbedStartInterpreter {
     override protected def interpretMapToResultAndOutput(future: Future[Results.Result]) =
-      mock[Future[(Results.Result, String)]]
+      mock[Future[(Results.Result, Either[Map[String, String], ExecuteError])]]
   }
 
   trait StubbedInterpretMapToResultAndExecuteInfo extends StubbedStartInterpreter {
-    override protected def interpretMapToResultAndExecuteInfo(future: Future[(Results.Result, String)]) =
+    protected def interpretMapToResultAndExecuteInfo(future: Future[(Results.Result, String)]) =
       mock[Future[(
         Results.Result with Product with Serializable,
         Either[ExecuteOutput, ExecuteFailure] with Product with Serializable
@@ -79,7 +79,7 @@ class ScalaInterpreterSpec extends FunSpec
   }
 
   trait StubbedInterpretConstructExecuteError extends StubbedStartInterpreter {
-    override protected def interpretConstructExecuteError(value: Option[AnyRef], output: String) =
+    protected def interpretConstructExecuteError(value: Option[AnyRef], output: String) =
       mock[ExecuteError]
   }
 
@@ -382,28 +382,46 @@ class ScalaInterpreterSpec extends FunSpec
 //      }
     }
 
-    describe("#truncateResult") {
+    describe("#prepareResult") {
       it("should truncate result of res result") {
+        interpreter.start()
+        doReturn(38).when(mockSparkIMain).eval("i")
+        doReturn(Vector(1, 2)).when(mockSparkIMain).eval("res4")
+        doReturn("snakes").when(mockSparkIMain).eval("resabc")
+
         //  Results that match
-        interpreter.truncateResult("res7: Int = 38") should be("38")
-        interpreter.truncateResult("res7: Int = 38",true) should be("Int = 38")
-        interpreter.truncateResult("res4: String = \nVector(1\n, 2\n)") should be ("Vector(1\n, 2\n)")
-        interpreter.truncateResult("res4: String = \nVector(1\n, 2\n)",true) should be ("String = Vector(1\n, 2\n)")
-        interpreter.truncateResult("res123") should be("")
-        interpreter.truncateResult("res1") should be("")
+        interpreter.prepareResult("i: Int = 38") should be((Some(38), Some("i = 38\n"), None))
+        interpreter.prepareResult("i: Int = 38",true) should be((Some(38), Some("i: Int = 38\n"), None))
+        // resN results are suppressed
+        interpreter.prepareResult("res4: String = \nVector(1\n, 2\n)") should be((Some(Vector(1, 2)), None, None))
+        interpreter.prepareResult("res4: String = \nVector(1\n, 2\n)",true) should be((Some(Vector(1, 2)), None, None))
+        // missing variables are None, unmatched lines are returned in text
+        interpreter.prepareResult("res123") should be((None, None, Some("res123\n")))
+        interpreter.prepareResult("res123: Int = 38") should be((None, None, Some("res123: Int = 38\n")))
         //  Results that don't match
-        interpreter.truncateResult("resabc: Int = 38") should be("")
+        interpreter.prepareResult("resabc: Int = 38") should be((Some("snakes"), Some("resabc = 38\n"), None))
+
+        interpreter.stop()
       }
 
       it("should truncate res results that have tuple values") {
-        interpreter.truncateResult("res0: (String, Int) = (hello,1)") should
-          be("(hello,1)")
+        interpreter.start()
+        doReturn(("hello", 1)).when(mockSparkIMain).eval("res0")
+
+        interpreter.prepareResult("res0: (String, Int) = (hello,1)") should be((Some(("hello", 1)), None, None))
+
+        interpreter.stop()
       }
 
       it("should truncate res results that have parameterized types") {
-        interpreter.truncateResult(
+        interpreter.start()
+        doReturn(scala.Tuple2).when(mockSparkIMain).eval("res0")
+
+        interpreter.prepareResult(
           "res0: Class[_ <: (String, Int)] = class scala.Tuple2"
-        ) should be("class scala.Tuple2")
+        ) should be((Some(scala.Tuple2), None, None))
+
+        interpreter.stop()
       }
     }
   }