You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zeppelin.apache.org by mo...@apache.org on 2016/11/14 19:00:24 UTC

[2/3] zeppelin git commit: Closes [ZEPPELIN-1505] Add Scio interpreter

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/scio/src/test/scala/org/apache/zeppelin/scio/DisplayHelpersTest.scala
----------------------------------------------------------------------
diff --git a/scio/src/test/scala/org/apache/zeppelin/scio/DisplayHelpersTest.scala b/scio/src/test/scala/org/apache/zeppelin/scio/DisplayHelpersTest.scala
new file mode 100644
index 0000000..1ba4c7e
--- /dev/null
+++ b/scio/src/test/scala/org/apache/zeppelin/scio/DisplayHelpersTest.scala
@@ -0,0 +1,548 @@
+/*
+ * 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.zeppelin.scio
+
+import com.google.api.services.bigquery.model.{TableFieldSchema, TableSchema}
+import com.spotify.scio.bigquery._
+import org.apache.avro.Schema
+import org.apache.avro.Schema.Parser
+import org.apache.avro.generic.{GenericData, GenericRecord}
+import org.apache.zeppelin.scio.avro.Account
+import org.apache.zeppelin.scio.util.TestUtils
+import org.junit.runner.RunWith
+import org.scalatest.junit.JUnitRunner
+import org.scalatest.{FlatSpec, Matchers}
+
+/**
+ * DisplayHelpersTest tests.
+ *
+ * Most tests have test scope implicit imports due to scala 2.10 bug
+ * https://issues.scala-lang.org/browse/SI-3346
+ *
+ * Note: we can't depend on the order of data coming from SCollection.
+ */
+@RunWith(classOf[JUnitRunner])
+class DisplayHelpersTest extends FlatSpec with Matchers {
+  private val testRowLimit = 20
+  sys.props("zeppelin.scio.maxResult") = 20.toString
+
+  import TestUtils._
+
+  // -----------------------------------------------------------------------------------------------
+  // AnyVal SCollection Tests
+  // -----------------------------------------------------------------------------------------------
+
+  private val anyValHeader = s"$table value"
+
+  "DisplayHelpers" should "support Integer SCollection via AnyVal" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq(1, 2, 3)) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs Seq(anyValHeader,
+                                           "1",
+                                           "2",
+                                           "3")
+    o.head should be(anyValHeader)
+  }
+
+  it should "support Long SCollection via AnyVal" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq(1L, 2L, 3L)) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs Seq(anyValHeader,
+                                           "1",
+                                           "2",
+                                           "3")
+    o.head should be(anyValHeader)
+  }
+
+  it should "support Double SCollection via AnyVal" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq(1.0D, 2.0D, 3.0D)) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs Seq(anyValHeader,
+                                           "1.0",
+                                           "2.0",
+                                           "3.0")
+    o.head should be(anyValHeader)
+  }
+
+  it should "support Float SCollection via AnyVal" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq(1.0F, 2.0F, 3.0F)) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs Seq(anyValHeader,
+                                           "1.0",
+                                           "2.0",
+                                           "3.0")
+    o.head should be(anyValHeader)
+  }
+
+  it should "support Short SCollection via AnyVal" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq(1.toShort, 2.toShort, 3.toShort)) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs Seq(anyValHeader,
+                                           "1",
+                                           "2",
+                                           "3")
+    o.head should be(anyValHeader)
+  }
+
+  it should "support Byte SCollection via AnyVal" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq(1.toByte, 2.toByte, 3.toByte)) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs Seq(anyValHeader,
+                                           "1",
+                                           "2",
+                                           "3")
+    o.head should be(anyValHeader)
+  }
+
+  it should "support Boolean SCollection via AnyVal" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq(true, false, true)) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs Seq(anyValHeader,
+                                           "true",
+                                           "false",
+                                           "true")
+    o.head should be(anyValHeader)
+  }
+
+  it should "support Char SCollection via AnyVal" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq('a', 'b', 'c')) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs Seq(anyValHeader,
+                                           "a",
+                                           "b",
+                                           "c")
+    o.head should be(anyValHeader)
+  }
+
+  it should "support SCollection of AnyVal over row limit" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinSCollection
+    val o = captureOut {
+      sideEffectWithData(1 to 21) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o.size should be > testRowLimit
+    o.head should be(anyValHeader)
+    o.last should be(rowLimitReached)
+  }
+
+  it should "support empty SCollection of AnyVal" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq.empty[AnyVal]) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs DisplayHelpers.sCollectionEmptyMsg.split(newline)
+  }
+
+  // -----------------------------------------------------------------------------------------------
+  // String SCollection Tests
+  // -----------------------------------------------------------------------------------------------
+
+  private val stringHeader = s"$table value"
+
+  it should "support String SCollection" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinStringSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq("a","b","c")) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs Seq(stringHeader,
+                                           "a",
+                                           "b",
+                                           "c")
+    o.head should be (stringHeader)
+  }
+
+  it should "support empty SCollection of String" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinStringSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq.empty[String]) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs DisplayHelpers.sCollectionEmptyMsg.split(newline)
+  }
+
+  it should "support SCollection of String over row limit" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinStringSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq.fill(21)("a")) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o.size should be > testRowLimit
+    o.head should be(stringHeader)
+    o.last should be(rowLimitReached)
+  }
+
+  // -----------------------------------------------------------------------------------------------
+  // KV SCollection Tests
+  // -----------------------------------------------------------------------------------------------
+
+  private val kvHeader = s"$table key${tab}value"
+
+  it should "support KV (ints) SCollection" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinKVSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq((1,2), (3,4))) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs Seq(kvHeader,
+                                           s"3${tab}4",
+                                           s"1${tab}2")
+    o.head should be (kvHeader)
+  }
+
+  it should "support KV (str keys) SCollection" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinKVSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq(("foo",2), ("bar",4))) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs Seq(kvHeader,
+                                           s"foo${tab}2",
+                                           s"bar${tab}4")
+    o.head should be (kvHeader)
+  }
+
+  it should "support KV (str values) SCollection" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinKVSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq((2,"foo"), (4,"bar"))) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs Seq(kvHeader,
+                                           s"2${tab}foo",
+                                           s"4${tab}bar")
+    o.head should be (kvHeader)
+  }
+
+  it should "support empty KV SCollection" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinKVSCollection
+    captureOut {
+      sideEffectWithData(Seq.empty[(Int, Int)]) { in =>
+        in.closeAndDisplay()
+      }
+    } should contain theSameElementsAs DisplayHelpers.sCollectionEmptyMsg.split(newline)
+  }
+
+  it should "support SCollection of KV over row limit" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinKVSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq.fill(21)(("foo", 1))) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o.size should be > testRowLimit
+    o.head should be(kvHeader)
+    o.last should be(rowLimitReached)
+  }
+
+  // -----------------------------------------------------------------------------------------------
+  // Product SCollection Tests
+  // -----------------------------------------------------------------------------------------------
+
+  private val testCaseClassHeader = s"$table foo${tab}bar${tab}a"
+
+  it should "support SCollection of Tuple of 3" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinProductSCollection
+    val tupleHeader = s"$table _1${tab}_2${tab}_3"
+    val o = captureOut {
+      sideEffectWithData(Seq.fill(3)((1,2,3))) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs (Seq(tupleHeader) ++ Seq.fill(3)(s"1${tab}2${tab}3"))
+    o.head should be(tupleHeader)
+  }
+
+  it should "support SCollection of Tuple of 22" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinProductSCollection
+    val tupleHeader = s"$table " + (1 to 21).map(i => s"_$i$tab").mkString + "_22"
+    val o = captureOut {
+      sideEffectWithData(
+        Seq.fill(3)((1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22))) { in =>
+          in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs (Seq(tupleHeader) ++
+      Seq.fill(3)((1 to 21).map(i => s"$i$tab").mkString + "22"))
+    o.head should be(tupleHeader)
+  }
+
+  it should "support SCollection of Case Class of 22" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinProductSCollection
+    val tupleHeader = s"$table " + (1 to 21).map(i => s"a$i$tab").mkString + "a22"
+    val o = captureOut {
+      sideEffectWithData(
+        Seq.fill(3)(CC22(1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22))) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs (Seq(tupleHeader) ++
+      Seq.fill(3)((1 to 21).map(i => s"$i$tab").mkString + "22"))
+    o.head should be(tupleHeader)
+  }
+
+  it should "support SCollection of Case Class" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinProductSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq.fill(3)(TestCaseClass(1, "foo", 2.0D))) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs (Seq(testCaseClassHeader) ++
+      Seq.fill(3)(s"1${tab}foo${tab}2.0"))
+    o.head should be(testCaseClassHeader)
+  }
+
+  it should "support empty SCollection of Product" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinProductSCollection
+    captureOut {
+      sideEffectWithData(Seq.empty[Product]) { in =>
+        in.closeAndDisplay()
+      }
+    } should contain theSameElementsAs DisplayHelpers.sCollectionEmptyMsg.split(newline)
+  }
+
+  it should "support SCollection of Product over row limit" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinProductSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq.fill(21)(TestCaseClass(1, "foo", 2.0D))) { in =>
+        in.closeAndDisplay()
+      }
+    }
+
+    o.size should be > testRowLimit
+    o.head should be(testCaseClassHeader)
+    o.last should be(rowLimitReached)
+  }
+
+  // -----------------------------------------------------------------------------------------------
+  // Avro SCollection Tests
+  // -----------------------------------------------------------------------------------------------
+
+  import scala.collection.JavaConverters._
+
+  private val schema = {
+    def f(name: String, tpe: Schema.Type) =
+      new Schema.Field(
+        name,
+        Schema.createUnion(List(Schema.create(Schema.Type.NULL), Schema.create(tpe)).asJava),
+        null, null)
+
+    val s = Schema.createRecord("GenericAccountRecord", null, null, false)
+    s.setFields(List(
+      f("id", Schema.Type.INT),
+      f("amount", Schema.Type.DOUBLE),
+      f("name", Schema.Type.STRING),
+      f("type", Schema.Type.STRING)
+    ).asJava)
+    s
+  }
+
+  private def getTestGenericAvro(i: Int): GenericRecord = {
+    val s: Schema = new Parser().parse(schema.toString)
+    val r = new GenericData.Record(s)
+    r.put("id", i)
+    r.put("amount", i.toDouble)
+    r.put("name", "user" + i)
+    r.put("type", "checking")
+    r
+  }
+
+  private def getTestAccountAvro(): Account = {
+    Account.newBuilder()
+      .setId(2)
+      .setAmount(2.0D)
+      .setName("user2")
+      .setType("checking")
+      .build()
+  }
+
+  private val avroGenericRecordHeader = s"$table id${tab}amount${tab}name${tab}type"
+  private val avroAccountHeader = s"$table id${tab}type${tab}name${tab}amount"
+
+  it should "support SCollection of GenericRecord" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinAvroSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq.fill(3)(getTestGenericAvro(1))) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs (Seq(avroGenericRecordHeader) ++
+      Seq.fill(3)(s"1${tab}1.0${tab}user1${tab}checking"))
+    o.head should be(avroGenericRecordHeader)
+  }
+
+  it should "support SCollection of SpecificRecord Avro" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinAvroSCollection
+
+    val o = captureOut {
+      sideEffectWithData(Seq.fill(3)(getTestAccountAvro())) { in =>
+        in.closeAndDisplay()
+      }
+    }
+    o should contain theSameElementsAs (Seq(avroAccountHeader) ++
+      Seq.fill(3)(s"2${tab}checking${tab}user2${tab}2.0"))
+    o.head should be(avroAccountHeader)
+  }
+
+  it should "support empty SCollection of SpecificRecord Avro" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinAvroSCollection
+    captureOut {
+      sideEffectWithData(Seq.empty[Account]) { in =>
+        in.closeAndDisplay()
+      }
+    } should contain theSameElementsAs DisplayHelpers.sCollectionEmptyMsg.split(newline)
+  }
+
+  it should "support empty SCollection of GenericRecord Avro" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinAvroSCollection
+    captureOut {
+      sideEffectWithData(Seq.empty[GenericRecord]) { in =>
+        in.closeAndDisplay()
+      }
+    } should contain theSameElementsAs DisplayHelpers.sCollectionEmptyMsg.split(newline)
+  }
+
+  it should "support SCollection of GenericRecord Avro over row limit" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinAvroSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq.fill(21)(getTestGenericAvro(1))) { in =>
+        in.closeAndDisplay()
+      }
+    }
+
+    o.size should be > testRowLimit
+    o.head should be(avroGenericRecordHeader)
+    o.last should be(rowLimitReached)
+  }
+
+  it should "support SCollection of SpecificRecord Avro over row limit" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinAvroSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq.fill(21)(getTestAccountAvro())) { in =>
+        in.closeAndDisplay()
+      }
+    }
+
+    o.size should be > testRowLimit
+    o.head should be(avroAccountHeader)
+    o.last should be(rowLimitReached)
+  }
+
+  // -----------------------------------------------------------------------------------------------
+  // TableRow SCollection Tests
+  // -----------------------------------------------------------------------------------------------
+
+  private val bQSchema = new TableSchema().setFields(List(
+    new TableFieldSchema().setName("id").setType("INTEGER"),
+    new TableFieldSchema().setName("amount").setType("FLOAT"),
+    new TableFieldSchema().setName("type").setType("STRING"),
+    new TableFieldSchema().setName("name").setType("STRING")
+  ).asJava)
+
+  private val bQHeader = s"$table id${tab}amount${tab}type${tab}name"
+
+  private def getBQTableRow(): TableRow = {
+    TableRow("id" -> 3, "amount" -> 3.0D, "type" -> "checking", "name" -> "user3")
+  }
+
+  it should "support SCollection of TableRow" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinBQTableSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq.fill(3)(getBQTableRow())) { in =>
+        in.closeAndDisplay(bQSchema)
+      }
+    }
+    o should contain theSameElementsAs (Seq(bQHeader) ++
+      Seq.fill(3)(s"3${tab}3.0${tab}checking${tab}user3"))
+    o.head should be(bQHeader)
+  }
+
+  it should "print error on empty BQ schema" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinBQTableSCollection
+    captureOut {
+      sideEffectWithData(Seq.fill(3)(getBQTableRow())) { in =>
+        in.closeAndDisplay(new TableSchema())
+      }
+    } should contain theSameElementsAs DisplayHelpers.bQSchemaIncomplete.split(newline)
+  }
+
+  it should "support SCollection of TableRow over row limit" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinBQTableSCollection
+    val o = captureOut {
+      sideEffectWithData(Seq.fill(21)(getBQTableRow())) { in =>
+        in.closeAndDisplay(bQSchema)
+      }
+    }
+
+    o.size should be > testRowLimit
+    o.head should be(bQHeader)
+    o.last should be(rowLimitReached)
+  }
+
+  it should "support empty SCollection of TableRow" in {
+    import org.apache.zeppelin.scio.DisplaySCollectionImplicits.ZeppelinBQTableSCollection
+    captureOut {
+      sideEffectWithData(Seq.empty[TableRow]) { in =>
+        in.closeAndDisplay(new TableSchema())
+      }
+    } should contain theSameElementsAs DisplayHelpers.sCollectionEmptyMsg.split(newline)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/scio/src/test/scala/org/apache/zeppelin/scio/TestCC.scala
----------------------------------------------------------------------
diff --git a/scio/src/test/scala/org/apache/zeppelin/scio/TestCC.scala b/scio/src/test/scala/org/apache/zeppelin/scio/TestCC.scala
new file mode 100644
index 0000000..8928b99
--- /dev/null
+++ b/scio/src/test/scala/org/apache/zeppelin/scio/TestCC.scala
@@ -0,0 +1,24 @@
+/*
+ * 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.zeppelin.scio
+
+case class TestCaseClass(foo: Int, bar: String, a: Double)
+
+case class CC22(a1: Int, a2: Int, a3: Int, a4: Int, a5: Int, a6: Int, a7: Int, a8: Int, a9: Int,
+                a10: Int, a11: Int, a12: Int, a13: Int, a14: Int, a15: Int, a16: Int, a17: Int,
+                a18: Int, a19: Int, a20: Int, a21: Int, a22: Int)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/scio/src/test/scala/org/apache/zeppelin/scio/util/TestUtils.scala
----------------------------------------------------------------------
diff --git a/scio/src/test/scala/org/apache/zeppelin/scio/util/TestUtils.scala b/scio/src/test/scala/org/apache/zeppelin/scio/util/TestUtils.scala
new file mode 100644
index 0000000..72271b8
--- /dev/null
+++ b/scio/src/test/scala/org/apache/zeppelin/scio/util/TestUtils.scala
@@ -0,0 +1,50 @@
+/*
+ * 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.zeppelin.scio.util
+
+import java.io.{ByteArrayOutputStream, PrintStream}
+
+import com.google.common.base.Charsets
+import com.spotify.scio.ScioContext
+import com.spotify.scio.values.SCollection
+import org.apache.zeppelin.scio.DisplayHelpers
+
+import scala.reflect.ClassTag
+
+object TestUtils {
+  val tab = DisplayHelpers.tab
+  val newline = DisplayHelpers.newline
+  val table = DisplayHelpers.table
+  val rowLimitReached = DisplayHelpers.rowLimitReachedMsg.replaceAll(newline,"")
+
+  private[scio] def sideEffectWithData[T: ClassTag](data: Iterable[T])
+                                                   (fn: SCollection[T] => Unit): Unit = {
+    val sc = ScioContext()
+    fn(sc.parallelize(data))
+    if (!sc.isClosed) sc.close()
+  }
+
+  private[scio] def captureOut[T](body: => T): Seq[String] = {
+    val bytes = new ByteArrayOutputStream()
+    val stream = new PrintStream(bytes)
+    Console.withOut(stream) { body }
+    bytes.toString(Charsets.UTF_8.toString).split(DisplayHelpers.newline)
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/spark/pom.xml
----------------------------------------------------------------------
diff --git a/spark/pom.xml b/spark/pom.xml
index 46a46f1..9e1549d 100644
--- a/spark/pom.xml
+++ b/spark/pom.xml
@@ -320,6 +320,9 @@
           <forkCount>1</forkCount>
           <reuseForks>false</reuseForks>
           <argLine>-Xmx1024m -XX:MaxPermSize=256m</argLine>
+          <excludes>
+            <exclude>**/SparkRInterpreterTest.java</exclude>
+          </excludes>
         </configuration>
       </plugin>
 
@@ -389,6 +392,12 @@
         <groupId>org.scala-tools</groupId>
         <artifactId>maven-scala-plugin</artifactId>
         <version>2.15.2</version>
+        <configuration>
+          <excludes>
+            <exclude>**/ZeppelinR.scala</exclude>
+            <exclude>**/SparkRBackend.scala</exclude>
+          </excludes>
+        </configuration>
         <executions>
           <execution>
             <id>compile</id>
@@ -427,25 +436,6 @@
           </testExcludes>
         </configuration>
       </plugin>
-      <plugin>
-        <groupId>org.scala-tools</groupId>
-        <artifactId>maven-scala-plugin</artifactId>
-        <configuration>
-          <excludes>
-            <exclude>**/ZeppelinR.scala</exclude>
-            <exclude>**/SparkRBackend.scala</exclude>
-          </excludes>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <configuration>
-          <excludes>
-            <exclude>**/SparkRInterpreterTest.java</exclude>
-          </excludes>
-        </configuration>
-      </plugin>
     </plugins>
   </build>
 

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/spark/src/main/java/org/apache/zeppelin/spark/LogOutputStream.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/zeppelin/spark/LogOutputStream.java b/spark/src/main/java/org/apache/zeppelin/spark/LogOutputStream.java
deleted file mode 100644
index d941cd7..0000000
--- a/spark/src/main/java/org/apache/zeppelin/spark/LogOutputStream.java
+++ /dev/null
@@ -1,116 +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.zeppelin.spark;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-
-
-/**
- * Minor modification of LogOutputStream of apache commons exec.
- * LogOutputStream of apache commons exec has one issue that method flush doesn't throw IOException,
- * so that SparkOutputStream can not extend it correctly.
- */
-public abstract class LogOutputStream extends OutputStream {
-  private static final int INTIAL_SIZE = 132;
-  private static final int CR = 13;
-  private static final int LF = 10;
-  private final ByteArrayOutputStream buffer;
-  private boolean skip;
-  private final int level;
-
-  public LogOutputStream() {
-    this(999);
-  }
-
-  public LogOutputStream(int level) {
-    this.buffer = new ByteArrayOutputStream(132);
-    this.skip = false;
-    this.level = level;
-  }
-
-  @Override
-  public void write(int cc) throws IOException {
-    byte c = (byte) cc;
-    if (c != 10 && c != 13) {
-      this.buffer.write(cc);
-    } else if (!this.skip) {
-      this.processBuffer();
-    }
-
-    this.skip = c == 13;
-  }
-
-  @Override
-  public void flush() throws IOException {
-    if (this.buffer.size() > 0) {
-      this.processBuffer();
-    }
-
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (this.buffer.size() > 0) {
-      this.processBuffer();
-    }
-
-    super.close();
-  }
-
-  public int getMessageLevel() {
-    return this.level;
-  }
-
-  @Override
-  public void write(byte[] b, int off, int len) throws IOException {
-    int offset = off;
-    int blockStartOffset = off;
-
-    for (int remaining = len; remaining > 0; blockStartOffset = offset) {
-      while (remaining > 0 && b[offset] != 10 && b[offset] != 13) {
-        ++offset;
-        --remaining;
-      }
-
-      int blockLength = offset - blockStartOffset;
-      if (blockLength > 0) {
-        this.buffer.write(b, blockStartOffset, blockLength);
-      }
-
-      while (remaining > 0 && (b[offset] == 10 || b[offset] == 13)) {
-        this.write(b[offset]);
-        ++offset;
-        --remaining;
-      }
-    }
-
-  }
-
-  protected void processBuffer() {
-    this.processLine(this.buffer.toString());
-    this.buffer.reset();
-  }
-
-  protected void processLine(String line) {
-    this.processLine(line, this.level);
-  }
-
-  protected abstract void processLine(String var1, int var2);
-}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/spark/src/main/java/org/apache/zeppelin/spark/PySparkInterpreter.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/zeppelin/spark/PySparkInterpreter.java b/spark/src/main/java/org/apache/zeppelin/spark/PySparkInterpreter.java
index 582cb6b..13407b2 100644
--- a/spark/src/main/java/org/apache/zeppelin/spark/PySparkInterpreter.java
+++ b/spark/src/main/java/org/apache/zeppelin/spark/PySparkInterpreter.java
@@ -55,6 +55,7 @@ import org.apache.zeppelin.interpreter.LazyOpenInterpreter;
 import org.apache.zeppelin.interpreter.WrappedInterpreter;
 import org.apache.zeppelin.interpreter.InterpreterResult.Code;
 import org.apache.zeppelin.interpreter.thrift.InterpreterCompletion;
+import org.apache.zeppelin.interpreter.util.InterpreterOutputStream;
 import org.apache.zeppelin.spark.dep.SparkDependencyContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -71,7 +72,7 @@ public class PySparkInterpreter extends Interpreter implements ExecuteResultHand
   private GatewayServer gatewayServer;
   private DefaultExecutor executor;
   private int port;
-  private SparkOutputStream outputStream;
+  private InterpreterOutputStream outputStream;
   private BufferedWriter ins;
   private PipedInputStream in;
   private ByteArrayOutputStream input;
@@ -196,7 +197,7 @@ public class PySparkInterpreter extends Interpreter implements ExecuteResultHand
     cmd.addArgument(Integer.toString(port), false);
     cmd.addArgument(Integer.toString(getSparkInterpreter().getSparkVersion().toNumber()), false);
     executor = new DefaultExecutor();
-    outputStream = new SparkOutputStream(logger);
+    outputStream = new InterpreterOutputStream(logger);
     PipedOutputStream ps = new PipedOutputStream();
     in = null;
     try {

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/spark/src/main/java/org/apache/zeppelin/spark/SparkInterpreter.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/zeppelin/spark/SparkInterpreter.java b/spark/src/main/java/org/apache/zeppelin/spark/SparkInterpreter.java
index 53bf30b..41e83ef 100644
--- a/spark/src/main/java/org/apache/zeppelin/spark/SparkInterpreter.java
+++ b/spark/src/main/java/org/apache/zeppelin/spark/SparkInterpreter.java
@@ -55,6 +55,7 @@ import org.apache.zeppelin.interpreter.InterpreterResult;
 import org.apache.zeppelin.interpreter.InterpreterResult.Code;
 import org.apache.zeppelin.interpreter.InterpreterUtils;
 import org.apache.zeppelin.interpreter.WrappedInterpreter;
+import org.apache.zeppelin.interpreter.util.InterpreterOutputStream;
 import org.apache.zeppelin.resource.ResourcePool;
 import org.apache.zeppelin.resource.WellKnownResourceName;
 import org.apache.zeppelin.interpreter.thrift.InterpreterCompletion;
@@ -110,7 +111,7 @@ public class SparkInterpreter extends Interpreter {
   private static Integer sharedInterpreterLock = new Integer(0);
   private static AtomicInteger numReferenceOfSparkContext = new AtomicInteger(0);
 
-  private SparkOutputStream out;
+  private InterpreterOutputStream out;
   private SparkDependencyResolver dep;
 
   /**
@@ -126,7 +127,7 @@ public class SparkInterpreter extends Interpreter {
 
   public SparkInterpreter(Properties property) {
     super(property);
-    out = new SparkOutputStream(logger);
+    out = new InterpreterOutputStream(logger);
   }
 
   public SparkInterpreter(Properties property, SparkContext sc) {

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/spark/src/main/java/org/apache/zeppelin/spark/SparkOutputStream.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/zeppelin/spark/SparkOutputStream.java b/spark/src/main/java/org/apache/zeppelin/spark/SparkOutputStream.java
deleted file mode 100644
index e454994..0000000
--- a/spark/src/main/java/org/apache/zeppelin/spark/SparkOutputStream.java
+++ /dev/null
@@ -1,88 +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.zeppelin.spark;
-
-import org.apache.zeppelin.interpreter.InterpreterOutput;
-import org.slf4j.Logger;
-
-import java.io.IOException;
-
-/**
- * InterpreterOutput can be attached / detached.
- */
-public class SparkOutputStream extends LogOutputStream {
-
-  public static Logger logger;
-  InterpreterOutput interpreterOutput;
-
-  public SparkOutputStream(Logger logger) {
-    this.logger = logger;
-  }
-
-  public InterpreterOutput getInterpreterOutput() {
-    return interpreterOutput;
-  }
-
-  public void setInterpreterOutput(InterpreterOutput interpreterOutput) {
-    this.interpreterOutput = interpreterOutput;
-  }
-
-  @Override
-  public void write(int b) throws IOException {
-    super.write(b);
-    if (interpreterOutput != null) {
-      interpreterOutput.write(b);
-    }
-  }
-
-  @Override
-  public void write(byte [] b) throws IOException {
-    super.write(b);
-    if (interpreterOutput != null) {
-      interpreterOutput.write(b);
-    }
-  }
-
-  @Override
-  public void write(byte [] b, int offset, int len) throws IOException {
-    super.write(b, offset, len);
-    if (interpreterOutput != null) {
-      interpreterOutput.write(b, offset, len);
-    }
-  }
-
-  @Override
-  protected void processLine(String s, int i) {
-    logger.debug("Interpreter output:" + s);
-  }
-
-  @Override
-  public void close() throws IOException {
-    super.close();
-    if (interpreterOutput != null) {
-      interpreterOutput.close();
-    }
-  }
-
-  @Override
-  public void flush() throws IOException {
-    super.flush();
-    if (interpreterOutput != null) {
-      interpreterOutput.flush();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/spark/src/main/java/org/apache/zeppelin/spark/ZeppelinR.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/zeppelin/spark/ZeppelinR.java b/spark/src/main/java/org/apache/zeppelin/spark/ZeppelinR.java
index 961793d..98c6de3 100644
--- a/spark/src/main/java/org/apache/zeppelin/spark/ZeppelinR.java
+++ b/spark/src/main/java/org/apache/zeppelin/spark/ZeppelinR.java
@@ -22,6 +22,7 @@ import org.apache.commons.io.IOUtils;
 import org.apache.zeppelin.interpreter.InterpreterException;
 import org.apache.zeppelin.interpreter.InterpreterOutput;
 import org.apache.zeppelin.interpreter.InterpreterOutputListener;
+import org.apache.zeppelin.interpreter.util.InterpreterOutputStream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,7 +39,7 @@ public class ZeppelinR implements ExecuteResultHandler {
   private final String rCmdPath;
   private final SparkVersion sparkVersion;
   private DefaultExecutor executor;
-  private SparkOutputStream outputStream;
+  private InterpreterOutputStream outputStream;
   private PipedOutputStream input;
   private final String scriptPath;
   private final String libPath;
@@ -146,7 +147,7 @@ public class ZeppelinR implements ExecuteResultHandler {
     logger.debug(cmd.toString());
 
     executor = new DefaultExecutor();
-    outputStream = new SparkOutputStream(logger);
+    outputStream = new InterpreterOutputStream(logger);
 
     input = new PipedOutputStream();
     PipedInputStream in = new PipedInputStream(input);

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/zeppelin-distribution/src/bin_license/LICENSE
----------------------------------------------------------------------
diff --git a/zeppelin-distribution/src/bin_license/LICENSE b/zeppelin-distribution/src/bin_license/LICENSE
index 82e2c6c..bab53fa 100644
--- a/zeppelin-distribution/src/bin_license/LICENSE
+++ b/zeppelin-distribution/src/bin_license/LICENSE
@@ -1,13 +1,9 @@
+The following components are provided under Apache License.
+
     (Apache 2.0) nvd3.js v1.7.1 (http://nvd3.org/) - https://github.com/novus/nvd3/blob/v1.7.1/LICENSE.md
     (Apache 2.0) gson v2.2 (com.google.code.gson:gson:jar:2.2 - https://github.com/google/gson) - https://github.com/google/gson/blob/gson-2.2/LICENSE
     (Apache 2.0) Amazon Web Services SDK for Java v1.10.62 (https://aws.amazon.com/sdk-for-java/) - https://raw.githubusercontent.com/aws/aws-sdk-java/1.10.62/LICENSE.txt
     (Apache 2.0) JavaEWAH v0.7.9 (https://github.com/lemire/javaewah) - https://github.com/lemire/javaewah/blob/master/LICENSE-2.0.txt
-
-
-
-The following components are provided under Apache License.
-
-
     (Apache 2.0) Apache Commons Logging (commons-logging:commons-logging:1.1.1 - http://commons.apache.org/proper/commons-logging/)
     (Apache 2.0) Apache Commons Codec (commons-codec:commons-codec:1.5 - http://commons.apache.org/proper/commons-codec/)
     (Apache 2.0) Apache Commons Collections (commons-collections:commons-collections:3.2.1 - http://commons.apache.org/proper/commons-configuration/)
@@ -19,7 +15,7 @@ The following components are provided under Apache License.
     (Apache 2.0) Http Components (org.apache.httpcomponents:httpclient:4.3.6 - https://github.com/apache/httpclient)
     (Apache 2.0) Apache Commons Lang (org.apache.commons:commons-lang:2.5 - http://commons.apache.org/proper/commons-lang/)
     (Apache 2.0) Apache Commons Lang 3 (org.apache.commons:commons-lang3:3.4 - http://commons.apache.org/proper/commons-lang/)
-    (Apache 2.0) Apache Commons Math 3 (org.apache.commons:commons-math3:3.4.1 - http://commons.apache.org/proper/commons-math/)
+    (Apache 2.0) Apache Commons Math 3 (org.apache.commons:commons-math3:3.6.1 - http://commons.apache.org/proper/commons-math/)
     (Apache 2.0) Apache Commons Net (commons-net:commons-net:2.2 - http://commons.apache.org/proper/commons-net/)
     (Apache 2.0) Apache log4j (log4j:log4j:1.2.17 - http://logging.apache.org/log4j/1.2/)
     (Apache 2.0) Apache Commons Pool2 (commons-exec:commons-pool2:2.3 - https://commons.apache.org/proper/commons-pool/)
@@ -43,17 +39,24 @@ The following components are provided under Apache License.
     (Apache 2.0) Apache Lens (http://lens.apache.org/)
     (Apache 2.0) Apache Flink (http://flink.apache.org/)
     (Apache 2.0) Apache Beam (http://beam.apache.org/)
-    (Apache 2.0) Apache Thrift (http://thrift.apache.org/)
+    (Apache 2.0) Apache Thrift 0.9.2 (org.apache.thrift:libthrift:0.9.2 - http://thrift.apache.org/)
     (Apache 2.0) Apache Lucene (https://lucene.apache.org/)
     (Apache 2.0) Apache Zookeeper (org.apache.zookeeper:zookeeper:jar:3.4.5 - http://zookeeper.apache.org/)
-    (Apache 2.0) Chill (com.twitter:chill-java:jar:0.8.0 - https://github.com/twitter/chill/)
+    (Apache 2.0) Chill (com.twitter:chill:0.8.0 - https://github.com/twitter/chill/)
+    (Apache 2.0) Chill Java (com.twitter:chill-java:jar:0.8.0 - https://github.com/twitter/chill/)
+    (Apache 2.0) Chill Protobuf (com.twitter:chill-protobuf:0.8.0 - https://github.com/twitter/chill/)
     (Apache 2.0) QDox (com.thoughtworks.qdox:qdox:jar:2.0-M3 - https://github.com/paul-hammant/qdox/)
     (Apache 2.0) Codehaus Plexus (org.codehaus.plexus:plexus:jar:1.5.6 - https://codehaus-plexus.github.io/)
+    (Apache 2.0) Codehaus Plexus Interpolation (org.codehaus.plexus:plexus-interpolation:1.14 - https://codehaus-plexus.github.io/)
+    (Apache 2.0) Codehaus Plexus Component Annotations (org.codehaus.plexus:plexus-component-annotations:1.5.5 - https://codehaus-plexus.github.io/)
+    (Apache 2.0) Codehaus Plexus Classwords (org.codehaus.plexus:plexus-classwords:2.4 - https://codehaus-plexus.github.io/)
+    (Apache 2.0) Codehaus Plexus Utils (org.codehaus.plexus:plexus-utils:2.0.7 - https://codehaus-plexus.github.io/)
     (Apache 2.0) findbugs jsr305 (com.google.code.findbugs:jsr305:jar:1.3.9 - http://findbugs.sourceforge.net/)
     (Apache 2.0) Google Guava (com.google.guava:guava:15.0 - https://code.google.com/p/guava-libraries/)
-    (Apache 2.0) Jackson (com.fasterxml.jackson.core:jackson-core:2.5.3 - https://github.com/FasterXML/jackson-core)
-    (Apache 2.0) Jackson (com.fasterxml.jackson.core:jackson-annotations:2.5.3 - https://github.com/FasterXML/jackson-core)
-    (Apache 2.0) Jackson (com.fasterxml.jackson.core:jackson-databind:2.5.3 - https://github.com/FasterXML/jackson-core)
+    (Apache 2.0) Jackson (com.fasterxml.jackson.core:jackson-core:2.7.0 - https://github.com/FasterXML/jackson-core)
+    (Apache 2.0) Jackson (com.fasterxml.jackson.core:jackson-annotations:2.7.0 - https://github.com/FasterXML/jackson-core)
+    (Apache 2.0) Jackson (com.fasterxml.jackson.core:jackson-databind:2.7.0 - https://github.com/FasterXML/jackson-core)
+    (Apache 2.0) Jackson Mapper ASL (org.codehaus.jackson:jackson-mapper-asl:1.9.13 - https://mvnrepository.com/artifact/org.codehaus.jackson/jackson-mapper-asl/1.9.13)
     (Apache 2.0) javax.servlet (org.eclipse.jetty.orbit:javax.servlet:jar:3.1.0.v201112011016 - http://www.eclipse.org/jetty)
     (Apache 2.0) Joda-Time (joda-time:joda-time:2.8.1 - http://www.joda.org/joda-time/)
     (Apache 2.0) Jackson (org.codehaus.jackson:jackson-core-asl:1.9.13 - http://jackson.codehaus.org/)
@@ -92,7 +95,7 @@ The following components are provided under Apache License.
     (Apache 2.0) Lucene Spatial 3D (org.apache.lucene:lucene-spatial3d:5.3.1 - http://lucene.apache.org/lucene-parent/lucene-spatial3d)
     (Apache 2.0) Lucene Suggest (org.apache.lucene:lucene-suggest:5.3.1 - http://lucene.apache.org/lucene-parent/lucene-suggest)
     (Apache 2.0) Elasticsearch: Core (org.elasticsearch:elasticsearch:2.1.0 - http://nexus.sonatype.org/oss-repository-hosting.html/parent/elasticsearch)
-    (Apache 2.0) Joda convert (org.joda:joda-convert:1.2 - http://joda-convert.sourceforge.net)
+    (Apache 2.0) Joda convert (org.joda:joda-convert:1.8.1 - http://joda-convert.sourceforge.net)
     (Apache 2.0) Shiro Core (org.apache.shiro:shiro-core:1.2.3 - https://shiro.apache.org)
     (Apache 2.0) Shiro Web (org.apache.shiro:shiro-web:1.2.3 - https://shiro.apache.org)
     (Apache 2.0) SnakeYAML (org.yaml:snakeyaml:1.15 - http://www.snakeyaml.org)
@@ -121,9 +124,10 @@ The following components are provided under Apache License.
     (Apache 2.0) parboiled-core (org.parboiled:parboiled-core:1.1.7 - https://github.com/sirthias/parboiled)
     (Apache 2.0) ZkClient (com.101tec:zkclient:0.7 - https://github.com/sgroschupf/zkclient)
     (Apache 2.0) jackson-module-scala (com.fasterxml.jackson.module:jackson-module-scala_2.10:2.4.4 - http://wiki.fasterxml.com/JacksonModuleScala)
-    (Apache 2.0) BigQuery API v2-rev295-1.22.0 (com.google.apis:google-api-services-bigquery:v2-rev295-1.22.0 - http://nexus.sonatype.org/oss-repository-hosting.html/google-api-services-bigquery)
+    (Apache 2.0) BigQuery API v2-rev317-1.22.0 (com.google.apis:google-api-services-bigquery:v2-rev317-1.22.0 - http://nexus.sonatype.org/oss-repository-hosting.html/google-api-services-bigquery)
     (Apache 2.0) Google Cloud Debugger API v2-rev8-1.22.0 (com.google.apis:google-api-services-clouddebugger:v2-rev8-1.22.0 - http://nexus.sonatype.org/oss-repository-hosting.html/google-api-services-clouddebugger)
-    (Apache 2.0) Google Dataflow API v1b3-rev30-1.22.0 (com.google.apis:google-api-services-dataflow:v1b3-rev30-1.22.0 - http://nexus.sonatype.org/oss-repository-hosting.html/google-api-services-dataflow)
+    (Apache 2.0) Google Dataflow API v1b3-rev36-1.22.0 (com.google.apis:google-api-services-dataflow:v1b3-rev36-1.22.0 - http://nexus.sonatype.org/oss-repository-hosting.html/google-api-services-dataflow)
+    (Apache 2.0) Google Dataflow SDK 1.7.0 (com.google.cloud.dataflow:google-cloud-dataflow-java-sdk-all:1.7.0 - https://github.com/GoogleCloudPlatform/DataflowJavaSDK)
     (Apache 2.0) Google Cloud Pub/Sub API v1-rev10-1.22.0 (com.google.apis:google-api-services-pubsub:v1-rev10-1.22.0 - http://nexus.sonatype.org/oss-repository-hosting.html/google-api-services-pubsub)
     (Apache 2.0) Cloud Storage JSON API v1-rev71-1.22.0 (com.google.apis:google-api-services-storage:v1-rev71-1.22.0 - http://nexus.sonatype.org/oss-repository-hosting.html/google-api-services-storage)
     (Apache 2.0) gcsio.jar (com.google.cloud.bigdataoss:gcsio:1.4.5 - https://github.com/GoogleCloudPlatform/BigData-interop/gcsio/)
@@ -165,6 +169,52 @@ The following components are provided under Apache License.
     (Apache 2.0) tez-yarn-timeline-history-with-acls (org.apache.tez:tez-yarn-timeline-history-with-acls:0.7.0 - http://tez.apache.org)
     (Apache 2.0) jna (net.java.dev.jna:jna:4.1.0 https://github.com/java-native-access/jna)
     (Apache 2.0) MathJax v2.7.0 - https://github.com/mathjax/MathJax/blob/2.7.0/LICENSE
+    (Apache 2.0) Scio REPL 0.2.4 (com.spotify:scio-repl:0.2.4 - https://github.com/spotify/scio)
+    (Apache 2.0) Scio BigQuery 0.2.4 (com.spotify:scio-bigquery:0.2.4 - https://github.com/spotify/scio)
+    (Apache 2.0) Scio Core 0.2.4 (com.spotify:scio-core:0.2.4 - https://github.com/spotify/scio)
+    (Apache 2.0) Scio Extra 0.2.4 (com.spotify:scio-extra:0.2.4 - https://github.com/spotify/scio)
+    (Apache 2.0) Scio Test 0.2.4 (com.spotify:scio-test:0.2.4 - https://github.com/spotify/scio)
+    (Apache 2.0) Netty Http2 Codec 4.1.0.CR1 (io.netty:netty-codec-http2:4.1.0.CR1 - https://github.com/netty/netty)
+    (Apache 2.0) Netty Http Codec 4.1.0.CR1 (io.netty:netty-codec-http:4.1.0.CR1 - https://github.com/netty/netty)
+    (Apache 2.0) Netty Handler 4.1.0.CR1 (io.netty:netty-handler:4.1.0.CR1 - https://github.com/netty/netty)
+    (Apache 2.0) Netty Buffer 4.1.0.CR1 (io.netty:netty-buffer:4.1.0.CR1 - https://github.com/netty/netty)
+    (Apache 2.0) Netty Common 4.1.0.CR1 (io.netty:netty-common:4.1.0.CR1 - https://github.com/netty/netty)
+    (Apache 2.0) Netty Transport 4.1.0.CR1 (io.netty:netty-transport:4.1.0.CR1 - https://github.com/netty/netty)
+    (Apache 2.0) Netty Resolver 4.1.0.CR1 (io.netty:netty-resolver:4.1.0.CR1 - https://github.com/netty/netty)
+    (Apache 2.0) Netty Codec 4.1.0.CR1 (io.netty:netty-codec:4.1.0.CR1 - https://github.com/netty/netty)
+    (Apache 2.0) Pubsub v1 GRPC Proto 0.0.2 (com.google.api.grpc:grpc-pubsub-v1:0.0.2 - https://github.com/googleapis/googleapis)
+    (Apache 2.0) Core GRPC Proto 0.0.3 (com.google.api.grpc:grpc-core-proto:0.0.3 - https://github.com/googleapis/googleapis)
+    (Apache 2.0) Bigtable Proto 0.3.0 (com.google.cloud.bigtable:bigtable-protos:0.3.0 - https://github.com/googleapis/googleapis)
+    (Apache 2.0) Java Google API Client 1.22.0 (com.google.api-client:google-api-client:1.22.0 - https://github.com/googleapis/googleapis)
+    (Apache 2.0) Java6 Google API Client 1.22.0 (com.google.api-client:google-api-client-java6:1.22.0 - https://github.com/googleapis/googleapis)
+    (Apache 2.0) Jackson2 Google API Client 1.22.0 (com.google.api-client:google-api-client-jackson2:1.22.0 - https://github.com/googleapis/googleapis)
+    (Apache 2.0) Google HTTP Client 1.22.0 (com.google.http-client:google-http-client:1.22.0 - https://github.com/google/google-http-java-client)
+    (Apache 2.0) Jackson Google HTTP Client 1.22.0 (com.google.http-client:google-http-client-jackson:1.22.0 - https://github.com/google/google-http-java-client)
+    (Apache 2.0) Jackson2 Google HTTP Client 1.22.0 (com.google.http-client:google-http-client-jackson2:1.22.0 - https://github.com/google/google-http-java-client)
+    (Apache 2.0) Protobuf Google HTTP Client 1.22.0 (com.google.http-client:google-http-client-protobuf:1.22.0 - https://github.com/google/google-http-java-client)
+    (Apache 2.0) Google OAuth Java6 Client 1.22.0 (com.google.oauth-client:google-oauth-client-java6:1.22.0 - https://github.com/google/google-oauth-java-client)
+    (Apache 2.0) Google OAuth Client 1.22.0 (com.google.oauth-client:google-oauth-client:1.22.0 - https://github.com/google/google-oauth-java-client)
+    (Apache 2.0) Google API Datastore Protobuf v1beta2-rev1-4.0.0 (com.google.apis:google-api-services-datastore-protobuf:v1beta2-rev1-4.0.0 - https://github.com/googleapis/googleapis)
+    (Apache 2.0) Google Datastore v1 Protobuf Client 1.1.0 (com.google.cloud.datastore:datastore-v1-proto-client:1.1.0 - https://github.com/GoogleCloudPlatform/google-cloud-datastore)
+    (Apache 2.0) Google Datastore v1 Protos 1.0.1 (com.google.cloud.datastore:datastore-v1-protos:1.0.1 - https://github.com/googleapis/googleapis)
+    (Apache 2.0) Twitter Algebird 0.12.1 (com.twitter:algebird-core:0.12.1 - https://github.com/twitter/algebird)
+    (Apache 2.0) Breeze 0.12 (org.scalanlp:breeze:0.12 - https://github.com/scalanlp/breeze)
+    (Apache 2.0) Breeze Macros 0.12 (org.scalanlp:breeze-macros:0.12 - https://github.com/scalanlp/breeze)
+    (Apache 2.0) Opencsv 2.3 (net.sf.opencsv:opencsv:2.3 - http://opencsv.sourceforge.net/)
+    (Apache 2.0) Shapeless 2.0.0 (com.chuusai:shapeless:2.0.0 - https://github.com/milessabin/shapeless)
+    (Apache 2.0) Katan CSV 0.1.12 (com.nrinaudo:kantan.csv:0.1.12 - https://github.com/nrinaudo/kantan.csv)
+    (Apache 2.0) Katan Codecs 0.1.6 (com.nrinaudo:kantan.codecs:0.1.6 - https://github.com/nrinaudo/kantan.csv)
+    (Apache 2.0) jansi 1.4 (org.fusesource.jansi:jansi:1.4 - https://github.com/fusesource/jansi)/
+    (Apache 2.0) Maven Plugin API 3.0 (org.apache.maven:maven-plugin-api:3.0 - https://github.com/apache/maven)
+    (Apache 2.0) Maven Artifact 3.0 (org.apache.maven:maven-artifact:3.0 - https://github.com/apache/maven)
+    (Apache 2.0) Maven Model Builder 3.0.3 (org.apache.maven:maven-model-builder:3.0.3 - https://github.com/apache/maven)
+    (Apache 2.0) Maven Repository Metadata 3.0.3 (org.apache.maven:maven-repository-metadata:3.0.3 - https://github.com/apache/maven)
+    (Apache 2.0) Maven Wagon Provider API 1.0 (org.apache.maven.wagon:wagon-provider-api:1.0 - https://mvnrepository.com/artifact/org.apache.maven.wagon/wagon-provider-api/1.0)
+    (Apache 2.0) Maven Wagon HTTP Lightweight 1.0 (org.apache.maven.wagon:wagon-http-lightweight:1.0 - https://mvnrepository.com/artifact/org.apache.maven.wagon/wagon-http-lightweight/1.0)
+    (Apache 2.0) Maven Wagon HTTP 1.0 (org.apache.maven.wagon:wagon-http:1.0 - https://mvnrepository.com/artifact/org.apache.maven.wagon/wagon-http/1.0)
+    (Apache 2.0) Maven Wagon HTTP Shared 1.0 (org.apache.maven.wagon:wagon-http-shared:1.0 - https://mvnrepository.com/artifact/org.apache.maven.wagon/wagon-http-shared/1.0)
+    (Apache 2.0) Commons HTTP Client 3.1 (commons-httpclient:commons-httpclient:3.1 - https://mvnrepository.com/artifact/commons-httpclient/commons-httpclient/3.1)
+    (Apache 2.0) Scalatest 2.2.4 (org.scalatest:scalatest_2.10:2.2.4 - https://github.com/scalatest/scalatest)
 
 ========================================================================
 MIT licenses
@@ -197,14 +247,13 @@ The text of each license is also included at licenses/LICENSE-[project]-[version
     (The MIT License) Moment v2.9.0 (https://github.com/moment/moment) - https://github.com/moment/moment/blob/2.9.0/LICENSE
     (The MIT License) Pikaday v1.3.2 (https://github.com/dbushell/Pikaday) - https://github.com/dbushell/Pikaday/blob/1.3.2/LICENSE
     (The MIT License) slf4j v1.7.10 (org.slf4j:slf4j-api:jar:1.7.10 - http://www.slf4j.org) - http://www.slf4j.org/license.html
+    (The MIT License) slf4j v1.7.21 (org.slf4j:slf4j-simple:1.7.21 - http://www.slf4j.org) - http://www.slf4j.org/license.html
     (The MIT License) slf4j-log4j12 v1.7.10 (org.slf4j:slf4j-log4j12:jar:1.7.10 - http://www.slf4j.org) - http://www.slf4j.org/license.html
     (The MIT License) bcprov-jdk15on v1.51 (org.bouncycastle:bcprov-jdk15on:jar:1.51 - http://www.bouncycastle.org/java.html) - http://www.bouncycastle.org/licence.html
     (The MIT License) AnchorJS (https://github.com/bryanbraun/anchorjs) - https://github.com/bryanbraun/anchorjs/blob/master/README.md#license
     (The MIT License) moment-duration-format v1.3.0 (https://github.com/jsmreese/moment-duration-format) - https://github.com/jsmreese/moment-duration-format/blob/master/LICENSE
     (The MIT License) github-markdown-css 2.4.0 (https://github.com/sindresorhus/github-markdown-css) - https://github.com/sindresorhus/github-markdown-css/blob/gh-pages/license
     (The MIT License) scopt (com.github.scopt:scopt_2.10:3.2.0 - https://github.com/scopt/scopt)
-The following components are provided under the MIT License.
-
     (The MIT License) Objenesis (org.objenesis:objenesis:2.1 - https://github.com/easymock/objenesis) - Copyright (c) 2006-2015 the original author and authors
     (The MIT License) JCL 1.1.1 implemented over SLF4J (org.slf4j:jcl-over-slf4j:1.7.16 - http://www.slf4j.org)
     (The MIT License) JUL to SLF4J bridge (org.slf4j:jul-to-slf4j:1.7.16 - http://www.slf4j.org)
@@ -212,6 +261,11 @@ The following components are provided under the MIT License.
     (The MIT License) minimal-json (com.eclipsesource.minimal-json:minimal-json:0.9.4 - https://github.com/ralfstx/minimal-json)
     (The MIT License) pyrolite (net.razorvine:pyrolite:4.9) - https://github.com/irmen/Pyrolite/blob/v4.9/LICENSE)
     (The MIT License) libpam4j (org.kohsuke:libpam4j:1.8 https://github.com/kohsuke/libpam4j/blob/master/src/main/java/org/jvnet/libpam/PAM.java)
+    (The MIT License) Spire 0.7.4 (org.spire-math:spire:0.7.4 - https://github.com/non/spire)
+    (The MIT License) Spire Macros 0.7.4 (org.spire-math:spire-macros:0.7.4 - https://github.com/non/spire)
+    (The MIT License) Java String Similarity 0.12 (info.debatty:java-string-similarity:0.12 - https://github.com/tdebatty/java-string-similarity)
+    (The MIT License) Java LSH 0.10 (info.debatty:java-lsh:0.10 - https://github.com/tdebatty/java-LSH)
+    (The MIT License) JSoup 1.6.1 (org.jsoup:jsoup:1.6.1 - https://github.com/jhy/jsoup/)
 
 ========================================================================
 BSD-style licenses
@@ -242,8 +296,10 @@ The text of each license is also included at licenses/LICENSE-[project]-[version
     (BSD 3-Clause) io.grpc:grpc-protobuf-lite (io.grpc:grpc-protobuf-lite:0.14.1 - https://github.com/grpc/grpc-java)
     (BSD 3-Clause) io.grpc:grpc-protobuf-nano (io.grpc:grpc-protobuf-nano:0.14.1 - https://github.com/grpc/grpc-java)
     (BSD 3-Clause) io.grpc:grpc-stub (io.grpc:grpc-stub:0.14.1 - https://github.com/grpc/grpc-java)
-     
-     
+    (BSD-3-Clause) Kryo 3.0.3 (com.esotericsoftware:kryo-shaded:3.0.3 - https://github.com/EsotericSoftware/kryo)
+    (BSD-3-Clause) Minilog 1.3.0 (com.esotericsoftware:minlog:1.3.0 - https://github.com/EsotericSoftware/minlog)
+
+
 The following components are provided under the BSD-style License.
 
     (New BSD License) JGit (org.eclipse.jgit:org.eclipse.jgit:jar:4.1.1.201511131810-r - https://eclipse.org/jgit/)
@@ -275,6 +331,10 @@ The following components are provided under the BSD-style License.
     (BSD-style) spire-macros (org.spire-math:spire-macros_2.11:0.7.1 - http://spire-math.org)
     (The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net)
     (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net)
+    (BSD-3-Clause) Scalamacros Paradise 2.1.0 (org.scalamacros:paradise6:2.1.0 - https://github.com/scalamacros/paradise)
+    (BSD-3-Clause) Scalamacros Quasiquotes 2.1.0 (org.scalamacros:quasiquotes:2.1.0 - https://mvnrepository.com/artifact/org.scalamacros/quasiquotes_2.10/2.1.0)
+    (BSD-2-Clause) JUnit Interface 0.11 (com.novocode:junit-interface:0.11 - https://github.com/sbt/junit-interface)
+    (BSD-3-Clause) SBT Test Interface (org.scala-sbt:test-interface:1.0 - https://github.com/sbt/test-interface)
 
 ========================================================================
 CDDL license
@@ -291,11 +351,11 @@ The following components are provided under the CDDL License.
     (CDDL 1.1) jersey-json (com.sun.jersey:jersey-json:1.9 - https://jersey.java.net/jersey-json/)
     (CDDL 1.1) jersey-server (com.sun.jersey:jersey-server:1.9 - https://jersey.java.net/jersey-server/)
     (CDDL 1.1) jersey-guice (com.sun.jersey.contribs:jersey-guice:1.9 - https://jersey.java.net/jersey-contribs/jersey-guice/)
-    (CDDL 1.1) JAXB RI (com.sun.xml.bind:jaxb-impl:2.2.3-1 - http://jaxb.java.net/) 
+    (CDDL 1.1) JAXB RI (com.sun.xml.bind:jaxb-impl:2.2.3-1 - http://jaxb.java.net/)
     (CDDL 1.0) Java Servlet API (javax.servlet:javax.servlet-api:3.1.0 - http://servlet-spec.java.net)
     (CDDL 1.1) (GPL2 w/ CPE) JAXB API bundle for GlassFish V3 (javax.xml.bind:jaxb-api:2.2.2 - https://jaxb.dev.java.net/)
     (CDDL 1.0) (GNU General Public Library) Streaming API for XML (javax.xml.stream:stax-api:1.0-2 - no url defined)
-     
+
 
 ========================================================================
 EPL license
@@ -306,6 +366,10 @@ The following components are provided under the EPL License.
     (EPL 1.0) Aether (org.sonatype.aether - http://www.eclipse.org/aether/)
     (EPL 1.0) JDT Annotations For Enhanced Null Analysis (org.eclipse.jdt:org.eclipse.jdt.annotation:1.1.0 - https://repo.eclipse.org/content/repositories/eclipse-releases/org/eclipse/jdt/org.eclipse.jdt.annotation)
     (EPL 1.0) JRuby (org.jruby.jruby-complete:v1.6.8 - http://www.jruby.org/)
+    (EPL 1.0) Sisu Inject Plexus 2.2.2 (org.sonatype.sisu:sisu-inject-plexus:2.2.2 - https://github.com/sonatype/sisu)
+    (EPL 1.0) Sisu Inject Bean 2.2.2 (org.sonatype.sisu:sisu-inject-bean:2.2.2 - https://github.com/sonatype/sisu)
+    (EPL 1.0) Sisu Inject Guice (org.sonatype.sisu:sisu-inject-guice:no_aop- https://github.com/sonatype/sisu)
+    (EPL 1.0) JUnit 4.11 (junit:junit:4.11 - https://github.com/junit-team/junit4)
 
 
 
@@ -342,3 +406,8 @@ Creative Commons CC0 (http://creativecommons.org/publicdomain/zero/1.0/)
     (Public Domain, per Creative Commons CC0) HdrHistogram (org.hdrhistogram:HdrHistogram:2.1.6 - http://hdrhistogram.github.io/HdrHistogram/)
     (Public Domain) XZ for Java (org.tukaani:xz:1.0 - http://tukaani.org/xz/java.html)
     (Public Domain) AOP alliance (aopalliance:aopalliance:1.0 - http://aopalliance.sourceforge.net)
+
+========================================================================
+Multiple licenses
+========================================================================
+  (LGPLv2) (GPLv2) (MPL 1.1) Jtransforms (com.github.rwl:jtransforms:2.4.0 - https://sourceforge.net/projects/jtransforms/)

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/zeppelin-distribution/src/bin_license/licenses/LICENSE-java-lsh-0.10
----------------------------------------------------------------------
diff --git a/zeppelin-distribution/src/bin_license/licenses/LICENSE-java-lsh-0.10 b/zeppelin-distribution/src/bin_license/licenses/LICENSE-java-lsh-0.10
new file mode 100644
index 0000000..73e1545
--- /dev/null
+++ b/zeppelin-distribution/src/bin_license/licenses/LICENSE-java-lsh-0.10
@@ -0,0 +1,7 @@
+Copyright 2015 Thibault Debatty.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/zeppelin-distribution/src/bin_license/licenses/LICENSE-java-string-similarity-0.12
----------------------------------------------------------------------
diff --git a/zeppelin-distribution/src/bin_license/licenses/LICENSE-java-string-similarity-0.12 b/zeppelin-distribution/src/bin_license/licenses/LICENSE-java-string-similarity-0.12
new file mode 100644
index 0000000..73e1545
--- /dev/null
+++ b/zeppelin-distribution/src/bin_license/licenses/LICENSE-java-string-similarity-0.12
@@ -0,0 +1,7 @@
+Copyright 2015 Thibault Debatty.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/zeppelin-distribution/src/bin_license/licenses/LICENSE-jsoup-1.6.1
----------------------------------------------------------------------
diff --git a/zeppelin-distribution/src/bin_license/licenses/LICENSE-jsoup-1.6.1 b/zeppelin-distribution/src/bin_license/licenses/LICENSE-jsoup-1.6.1
new file mode 100644
index 0000000..9e15540
--- /dev/null
+++ b/zeppelin-distribution/src/bin_license/licenses/LICENSE-jsoup-1.6.1
@@ -0,0 +1,21 @@
+The MIT License
+
+� 2009-2016, Jonathan Hedley <jo...@hedley.net>
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/zeppelin-distribution/src/bin_license/licenses/LICENSE-jtransforms-2.4.0
----------------------------------------------------------------------
diff --git a/zeppelin-distribution/src/bin_license/licenses/LICENSE-jtransforms-2.4.0 b/zeppelin-distribution/src/bin_license/licenses/LICENSE-jtransforms-2.4.0
new file mode 100644
index 0000000..870b10d
--- /dev/null
+++ b/zeppelin-distribution/src/bin_license/licenses/LICENSE-jtransforms-2.4.0
@@ -0,0 +1,33 @@
+/* ***** BEGIN LICENSE BLOCK *****
+ * Version: MPL 1.1/GPL 2.0/LGPL 2.1
+ *
+ * The contents of this file are subject to the Mozilla Public License Version
+ * 1.1 (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.mozilla.org/MPL/
+ *
+ * Software distributed under the License is distributed on an "AS IS" basis,
+ * WITHOUT WARRANTY OF ANY KIND, either express or implied. See the License
+ * for the specific language governing rights and limitations under the
+ * License.
+ *
+ * The Original Code is JTransforms.
+ *
+ * The Initial Developer of the Original Code is
+ * Piotr Wendykier, Emory University.
+ * Portions created by the Initial Developer are Copyright (C) 2007-2009
+ * the Initial Developer. All Rights Reserved.
+ *
+ * Alternatively, the contents of this file may be used under the terms of
+ * either the GNU General Public License Version 2 or later (the "GPL"), or
+ * the GNU Lesser General Public License Version 2.1 or later (the "LGPL"),
+ * in which case the provisions of the GPL or the LGPL are applicable instead
+ * of those above. If you wish to allow use of your version of this file only
+ * under the terms of either the GPL or the LGPL, and not to allow others to
+ * use your version of this file under the terms of the MPL, indicate your
+ * decision by deleting the provisions above and replace them with the notice
+ * and other provisions required by the GPL or the LGPL. If you do not delete
+ * the provisions above, a recipient may use your version of this file under
+ * the terms of any one of the MPL, the GPL or the LGPL.
+ *
+ * ***** END LICENSE BLOCK ***** */

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/zeppelin-distribution/src/bin_license/licenses/LICENSE-junit-interface-0.11
----------------------------------------------------------------------
diff --git a/zeppelin-distribution/src/bin_license/licenses/LICENSE-junit-interface-0.11 b/zeppelin-distribution/src/bin_license/licenses/LICENSE-junit-interface-0.11
new file mode 100644
index 0000000..c9ef892
--- /dev/null
+++ b/zeppelin-distribution/src/bin_license/licenses/LICENSE-junit-interface-0.11
@@ -0,0 +1,24 @@
+Copyright (c) 2009-2012, Stefan Zeiger
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright notice,
+      this list of conditions and the following disclaimer.
+
+    * Redistributions in binary form must reproduce the above copyright
+      notice, this list of conditions and the following disclaimer in the
+      documentation and/or other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
+LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+POSSIBILITY OF SUCH DAMAGE.

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/zeppelin-distribution/src/bin_license/licenses/LICENSE-kryo-shaded-3.0.3
----------------------------------------------------------------------
diff --git a/zeppelin-distribution/src/bin_license/licenses/LICENSE-kryo-shaded-3.0.3 b/zeppelin-distribution/src/bin_license/licenses/LICENSE-kryo-shaded-3.0.3
new file mode 100644
index 0000000..e1cd884
--- /dev/null
+++ b/zeppelin-distribution/src/bin_license/licenses/LICENSE-kryo-shaded-3.0.3
@@ -0,0 +1,10 @@
+Copyright (c) 2008, Nathan Sweet
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution.
+    * Neither the name of Esoteric Software nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/zeppelin-distribution/src/bin_license/licenses/LICENSE-minilog-1.3.0
----------------------------------------------------------------------
diff --git a/zeppelin-distribution/src/bin_license/licenses/LICENSE-minilog-1.3.0 b/zeppelin-distribution/src/bin_license/licenses/LICENSE-minilog-1.3.0
new file mode 100644
index 0000000..e1cd884
--- /dev/null
+++ b/zeppelin-distribution/src/bin_license/licenses/LICENSE-minilog-1.3.0
@@ -0,0 +1,10 @@
+Copyright (c) 2008, Nathan Sweet
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution.
+    * Neither the name of Esoteric Software nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/zeppelin-distribution/src/bin_license/licenses/LICENSE-sbt-test-interface-1.0
----------------------------------------------------------------------
diff --git a/zeppelin-distribution/src/bin_license/licenses/LICENSE-sbt-test-interface-1.0 b/zeppelin-distribution/src/bin_license/licenses/LICENSE-sbt-test-interface-1.0
new file mode 100644
index 0000000..21655fa
--- /dev/null
+++ b/zeppelin-distribution/src/bin_license/licenses/LICENSE-sbt-test-interface-1.0
@@ -0,0 +1,25 @@
+
+Copyright (c) 2009, 2010  Josh Cough, Mark Harrah
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions
+are met:
+1. Redistributions of source code must retain the above copyright
+   notice, this list of conditions and the following disclaimer.
+2. Redistributions in binary form must reproduce the above copyright
+   notice, this list of conditions and the following disclaimer in the
+   documentation and/or other materials provided with the distribution.
+3. The name of the author may not be used to endorse or promote products
+   derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/zeppelin-distribution/src/bin_license/licenses/LICENSE-scalamacros-paradise-2.1.0
----------------------------------------------------------------------
diff --git a/zeppelin-distribution/src/bin_license/licenses/LICENSE-scalamacros-paradise-2.1.0 b/zeppelin-distribution/src/bin_license/licenses/LICENSE-scalamacros-paradise-2.1.0
new file mode 100644
index 0000000..6c8bb4d
--- /dev/null
+++ b/zeppelin-distribution/src/bin_license/licenses/LICENSE-scalamacros-paradise-2.1.0
@@ -0,0 +1,27 @@
+Copyright (c) 2013-2015 EPFL
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+  * Redistributions of source code must retain the above copyright notice,
+    this list of conditions and the following disclaimer.
+  * Redistributions in binary form must reproduce the above copyright notice,
+    this list of conditions and the following disclaimer in the documentation
+    and/or other materials provided with the distribution.
+  * Neither the name of the EPFL nor the names of its contributors
+    may be used to endorse or promote products derived from this software
+    without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
+CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/zeppelin-distribution/src/bin_license/licenses/LICENSE-scalamacros-quasiquotes-2.1.0
----------------------------------------------------------------------
diff --git a/zeppelin-distribution/src/bin_license/licenses/LICENSE-scalamacros-quasiquotes-2.1.0 b/zeppelin-distribution/src/bin_license/licenses/LICENSE-scalamacros-quasiquotes-2.1.0
new file mode 100644
index 0000000..6c8bb4d
--- /dev/null
+++ b/zeppelin-distribution/src/bin_license/licenses/LICENSE-scalamacros-quasiquotes-2.1.0
@@ -0,0 +1,27 @@
+Copyright (c) 2013-2015 EPFL
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+  * Redistributions of source code must retain the above copyright notice,
+    this list of conditions and the following disclaimer.
+  * Redistributions in binary form must reproduce the above copyright notice,
+    this list of conditions and the following disclaimer in the documentation
+    and/or other materials provided with the distribution.
+  * Neither the name of the EPFL nor the names of its contributors
+    may be used to endorse or promote products derived from this software
+    without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
+CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/zeppelin-distribution/src/bin_license/licenses/LICENSE-slf4j-simple-1.7.21
----------------------------------------------------------------------
diff --git a/zeppelin-distribution/src/bin_license/licenses/LICENSE-slf4j-simple-1.7.21 b/zeppelin-distribution/src/bin_license/licenses/LICENSE-slf4j-simple-1.7.21
new file mode 100644
index 0000000..28ba154
--- /dev/null
+++ b/zeppelin-distribution/src/bin_license/licenses/LICENSE-slf4j-simple-1.7.21
@@ -0,0 +1,21 @@
+Copyright (c) 2004-2013 QOS.ch
+ All rights reserved.
+
+ Permission is hereby granted, free  of charge, to any person obtaining
+ a  copy  of this  software  and  associated  documentation files  (the
+ "Software"), to  deal in  the Software without  restriction, including
+ without limitation  the rights to  use, copy, modify,  merge, publish,
+ distribute,  sublicense, and/or sell  copies of  the Software,  and to
+ permit persons to whom the Software  is furnished to do so, subject to
+ the following conditions:
+
+ The  above  copyright  notice  and  this permission  notice  shall  be
+ included in all copies or substantial portions of the Software.
+
+ THE  SOFTWARE IS  PROVIDED  "AS  IS", WITHOUT  WARRANTY  OF ANY  KIND,
+ EXPRESS OR  IMPLIED, INCLUDING  BUT NOT LIMITED  TO THE  WARRANTIES OF
+ MERCHANTABILITY,    FITNESS    FOR    A   PARTICULAR    PURPOSE    AND
+ NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+ LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+ OF CONTRACT, TORT OR OTHERWISE,  ARISING FROM, OUT OF OR IN CONNECTION
+ WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/zeppelin-distribution/src/bin_license/licenses/LICENSE-spire-0.7.4
----------------------------------------------------------------------
diff --git a/zeppelin-distribution/src/bin_license/licenses/LICENSE-spire-0.7.4 b/zeppelin-distribution/src/bin_license/licenses/LICENSE-spire-0.7.4
new file mode 100644
index 0000000..645cba6
--- /dev/null
+++ b/zeppelin-distribution/src/bin_license/licenses/LICENSE-spire-0.7.4
@@ -0,0 +1,19 @@
+Copyright (c) 2011-2012 Erik Osheim, Tom Switzer
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+of the Software, and to permit persons to whom the Software is furnished to do
+so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/zeppelin-distribution/src/bin_license/licenses/LICENSE-spire-macros-0.7.4
----------------------------------------------------------------------
diff --git a/zeppelin-distribution/src/bin_license/licenses/LICENSE-spire-macros-0.7.4 b/zeppelin-distribution/src/bin_license/licenses/LICENSE-spire-macros-0.7.4
new file mode 100644
index 0000000..645cba6
--- /dev/null
+++ b/zeppelin-distribution/src/bin_license/licenses/LICENSE-spire-macros-0.7.4
@@ -0,0 +1,19 @@
+Copyright (c) 2011-2012 Erik Osheim, Tom Switzer
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+of the Software, and to permit persons to whom the Software is furnished to do
+so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/f127237f/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/util/InterpreterOutputStream.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/util/InterpreterOutputStream.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/util/InterpreterOutputStream.java
new file mode 100644
index 0000000..b6f01b1
--- /dev/null
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/util/InterpreterOutputStream.java
@@ -0,0 +1,91 @@
+/*
+ * 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.zeppelin.interpreter.util;
+
+import org.apache.zeppelin.interpreter.InterpreterOutput;
+import org.slf4j.Logger;
+
+import java.io.IOException;
+
+/**
+ * Output Stream integrated with InterpreterOutput.
+ *
+ * Can be used to channel output from interpreters.
+ */
+public class InterpreterOutputStream extends LogOutputStream {
+  public static Logger logger;
+  InterpreterOutput interpreterOutput;
+
+  public InterpreterOutputStream(Logger logger) {
+    this.logger = logger;
+  }
+
+  public InterpreterOutput getInterpreterOutput() {
+    return interpreterOutput;
+  }
+
+  public void setInterpreterOutput(InterpreterOutput interpreterOutput) {
+    this.interpreterOutput = interpreterOutput;
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    super.write(b);
+    if (interpreterOutput != null) {
+      interpreterOutput.write(b);
+    }
+  }
+
+  @Override
+  public void write(byte [] b) throws IOException {
+    super.write(b);
+    if (interpreterOutput != null) {
+      interpreterOutput.write(b);
+    }
+  }
+
+  @Override
+  public void write(byte [] b, int offset, int len) throws IOException {
+    super.write(b, offset, len);
+    if (interpreterOutput != null) {
+      interpreterOutput.write(b, offset, len);
+    }
+  }
+
+  @Override
+  protected void processLine(String s, int i) {
+    logger.debug("Interpreter output:" + s);
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    if (interpreterOutput != null) {
+      interpreterOutput.close();
+    }
+  }
+
+
+  @Override
+  public void flush() throws IOException {
+    super.flush();
+    if (interpreterOutput != null) {
+      interpreterOutput.flush();
+    }
+  }
+}