You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zeppelin.apache.org by zj...@apache.org on 2018/06/15 01:10:08 UTC

[13/13] zeppelin git commit: ZEPPELIN-2035. BI directional RPC framework between ZeppelinServer and InterpreterProcess on top of thrift

ZEPPELIN-2035. BI directional RPC framework between ZeppelinServer and InterpreterProcess on top of thrift

### What is this PR for?

This PR introduce the bi directional RPC framework between ZeppelinServer and InterpreterProcess on top of thrift, see the design doc here. https://docs.google.com/document/d/1B_4b7UuvE7RwbfpTjxV2RSxdDeF-p9vXVYC8ace76-Q/edit?usp=sharing

This PR is pretty big, most of the code changes is due to regeneration of thrift code.
Another improvement is introduce the builder pattern of InterpreterContext as the arguments of InterpreterContext constructor is too many.

### What type of PR is it?
[ Improvement | Refactoring]

### Todos
* [ ] - Task

### What is the Jira issue?
* https://issues.apache.org/jira/browse/ZEPPELIN-2035

### How should this be tested?
* CI pass

### Screenshots (if appropriate)

### Questions:
* Does the licenses files need update? No
* Is there breaking changes for older versions? No
* Does this needs documentation? No

Author: Jeff Zhang <zj...@apache.org>

Closes #2986 from zjffdu/ZEPPELIN-2035 and squashes the following commits:

c02d5587a [Jeff Zhang] ZEPPELIN-2035. BI directional RPC framework between ZeppelinServer and InterpreterProcess on top of thrift


Project: http://git-wip-us.apache.org/repos/asf/zeppelin/repo
Commit: http://git-wip-us.apache.org/repos/asf/zeppelin/commit/7af86168
Tree: http://git-wip-us.apache.org/repos/asf/zeppelin/tree/7af86168
Diff: http://git-wip-us.apache.org/repos/asf/zeppelin/diff/7af86168

Branch: refs/heads/master
Commit: 7af86168254e0ad08234c57043e18179fca8d04c
Parents: d88a002
Author: Jeff Zhang <zj...@apache.org>
Authored: Tue Mar 20 14:05:31 2018 +0800
Committer: Jeff Zhang <zj...@apache.org>
Committed: Fri Jun 15 09:09:49 2018 +0800

----------------------------------------------------------------------
 bin/interpreter.sh                              |     9 +-
 .../ElasticsearchInterpreterTest.java           |     8 +-
 .../zeppelin/flink/FlinkInterpreterTest.java    |    14 +-
 .../org/apache/zeppelin/groovy/GObject.java     |   131 +-
 .../zeppelin/groovy/GroovyZeppelinContext.java  |    49 +
 .../apache/zeppelin/helium/DevInterpreter.java  |    22 +-
 .../zeppelin/helium/DevZeppelinContext.java     |    49 +
 .../helium/ZeppelinApplicationDevServer.java    |     2 +-
 .../zeppelin/helium/ZeppelinDevServer.java      |     4 +-
 .../zeppelin/ignite/IgniteInterpreterTest.java  |    13 +-
 .../ignite/IgniteSqlInterpreterTest.java        |    17 +-
 .../jdbc/JDBCInterpreterInterpolationTest.java  |     7 +-
 .../zeppelin/jdbc/JDBCInterpreterTest.java      |    29 +-
 .../zeppelin/livy/LivySparkSQLInterpreter.java  |    33 +-
 .../apache/zeppelin/livy/LivyInterpreterIT.java |   132 +-
 .../graph/neo4j/Neo4jCypherInterpreterTest.java |    36 +-
 .../zeppelin/pig/PigInterpreterSparkTest.java   |     3 +-
 .../apache/zeppelin/pig/PigInterpreterTest.java |     3 +-
 .../zeppelin/pig/PigInterpreterTezTest.java     |     3 +-
 .../zeppelin/pig/PigQueryInterpreterTest.java   |     3 +-
 .../python/BasePythonInterpreterTest.java       |    32 +-
 .../zeppelin/python/IPythonInterpreterTest.java |     3 +-
 .../python/PythonCondaInterpreterTest.java      |    38 +-
 .../python/PythonDockerInterpreterTest.java     |    50 +-
 .../python/PythonInterpreterMatplotlibTest.java |    12 +-
 .../python/PythonInterpreterPandasSqlTest.java  |    12 +-
 .../zeppelin/python/PythonInterpreterTest.java  |     3 +-
 .../scalding/ScaldingInterpreterTest.java       |    29 +-
 .../zeppelin/scio/ScioInterpreterTest.java      |    13 +-
 .../zeppelin/shell/ShellInterpreterTest.java    |     3 +-
 .../zeppelin/spark/NewSparkInterpreter.java     |    13 +-
 .../zeppelin/spark/OldSparkInterpreter.java     |    10 +-
 .../zeppelin/spark/SparkRInterpreter.java       |     5 +-
 .../zeppelin/spark/DepInterpreterTest.java      |    28 +-
 .../zeppelin/spark/IPySparkInterpreterTest.java |    71 +-
 .../zeppelin/spark/NewSparkInterpreterTest.java |    42 +-
 .../spark/NewSparkSqlInterpreterTest.java       |    39 +-
 .../zeppelin/spark/OldSparkInterpreterTest.java |    72 +-
 .../spark/OldSparkSqlInterpreterTest.java       |    20 +-
 .../spark/PySparkInterpreterMatplotlibTest.java |    50 +-
 .../zeppelin/spark/PySparkInterpreterTest.java  |    20 +-
 .../zeppelin/spark/SparkRInterpreterTest.java   |    21 +-
 .../apache/zeppelin/spark/SparkShimsTest.java   |    30 +-
 .../org/apache/zeppelin/spark/SparkShims.java   |    32 +-
 .../org/apache/zeppelin/spark/Spark1Shims.java  |     7 +-
 .../org/apache/zeppelin/spark/Spark2Shims.java  |     7 +-
 .../angular/AbstractAngularElemTest.scala       |    11 +-
 .../angular/AbstractAngularModelTest.scala      |    11 +-
 .../zeppelin/conf/ZeppelinConfiguration.java    |    10 +-
 .../apache/zeppelin/display/AngularObject.java  |    12 +-
 .../interpreter/BaseZeppelinContext.java        |   315 +-
 .../interpreter/InterpreterContext.java         |   175 +-
 .../interpreter/RemoteWorksController.java      |    29 -
 .../launcher/InterpreterLaunchContext.java      |    16 +-
 .../interpreter/remote/RemoteEventClient.java   |    34 -
 .../remote/RemoteInterpreterContextRunner.java  |    37 -
 .../remote/RemoteInterpreterEventClient.java    |   578 +-
 .../remote/RemoteInterpreterServer.java         |   362 +-
 .../remote/RemoteInterpreterUtils.java          |    36 +-
 .../interpreter/thrift/AngularObjectId.java     |   625 +
 .../thrift/AppOutputAppendEvent.java            |   833 ++
 .../thrift/AppOutputUpdateEvent.java            |   938 ++
 .../thrift/AppStatusUpdateEvent.java            |   730 +
 .../interpreter/thrift/CallbackInfo.java        |   518 -
 .../thrift/InterpreterCompletion.java           |     4 +-
 .../interpreter/thrift/OutputAppendEvent.java   |   833 ++
 .../thrift/OutputUpdateAllEvent.java            |   679 +
 .../interpreter/thrift/OutputUpdateEvent.java   |   938 ++
 .../interpreter/thrift/RegisterInfo.java        |   623 +
 .../thrift/RemoteApplicationResult.java         |     6 +-
 .../RemoteInterpreterCallbackService.java       |   879 --
 .../thrift/RemoteInterpreterContext.java        |   117 +-
 .../thrift/RemoteInterpreterEvent.java          |     4 +-
 .../thrift/RemoteInterpreterEventService.java   | 13261 +++++++++++++++++
 .../thrift/RemoteInterpreterEventType.java      |     2 +-
 .../thrift/RemoteInterpreterResult.java         |     4 +-
 .../thrift/RemoteInterpreterResultMessage.java  |     4 +-
 .../thrift/RemoteInterpreterService.java        |  5682 ++-----
 .../interpreter/thrift/RunParagraphsEvent.java  |   828 +
 .../ZeppelinServerResourceParagraphRunner.java  |     4 +-
 .../thrift/RemoteInterpreterEventService.thrift |   111 +
 .../main/thrift/RemoteInterpreterService.thrift |    52 +-
 .../src/main/thrift/genthrift.sh                |     1 +
 .../display/AngularObjectRegistryTest.java      |     7 +-
 .../zeppelin/display/AngularObjectTest.java     |     5 +-
 .../interpreter/InterpreterContextTest.java     |     4 +-
 .../zeppelin/interpreter/InterpreterTest.java   |    22 +-
 .../interpreter/LazyOpenInterpreterTest.java    |     4 +-
 .../interpreter/ZeppCtxtVariableTest.java       |    22 +-
 .../remote/RemoteInterpreterServerTest.java     |    28 +-
 .../apache/zeppelin/socket/NotebookServer.java  |   173 +-
 .../src/test/resources/log4j.properties         |     2 +
 .../interpreter/InterpreterSetting.java         |    22 +-
 .../interpreter/InterpreterSettingManager.java  |     5 +-
 .../interpreter/ManagedInterpreterGroup.java    |     4 -
 .../RemoteInterpreterEventServer.java           |   496 +
 .../launcher/ShellScriptLauncher.java           |     5 +-
 .../recovery/FileSystemRecoveryStorage.java     |     6 -
 .../remote/RemoteAngularObjectRegistry.java     |     1 +
 .../interpreter/remote/RemoteInterpreter.java   |    17 +-
 .../remote/RemoteInterpreterEventPoller.java    |   528 -
 .../remote/RemoteInterpreterManagedProcess.java |    96 +-
 .../remote/RemoteInterpreterProcess.java        |    15 -
 .../RemoteInterpreterProcessListener.java       |    15 +-
 .../java/org/apache/zeppelin/notebook/Note.java |     4 +
 .../org/apache/zeppelin/notebook/Paragraph.java |    35 +-
 .../interpreter/AbstractInterpreterTest.java    |     7 +
 .../interpreter/ConfInterpreterTest.java        |    29 +-
 .../InterpreterSettingManagerTest.java          |     4 +-
 .../launcher/ShellScriptLauncherTest.java       |     2 +-
 .../launcher/SparkInterpreterLauncherTest.java  |    10 +-
 .../lifecycle/TimeoutLifecycleManagerTest.java  |    19 +-
 .../recovery/FileSystemRecoveryStorageTest.java |    26 +-
 .../remote/AppendOutputRunnerTest.java          |     1 -
 .../remote/RemoteAngularObjectTest.java         |    66 +-
 .../RemoteInterpreterEventPollerTest.java       |    55 -
 .../RemoteInterpreterOutputTestStream.java      |    73 +-
 .../remote/RemoteInterpreterTest.java           |    85 +-
 .../resource/DistributedResourcePoolTest.java   |    36 +-
 .../zeppelin/scheduler/RemoteSchedulerTest.java |   105 +-
 .../interpreter/test/interpreter-setting.json   |    70 +
 121 files changed, 23746 insertions(+), 8889 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/bin/interpreter.sh
----------------------------------------------------------------------
diff --git a/bin/interpreter.sh b/bin/interpreter.sh
index a8375af..64b3af4 100755
--- a/bin/interpreter.sh
+++ b/bin/interpreter.sh
@@ -24,7 +24,7 @@ function usage() {
     echo "usage) $0 -p <port> -r <intp_port> -d <interpreter dir to load> -l <local interpreter repo dir to load> -g <interpreter group name>"
 }
 
-while getopts "hc:p:r:d:l:v:u:g:" o; do
+while getopts "hc:p:r:i:d:l:v:u:g:" o; do
     case ${o} in
         h)
             usage
@@ -42,6 +42,9 @@ while getopts "hc:p:r:d:l:v:u:g:" o; do
         r)
             INTP_PORT=${OPTARG} # This will be used for interpreter process port
             ;;
+        i)
+            INTP_GROUP_ID=${OPTARG} # This will be used for interpreter group id
+            ;;
         l)
             LOCAL_INTERPRETER_REPO=${OPTARG}
             ;;
@@ -218,9 +221,9 @@ if [[ ! -z "$ZEPPELIN_IMPERSONATE_USER" ]]; then
 fi
 
 if [[ -n "${SPARK_SUBMIT}" ]]; then
-    INTERPRETER_RUN_COMMAND+=' '` echo ${SPARK_SUBMIT} --class ${ZEPPELIN_SERVER} --driver-class-path \"${ZEPPELIN_INTP_CLASSPATH_OVERRIDES}:${ZEPPELIN_INTP_CLASSPATH}\" --driver-java-options \"${JAVA_INTP_OPTS}\" ${SPARK_SUBMIT_OPTIONS} ${ZEPPELIN_SPARK_CONF} ${SPARK_APP_JAR} ${CALLBACK_HOST} ${PORT} ${INTP_PORT}`
+    INTERPRETER_RUN_COMMAND+=' '` echo ${SPARK_SUBMIT} --class ${ZEPPELIN_SERVER} --driver-class-path \"${ZEPPELIN_INTP_CLASSPATH_OVERRIDES}:${ZEPPELIN_INTP_CLASSPATH}\" --driver-java-options \"${JAVA_INTP_OPTS}\" ${SPARK_SUBMIT_OPTIONS} ${ZEPPELIN_SPARK_CONF} ${SPARK_APP_JAR} ${CALLBACK_HOST} ${PORT} ${INTP_GROUP_ID} ${INTP_PORT}`
 else
-    INTERPRETER_RUN_COMMAND+=' '` echo ${ZEPPELIN_RUNNER} ${JAVA_INTP_OPTS} ${ZEPPELIN_INTP_MEM} -cp ${ZEPPELIN_INTP_CLASSPATH_OVERRIDES}:${ZEPPELIN_INTP_CLASSPATH} ${ZEPPELIN_SERVER} ${CALLBACK_HOST} ${PORT} ${INTP_PORT}`
+    INTERPRETER_RUN_COMMAND+=' '` echo ${ZEPPELIN_RUNNER} ${JAVA_INTP_OPTS} ${ZEPPELIN_INTP_MEM} -cp ${ZEPPELIN_INTP_CLASSPATH_OVERRIDES}:${ZEPPELIN_INTP_CLASSPATH} ${ZEPPELIN_SERVER} ${CALLBACK_HOST} ${PORT} ${INTP_GROUP_ID} ${INTP_PORT}`
 fi
 
 

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/elasticsearch/src/test/java/org/apache/zeppelin/elasticsearch/ElasticsearchInterpreterTest.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/test/java/org/apache/zeppelin/elasticsearch/ElasticsearchInterpreterTest.java b/elasticsearch/src/test/java/org/apache/zeppelin/elasticsearch/ElasticsearchInterpreterTest.java
index e186b44..4a412aa 100644
--- a/elasticsearch/src/test/java/org/apache/zeppelin/elasticsearch/ElasticsearchInterpreterTest.java
+++ b/elasticsearch/src/test/java/org/apache/zeppelin/elasticsearch/ElasticsearchInterpreterTest.java
@@ -161,9 +161,11 @@ public class ElasticsearchInterpreterTest {
   }
 
   private InterpreterContext buildContext(String noteAndParagraphId) {
-    final AngularObjectRegistry angularObjReg = new AngularObjectRegistry("elasticsearch", null);
-    return new InterpreterContext(noteAndParagraphId, noteAndParagraphId, null, null, null, null,
-        null, null, null, angularObjReg , null, null, null);
+    return InterpreterContext.builder()
+        .setNoteId(noteAndParagraphId)
+        .setParagraphId(noteAndParagraphId)
+        .setAngularObjectRegistry(new AngularObjectRegistry("elasticsearch", null))
+        .build();
   }
 
   @Theory

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/flink/src/test/java/org/apache/zeppelin/flink/FlinkInterpreterTest.java
----------------------------------------------------------------------
diff --git a/flink/src/test/java/org/apache/zeppelin/flink/FlinkInterpreterTest.java b/flink/src/test/java/org/apache/zeppelin/flink/FlinkInterpreterTest.java
index a95db39..128f567 100644
--- a/flink/src/test/java/org/apache/zeppelin/flink/FlinkInterpreterTest.java
+++ b/flink/src/test/java/org/apache/zeppelin/flink/FlinkInterpreterTest.java
@@ -17,9 +17,9 @@
  */
 package org.apache.zeppelin.flink;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-
+import org.apache.zeppelin.interpreter.InterpreterContext;
+import org.apache.zeppelin.interpreter.InterpreterResult;
+import org.apache.zeppelin.interpreter.InterpreterResult.Code;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -27,9 +27,8 @@ import org.junit.Test;
 import java.util.Arrays;
 import java.util.Properties;
 
-import org.apache.zeppelin.interpreter.InterpreterContext;
-import org.apache.zeppelin.interpreter.InterpreterResult;
-import org.apache.zeppelin.interpreter.InterpreterResult.Code;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
 
 public class FlinkInterpreterTest {
 
@@ -41,8 +40,7 @@ public class FlinkInterpreterTest {
     Properties p = new Properties();
     flink = new FlinkInterpreter(p);
     flink.open();
-    context = new InterpreterContext(null, null, null, null, null, null, null, null, null, null,
-            null, null, null);
+    context = InterpreterContext.builder().build();
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/groovy/src/main/java/org/apache/zeppelin/groovy/GObject.java
----------------------------------------------------------------------
diff --git a/groovy/src/main/java/org/apache/zeppelin/groovy/GObject.java b/groovy/src/main/java/org/apache/zeppelin/groovy/GObject.java
index 9948b89..e7f27c3 100644
--- a/groovy/src/main/java/org/apache/zeppelin/groovy/GObject.java
+++ b/groovy/src/main/java/org/apache/zeppelin/groovy/GObject.java
@@ -16,26 +16,23 @@
  */
 package org.apache.zeppelin.groovy;
 
-import org.slf4j.Logger;
-
-import java.io.StringWriter;
-import java.util.Collection;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
 import groovy.lang.Closure;
 import groovy.xml.MarkupBuilder;
-
+import org.apache.thrift.TException;
 import org.apache.zeppelin.annotation.ZeppelinApi;
 import org.apache.zeppelin.display.AngularObject;
 import org.apache.zeppelin.display.AngularObjectRegistry;
 import org.apache.zeppelin.display.GUI;
 import org.apache.zeppelin.display.ui.OptionInput.ParamOption;
 import org.apache.zeppelin.interpreter.InterpreterContext;
-import org.apache.zeppelin.interpreter.InterpreterContextRunner;
-import org.apache.zeppelin.interpreter.RemoteWorksController;
+import org.slf4j.Logger;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Properties;
 
 /**
  * Groovy interpreter for Zeppelin.
@@ -46,6 +43,7 @@ public class GObject extends groovy.lang.GroovyObjectSupport {
   Properties props;
   InterpreterContext interpreterContext;
   Map<String, Object> bindings;
+  GroovyZeppelinContext z;
 
   public GObject(Logger log, StringWriter out, Properties p, InterpreterContext ctx,
       Map<String, Object> bindings) {
@@ -54,6 +52,8 @@ public class GObject extends groovy.lang.GroovyObjectSupport {
     this.interpreterContext = ctx;
     this.props = p;
     this.bindings = bindings;
+    this.z = new GroovyZeppelinContext(null, 1000);
+    this.z.setInterpreterContext(this.interpreterContext);
   }
 
   public Object getProperty(String key) {
@@ -87,17 +87,17 @@ public class GObject extends groovy.lang.GroovyObjectSupport {
    * returns gui object.
    */
   public GUI getGui() {
-    return interpreterContext.getGui();
+    return z.getGui();
   }
 
   @ZeppelinApi
   public Object input(String name) {
-    return input(name, "");
+    return z.input(name, "");
   }
 
   @ZeppelinApi
   public Object input(String name, Object defaultValue) {
-    return getGui().input(name, defaultValue);
+    return z.input(name, defaultValue);
   }
 
   private ParamOption[] toParamOptions(Map<Object, String> options) {
@@ -111,23 +111,23 @@ public class GObject extends groovy.lang.GroovyObjectSupport {
 
   @ZeppelinApi
   public Object select(String name, Map<Object, String> options) {
-    return select(name, "", options);
+    return z.select(name, "", toParamOptions(options));
   }
 
   @ZeppelinApi
   public Object select(String name, Object defaultValue, Map<Object, String> options) {
-    return getGui().select(name, defaultValue, toParamOptions(options));
+    return z.select(name, defaultValue, toParamOptions(options));
   }
 
   @ZeppelinApi
   public Collection<Object> checkbox(String name, Map<Object, String> options) {
-    return checkbox(name, options.keySet(), options);
+    return z.checkbox(name, new ArrayList<Object>(options.keySet()), toParamOptions(options));
   }
 
   @ZeppelinApi
   public Collection<Object> checkbox(String name, Collection<Object> defaultChecked,
       Map<Object, String> options) {
-    return getGui().checkbox(name, defaultChecked, toParamOptions(options));
+    return z.checkbox(name, new ArrayList<Object>(defaultChecked), toParamOptions(options));
   }
 
 
@@ -236,23 +236,12 @@ public class GObject extends groovy.lang.GroovyObjectSupport {
    * @return value
    */
   public Object angular(String name) {
-    AngularObject ao = getAngularObject(name);
-    if (ao == null) {
-      return null;
-    } else {
-      return ao.get();
-    }
+    return z.angular(name);
   }
 
   @SuppressWarnings("unchecked")
-  public void angularBind(String name, Object o, String noteId) {
-    AngularObjectRegistry registry = interpreterContext.getAngularObjectRegistry();
-
-    if (registry.get(name, noteId, null) == null) {
-      registry.add(name, o, noteId, null);
-    } else {
-      registry.get(name, noteId, null).set(o);
-    }
+  public void angularBind(String name, Object o, String noteId) throws TException {
+    z.angularBind(name, o, noteId);
   }
 
   /**
@@ -262,100 +251,56 @@ public class GObject extends groovy.lang.GroovyObjectSupport {
    * @param name name of the variable
    * @param o value
    */
-  public void angularBind(String name, Object o) {
+  public void angularBind(String name, Object o) throws TException {
     angularBind(name, o, interpreterContext.getNoteId());
   }
 
-  /*------------------------------------------RUN----------------------------------------*/
-  @ZeppelinApi
-  public List<InterpreterContextRunner> getInterpreterContextRunner(String noteId,
-      String paragraphId, InterpreterContext interpreterContext) {
-    RemoteWorksController remoteWorksController = interpreterContext.getRemoteWorksController();
-    if (remoteWorksController != null) {
-      return remoteWorksController.getRemoteContextRunner(noteId, paragraphId);
-    }
-    return new LinkedList<InterpreterContextRunner>();
-  }
-
-  @ZeppelinApi
-  public List<InterpreterContextRunner> getInterpreterContextRunner(String noteId,
-      InterpreterContext interpreterContext) {
-    RemoteWorksController remoteWorksController = interpreterContext.getRemoteWorksController();
-    if (remoteWorksController != null) {
-      return remoteWorksController.getRemoteContextRunner(noteId);
-    }
-    return new LinkedList<InterpreterContextRunner>();
-  }
-
   /**
    * Run paragraph by id.
    */
   @ZeppelinApi
-  public void run(String noteId, String paragraphId) {
-    run(noteId, paragraphId, interpreterContext);
+  public void run(String noteId, String paragraphId) throws IOException {
+    z.run(noteId, paragraphId);
   }
 
   /**
    * Run paragraph by id.
    */
   @ZeppelinApi
-  public void run(String paragraphId) {
-    String noteId = interpreterContext.getNoteId();
-    run(noteId, paragraphId, interpreterContext);
+  public void run(String paragraphId) throws IOException {
+    z.run(paragraphId);
   }
 
   /**
    * Run paragraph by id.
    */
   @ZeppelinApi
-  public void run(String noteId, String paragraphId, InterpreterContext context) {
-    if (paragraphId.equals(context.getParagraphId())) {
-      throw new RuntimeException("Can not run current Paragraph");
-    }
-    List<InterpreterContextRunner> runners = getInterpreterContextRunner(noteId, paragraphId,
-        context);
-    if (runners.size() <= 0) {
-      throw new RuntimeException("Paragraph " + paragraphId + " not found " + runners.size());
-    }
-    for (InterpreterContextRunner r : runners) {
-      r.run();
-    }
+  public void run(String noteId, String paragraphId, InterpreterContext context)
+      throws IOException {
+    z.run(noteId, paragraphId, context);
   }
 
-  public void runNote(String noteId) {
-    runNote(noteId, interpreterContext);
+  public void runNote(String noteId) throws IOException {
+    z.runNote(noteId);
   }
 
-  public void runNote(String noteId, InterpreterContext context) {
-    String runningNoteId = context.getNoteId();
-    String runningParagraphId = context.getParagraphId();
-    List<InterpreterContextRunner> runners = getInterpreterContextRunner(noteId, context);
-
-    if (runners.size() <= 0) {
-      throw new RuntimeException("Note " + noteId + " not found " + runners.size());
-    }
-
-    for (InterpreterContextRunner r : runners) {
-      if (r.getNoteId().equals(runningNoteId) && r.getParagraphId().equals(runningParagraphId)) {
-        continue;
-      }
-      r.run();
-    }
+  public void runNote(String noteId, InterpreterContext context) throws IOException {
+    z.runNote(noteId, context);
   }
 
   /**
    * Run all paragraphs. except this.
    */
   @ZeppelinApi
-  public void runAll() {
-    runAll(interpreterContext);
+  public void runAll() throws IOException {
+    z.runAll(interpreterContext);
   }
 
   /**
    * Run all paragraphs. except this.
    */
   @ZeppelinApi
-  public void runAll(InterpreterContext context) {
-    runNote(context.getNoteId());
+  public void runAll(InterpreterContext context) throws IOException {
+    z.runNote(context.getNoteId());
   }
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/groovy/src/main/java/org/apache/zeppelin/groovy/GroovyZeppelinContext.java
----------------------------------------------------------------------
diff --git a/groovy/src/main/java/org/apache/zeppelin/groovy/GroovyZeppelinContext.java b/groovy/src/main/java/org/apache/zeppelin/groovy/GroovyZeppelinContext.java
new file mode 100644
index 0000000..3d17462
--- /dev/null
+++ b/groovy/src/main/java/org/apache/zeppelin/groovy/GroovyZeppelinContext.java
@@ -0,0 +1,49 @@
+/*
+ * 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.groovy;
+
+import org.apache.zeppelin.interpreter.BaseZeppelinContext;
+import org.apache.zeppelin.interpreter.InterpreterHookRegistry;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * ZeppelinContext for Groovy
+ */
+public class GroovyZeppelinContext extends BaseZeppelinContext {
+
+  public GroovyZeppelinContext(InterpreterHookRegistry hooks, int maxResult) {
+    super(hooks, maxResult);
+  }
+
+  @Override
+  public Map<String, String> getInterpreterClassMap() {
+    return null;
+  }
+
+  @Override
+  public List<Class> getSupportedClasses() {
+    return null;
+  }
+
+  @Override
+  protected String showData(Object obj) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/helium-dev/src/main/java/org/apache/zeppelin/helium/DevInterpreter.java
----------------------------------------------------------------------
diff --git a/helium-dev/src/main/java/org/apache/zeppelin/helium/DevInterpreter.java b/helium-dev/src/main/java/org/apache/zeppelin/helium/DevInterpreter.java
index ba1a564..00846de 100644
--- a/helium-dev/src/main/java/org/apache/zeppelin/helium/DevInterpreter.java
+++ b/helium-dev/src/main/java/org/apache/zeppelin/helium/DevInterpreter.java
@@ -17,18 +17,17 @@
 
 package org.apache.zeppelin.helium;
 
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Properties;
-
 import org.apache.zeppelin.interpreter.Interpreter;
 import org.apache.zeppelin.interpreter.InterpreterContext;
-import org.apache.zeppelin.interpreter.InterpreterContextRunner;
 import org.apache.zeppelin.interpreter.InterpreterException;
-import org.apache.zeppelin.interpreter.InterpreterPropertyBuilder;
 import org.apache.zeppelin.interpreter.InterpreterResult;
 import org.apache.zeppelin.interpreter.thrift.InterpreterCompletion;
 
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Properties;
+
 /**
  * Dummy interpreter to support development mode for Zeppelin app
  */
@@ -36,6 +35,7 @@ public class DevInterpreter extends Interpreter {
 
   private InterpreterEvent interpreterEvent;
   private InterpreterContext context;
+  private DevZeppelinContext z;
 
   public static boolean isInterpreterName(String replName) {
     return replName.equals("dev");
@@ -59,6 +59,7 @@ public class DevInterpreter extends Interpreter {
 
   @Override
   public void open() {
+    this.z = new DevZeppelinContext(null, 1000);
   }
 
   @Override
@@ -66,10 +67,10 @@ public class DevInterpreter extends Interpreter {
   }
 
   public void rerun() {
-    for (InterpreterContextRunner r : context.getRunners()) {
-      if (context.getParagraphId().equals(r.getParagraphId())) {
-        r.run();
-      }
+    try {
+      z.run(context.getParagraphId());
+    } catch (IOException e) {
+      throw new RuntimeException("Fail to rerun", e);
     }
   }
 
@@ -77,6 +78,7 @@ public class DevInterpreter extends Interpreter {
   public InterpreterResult interpret(String st, InterpreterContext context)
       throws InterpreterException {
     this.context = context;
+    this.z.setInterpreterContext(context);
     try {
       return interpreterEvent.interpret(st, context);
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/helium-dev/src/main/java/org/apache/zeppelin/helium/DevZeppelinContext.java
----------------------------------------------------------------------
diff --git a/helium-dev/src/main/java/org/apache/zeppelin/helium/DevZeppelinContext.java b/helium-dev/src/main/java/org/apache/zeppelin/helium/DevZeppelinContext.java
new file mode 100644
index 0000000..75d193c
--- /dev/null
+++ b/helium-dev/src/main/java/org/apache/zeppelin/helium/DevZeppelinContext.java
@@ -0,0 +1,49 @@
+/*
+ * 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.helium;
+
+import org.apache.zeppelin.interpreter.BaseZeppelinContext;
+import org.apache.zeppelin.interpreter.InterpreterHookRegistry;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * ZeppelinContext for DevInterpreter
+ */
+public class DevZeppelinContext extends BaseZeppelinContext {
+  public DevZeppelinContext(InterpreterHookRegistry hooks, int maxResult) {
+    super(hooks, maxResult);
+  }
+
+  @Override
+  public Map<String, String> getInterpreterClassMap() {
+    return null;
+  }
+
+  @Override
+  public List<Class> getSupportedClasses() {
+    return null;
+  }
+
+  @Override
+  protected String showData(Object obj) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/helium-dev/src/main/java/org/apache/zeppelin/helium/ZeppelinApplicationDevServer.java
----------------------------------------------------------------------
diff --git a/helium-dev/src/main/java/org/apache/zeppelin/helium/ZeppelinApplicationDevServer.java b/helium-dev/src/main/java/org/apache/zeppelin/helium/ZeppelinApplicationDevServer.java
index 6b6d7cc..d0118e5 100644
--- a/helium-dev/src/main/java/org/apache/zeppelin/helium/ZeppelinApplicationDevServer.java
+++ b/helium-dev/src/main/java/org/apache/zeppelin/helium/ZeppelinApplicationDevServer.java
@@ -138,7 +138,7 @@ public class ZeppelinApplicationDevServer extends ZeppelinDevServer {
   protected InterpreterOutput createInterpreterOutput(
       final String noteId, final String paragraphId) {
     if (out == null) {
-      final RemoteInterpreterEventClient eventClient = getEventClient();
+      final RemoteInterpreterEventClient eventClient = getIntpEventClient();
       try {
         out = new InterpreterOutput(new InterpreterOutputListener() {
           @Override

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/helium-dev/src/main/java/org/apache/zeppelin/helium/ZeppelinDevServer.java
----------------------------------------------------------------------
diff --git a/helium-dev/src/main/java/org/apache/zeppelin/helium/ZeppelinDevServer.java b/helium-dev/src/main/java/org/apache/zeppelin/helium/ZeppelinDevServer.java
index 607839e..4c0e867 100644
--- a/helium-dev/src/main/java/org/apache/zeppelin/helium/ZeppelinDevServer.java
+++ b/helium-dev/src/main/java/org/apache/zeppelin/helium/ZeppelinDevServer.java
@@ -39,7 +39,7 @@ public class ZeppelinDevServer extends
   private DevInterpreter interpreter = null;
   private InterpreterOutput out;
   public ZeppelinDevServer(int port) throws TException, IOException {
-    super(null, port, ":");
+    super(null, port, null, ":");
   }
 
   @Override
@@ -68,7 +68,7 @@ public class ZeppelinDevServer extends
   protected InterpreterOutput createInterpreterOutput(
       final String noteId, final String paragraphId) {
     if (out == null) {
-      final RemoteInterpreterEventClient eventClient = getEventClient();
+      final RemoteInterpreterEventClient eventClient = getIntpEventClient();
       try {
         out = new InterpreterOutput(new InterpreterOutputListener() {
           @Override

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/ignite/src/test/java/org/apache/zeppelin/ignite/IgniteInterpreterTest.java
----------------------------------------------------------------------
diff --git a/ignite/src/test/java/org/apache/zeppelin/ignite/IgniteInterpreterTest.java b/ignite/src/test/java/org/apache/zeppelin/ignite/IgniteInterpreterTest.java
index 1f6c49c..de4588e 100644
--- a/ignite/src/test/java/org/apache/zeppelin/ignite/IgniteInterpreterTest.java
+++ b/ignite/src/test/java/org/apache/zeppelin/ignite/IgniteInterpreterTest.java
@@ -16,14 +16,13 @@
  */
 package org.apache.zeppelin.ignite;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 import org.apache.ignite.Ignite;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.zeppelin.interpreter.InterpreterContext;
+import org.apache.zeppelin.interpreter.InterpreterResult;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -31,8 +30,8 @@ import org.junit.Test;
 import java.util.Collections;
 import java.util.Properties;
 
-import org.apache.zeppelin.interpreter.InterpreterContext;
-import org.apache.zeppelin.interpreter.InterpreterResult;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Tests for Apache Ignite interpreter ({@link IgniteInterpreter}).
@@ -40,9 +39,7 @@ import org.apache.zeppelin.interpreter.InterpreterResult;
 public class IgniteInterpreterTest {
   private static final String HOST = "127.0.0.1:47500..47509";
 
-  private static final InterpreterContext INTP_CONTEXT =
-      new InterpreterContext(null, null, null, null, null, null, null, null, null, null,
-              null, null, null);
+  private static final InterpreterContext INTP_CONTEXT = InterpreterContext.builder().build();
 
   private IgniteInterpreter intp;
   private Ignite ignite;

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/ignite/src/test/java/org/apache/zeppelin/ignite/IgniteSqlInterpreterTest.java
----------------------------------------------------------------------
diff --git a/ignite/src/test/java/org/apache/zeppelin/ignite/IgniteSqlInterpreterTest.java b/ignite/src/test/java/org/apache/zeppelin/ignite/IgniteSqlInterpreterTest.java
index 5006fa3..3597769 100644
--- a/ignite/src/test/java/org/apache/zeppelin/ignite/IgniteSqlInterpreterTest.java
+++ b/ignite/src/test/java/org/apache/zeppelin/ignite/IgniteSqlInterpreterTest.java
@@ -16,8 +16,6 @@
  */
 package org.apache.zeppelin.ignite;
 
-import static org.junit.Assert.assertEquals;
-
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
@@ -25,6 +23,11 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.zeppelin.interpreter.InterpreterContext;
+import org.apache.zeppelin.interpreter.InterpreterException;
+import org.apache.zeppelin.interpreter.InterpreterResult;
+import org.apache.zeppelin.interpreter.InterpreterResult.Code;
+import org.apache.zeppelin.interpreter.InterpreterResult.Type;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -32,11 +35,7 @@ import org.junit.Test;
 import java.util.Collections;
 import java.util.Properties;
 
-import org.apache.zeppelin.interpreter.InterpreterContext;
-import org.apache.zeppelin.interpreter.InterpreterException;
-import org.apache.zeppelin.interpreter.InterpreterResult;
-import org.apache.zeppelin.interpreter.InterpreterResult.Code;
-import org.apache.zeppelin.interpreter.InterpreterResult.Type;
+import static org.junit.Assert.assertEquals;
 
 /**
  * Tests for Apache Ignite SQL interpreter ({@link IgniteSqlInterpreter}).
@@ -44,9 +43,7 @@ import org.apache.zeppelin.interpreter.InterpreterResult.Type;
 public class IgniteSqlInterpreterTest {
   private static final String HOST = "127.0.0.1:47500..47509";
 
-  private static final InterpreterContext INTP_CONTEXT =
-      new InterpreterContext(null, null, null, null, null, null, null, null, null, null, null,
-              null, null);
+  private static final InterpreterContext INTP_CONTEXT = InterpreterContext.builder().build();
 
   private Ignite ignite;
   private IgniteSqlInterpreter intp;

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/jdbc/src/test/java/org/apache/zeppelin/jdbc/JDBCInterpreterInterpolationTest.java
----------------------------------------------------------------------
diff --git a/jdbc/src/test/java/org/apache/zeppelin/jdbc/JDBCInterpreterInterpolationTest.java b/jdbc/src/test/java/org/apache/zeppelin/jdbc/JDBCInterpreterInterpolationTest.java
index fe7bc80..d55f9fe 100644
--- a/jdbc/src/test/java/org/apache/zeppelin/jdbc/JDBCInterpreterInterpolationTest.java
+++ b/jdbc/src/test/java/org/apache/zeppelin/jdbc/JDBCInterpreterInterpolationTest.java
@@ -69,8 +69,11 @@ public class JDBCInterpreterInterpolationTest extends BasicJDBCTestCaseAdapter {
                 "('mou', 'mouse');");
     resourcePool = new LocalResourcePool("JdbcInterpolationTest");
 
-    interpreterContext = new InterpreterContext("", "1", null, "", "",
-            new AuthenticationInfo("testUser"), null, null, null, null, resourcePool, null, null);
+    interpreterContext = InterpreterContext.builder()
+        .setParagraphId("paragraph_1")
+        .setAuthenticationInfo(new AuthenticationInfo("testUser"))
+        .setResourcePool(resourcePool)
+        .build();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/jdbc/src/test/java/org/apache/zeppelin/jdbc/JDBCInterpreterTest.java
----------------------------------------------------------------------
diff --git a/jdbc/src/test/java/org/apache/zeppelin/jdbc/JDBCInterpreterTest.java b/jdbc/src/test/java/org/apache/zeppelin/jdbc/JDBCInterpreterTest.java
index 65e8d8f..5ce8d6e 100644
--- a/jdbc/src/test/java/org/apache/zeppelin/jdbc/JDBCInterpreterTest.java
+++ b/jdbc/src/test/java/org/apache/zeppelin/jdbc/JDBCInterpreterTest.java
@@ -100,8 +100,9 @@ public class JDBCInterpreterTest extends BasicJDBCTestCaseAdapter {
             "insert into test_table(id, name) values ('a', 'a_name'),('b', 'b_name'),('c', ?);");
     insertStatement.setString(1, null);
     insertStatement.execute();
-    interpreterContext = new InterpreterContext("", "1", null, "", "",
-        new AuthenticationInfo("testUser"), null, null, null, null, null, null, null);
+    interpreterContext = InterpreterContext.builder()
+        .setAuthenticationInfo(new AuthenticationInfo("testUser"))
+        .build();
   }
 
 
@@ -433,8 +434,10 @@ public class JDBCInterpreterTest extends BasicJDBCTestCaseAdapter {
 
     // user1 runs jdbc1
     jdbc1.open();
-    InterpreterContext ctx1 = new InterpreterContext("", "1", "jdbc1", "", "", user1Credential,
-            null, null, null, null, null, null, null);
+    InterpreterContext ctx1 = InterpreterContext.builder()
+        .setAuthenticationInfo(user1Credential)
+        .setReplName("jdbc1")
+        .build();
     jdbc1.interpret("", ctx1);
 
     JDBCUserConfigurations user1JDBC1Conf = jdbc1.getJDBCConfiguration("user1");
@@ -444,8 +447,10 @@ public class JDBCInterpreterTest extends BasicJDBCTestCaseAdapter {
 
     // user1 runs jdbc2
     jdbc2.open();
-    InterpreterContext ctx2 = new InterpreterContext("", "1", "jdbc2", "", "", user1Credential,
-            null, null, null, null, null, null, null);
+    InterpreterContext ctx2 = InterpreterContext.builder()
+        .setAuthenticationInfo(user1Credential)
+        .setReplName("jdbc2")
+        .build();
     jdbc2.interpret("", ctx2);
 
     JDBCUserConfigurations user1JDBC2Conf = jdbc2.getJDBCConfiguration("user1");
@@ -455,8 +460,10 @@ public class JDBCInterpreterTest extends BasicJDBCTestCaseAdapter {
 
     // user2 runs jdbc1
     jdbc1.open();
-    InterpreterContext ctx3 = new InterpreterContext("", "1", "jdbc1", "", "", user2Credential,
-            null, null, null, null, null, null, null);
+    InterpreterContext ctx3 = InterpreterContext.builder()
+        .setAuthenticationInfo(user2Credential)
+        .setReplName("jdbc1")
+        .build();
     jdbc1.interpret("", ctx3);
 
     JDBCUserConfigurations user2JDBC1Conf = jdbc1.getJDBCConfiguration("user2");
@@ -466,8 +473,10 @@ public class JDBCInterpreterTest extends BasicJDBCTestCaseAdapter {
 
     // user2 runs jdbc2
     jdbc2.open();
-    InterpreterContext ctx4 = new InterpreterContext("", "1", "jdbc2", "", "", user2Credential,
-            null, null, null, null, null, null, null);
+    InterpreterContext ctx4 = InterpreterContext.builder()
+        .setAuthenticationInfo(user2Credential)
+        .setReplName("jdbc2")
+        .build();
     jdbc2.interpret("", ctx4);
 
     JDBCUserConfigurations user2JDBC2Conf = jdbc2.getJDBCConfiguration("user2");

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/livy/src/main/java/org/apache/zeppelin/livy/LivySparkSQLInterpreter.java
----------------------------------------------------------------------
diff --git a/livy/src/main/java/org/apache/zeppelin/livy/LivySparkSQLInterpreter.java b/livy/src/main/java/org/apache/zeppelin/livy/LivySparkSQLInterpreter.java
index 902336b..ac5c0f5 100644
--- a/livy/src/main/java/org/apache/zeppelin/livy/LivySparkSQLInterpreter.java
+++ b/livy/src/main/java/org/apache/zeppelin/livy/LivySparkSQLInterpreter.java
@@ -17,16 +17,7 @@
 
 package org.apache.zeppelin.livy;
 
-import static org.apache.commons.lang.StringEscapeUtils.escapeJavaScript;
-
 import org.apache.commons.lang.StringUtils;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Properties;
-
-import org.apache.zeppelin.display.GUI;
 import org.apache.zeppelin.interpreter.Interpreter;
 import org.apache.zeppelin.interpreter.InterpreterContext;
 import org.apache.zeppelin.interpreter.InterpreterException;
@@ -39,7 +30,12 @@ import org.apache.zeppelin.interpreter.ResultMessages;
 import org.apache.zeppelin.interpreter.WrappedInterpreter;
 import org.apache.zeppelin.scheduler.Scheduler;
 import org.apache.zeppelin.scheduler.SchedulerFactory;
-import org.apache.zeppelin.user.AuthenticationInfo;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+import static org.apache.commons.lang.StringEscapeUtils.escapeJavaScript;
 
 /**
  * Livy SparkSQL Interpreter for Zeppelin.
@@ -78,20 +74,9 @@ public class LivySparkSQLInterpreter extends BaseLivyInterpreter {
     // As we don't know whether livyserver use spark2 or spark1, so we will detect SparkSession
     // to judge whether it is using spark2.
     try {
-      InterpreterContext context = new InterpreterContext(
-          "noteId",
-          "paragraphId",
-          "replName",
-          "paragraphTitle",
-          "paragraphText",
-          new AuthenticationInfo(),
-          new HashMap<String, Object>(),
-          new GUI(),
-          new GUI(),
-          null,
-          null,
-          null,
-          new InterpreterOutput(null));
+      InterpreterContext context = InterpreterContext.builder()
+          .setInterpreterOut(new InterpreterOutput(null))
+          .build();
       InterpreterResult result = sparkInterpreter.interpret("spark", context);
       if (result.code() == InterpreterResult.Code.SUCCESS &&
           result.message().get(0).getData().contains("org.apache.spark.sql.SparkSession")) {

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/livy/src/test/java/org/apache/zeppelin/livy/LivyInterpreterIT.java
----------------------------------------------------------------------
diff --git a/livy/src/test/java/org/apache/zeppelin/livy/LivyInterpreterIT.java b/livy/src/test/java/org/apache/zeppelin/livy/LivyInterpreterIT.java
index 76e2103..4357bb8 100644
--- a/livy/src/test/java/org/apache/zeppelin/livy/LivyInterpreterIT.java
+++ b/livy/src/test/java/org/apache/zeppelin/livy/LivyInterpreterIT.java
@@ -17,23 +17,9 @@
 
 package org.apache.zeppelin.livy;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-
 import org.apache.commons.io.IOUtils;
 import org.apache.livy.test.framework.Cluster;
 import org.apache.livy.test.framework.Cluster$;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Properties;
-
 import org.apache.zeppelin.interpreter.Interpreter;
 import org.apache.zeppelin.interpreter.InterpreterContext;
 import org.apache.zeppelin.interpreter.InterpreterException;
@@ -44,6 +30,19 @@ import org.apache.zeppelin.interpreter.InterpreterResult;
 import org.apache.zeppelin.interpreter.InterpreterResultMessageOutput;
 import org.apache.zeppelin.interpreter.LazyOpenInterpreter;
 import org.apache.zeppelin.user.AuthenticationInfo;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Properties;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
 
 public class LivyInterpreterIT {
   private static final Logger LOGGER = LoggerFactory.getLogger(LivyInterpreterIT.class);
@@ -98,8 +97,12 @@ public class LivyInterpreterIT {
     AuthenticationInfo authInfo = new AuthenticationInfo("user1");
     MyInterpreterOutputListener outputListener = new MyInterpreterOutputListener();
     InterpreterOutput output = new InterpreterOutput(outputListener);
-    InterpreterContext context = new InterpreterContext("noteId", "paragraphId", "livy.spark",
-        "title", "text", authInfo, null, null, null, null, null, null, output);
+    InterpreterContext context = InterpreterContext.builder()
+        .setNoteId("noteId")
+        .setParagraphId("paragraphId")
+        .setAuthenticationInfo(authInfo)
+        .setInterpreterOut(output)
+        .build();
     sparkInterpreter.open();
 
     LivySparkSQLInterpreter sqlInterpreter = new LivySparkSQLInterpreter(properties);
@@ -127,8 +130,13 @@ public class LivyInterpreterIT {
     AuthenticationInfo authInfo = new AuthenticationInfo("user1");
     MyInterpreterOutputListener outputListener = new MyInterpreterOutputListener();
     InterpreterOutput output = new InterpreterOutput(outputListener);
-    final InterpreterContext context = new InterpreterContext("noteId", "paragraphId", "livy.spark",
-        "title", "text", authInfo, null, null, null, null, null, null, output);
+    final InterpreterContext context = InterpreterContext.builder()
+        .setNoteId("noteId")
+        .setParagraphId("paragraphId")
+        .setAuthenticationInfo(authInfo)
+        .setInterpreterOut(output)
+        .build();
+    ;
 
     InterpreterResult result = sparkInterpreter.interpret("sc.parallelize(1 to 10).sum()", context);
     assertEquals(InterpreterResult.Code.SUCCESS, result.code());
@@ -226,8 +234,12 @@ public class LivyInterpreterIT {
     AuthenticationInfo authInfo = new AuthenticationInfo("user1");
     MyInterpreterOutputListener outputListener = new MyInterpreterOutputListener();
     InterpreterOutput output = new InterpreterOutput(outputListener);
-    final InterpreterContext context = new InterpreterContext("noteId", "paragraphId", "livy.spark",
-        "title", "text", authInfo, null, null, null, null, null, null, output);
+    final InterpreterContext context = InterpreterContext.builder()
+        .setNoteId("noteId")
+        .setParagraphId("paragraphId")
+        .setAuthenticationInfo(authInfo)
+        .setInterpreterOut(output)
+        .build();
 
     InterpreterResult result = null;
     // test DataFrame api
@@ -312,8 +324,8 @@ public class LivyInterpreterIT {
     if (!isSpark2) {
       result = sparkInterpreter.interpret(
           "val df=sqlContext.createDataFrame(Seq((\"12characters12characters\",20)))"
-                  + ".toDF(\"col_1\", \"col_2\")\n"
-                  + "df.collect()", context);
+              + ".toDF(\"col_1\", \"col_2\")\n"
+              + "df.collect()", context);
       assertEquals(InterpreterResult.Code.SUCCESS, result.code());
       assertEquals(1, result.message().size());
       assertTrue(result.message().get(0).getData()
@@ -321,8 +333,8 @@ public class LivyInterpreterIT {
     } else {
       result = sparkInterpreter.interpret(
           "val df=spark.createDataFrame(Seq((\"12characters12characters\",20)))"
-                  + ".toDF(\"col_1\", \"col_2\")\n"
-                  + "df.collect()", context);
+              + ".toDF(\"col_1\", \"col_2\")\n"
+              + "df.collect()", context);
       assertEquals(InterpreterResult.Code.SUCCESS, result.code());
       assertEquals(1, result.message().size());
       assertTrue(result.message().get(0).getData()
@@ -331,7 +343,7 @@ public class LivyInterpreterIT {
     sparkInterpreter.interpret("df.registerTempTable(\"df\")", context);
     // test LivySparkSQLInterpreter which share the same SparkContext with LivySparkInterpreter
     result = sqlInterpreter.interpret("select * from df where col_1='12characters12characters'",
-            context);
+        context);
     assertEquals(InterpreterResult.Code.SUCCESS, result.code());
     assertEquals(InterpreterResult.Type.TABLE, result.message().get(0).getType());
     assertEquals("col_1\tcol_2\n12characters12cha...\t20", result.message().get(0).getData());
@@ -349,8 +361,12 @@ public class LivyInterpreterIT {
     AuthenticationInfo authInfo = new AuthenticationInfo("user1");
     MyInterpreterOutputListener outputListener = new MyInterpreterOutputListener();
     InterpreterOutput output = new InterpreterOutput(outputListener);
-    final InterpreterContext context = new InterpreterContext("noteId", "paragraphId",
-            "livy.pyspark", "title", "text", authInfo, null, null, null, null, null, null, output);
+    final InterpreterContext context = InterpreterContext.builder()
+        .setNoteId("noteId")
+        .setParagraphId("paragraphId")
+        .setAuthenticationInfo(authInfo)
+        .setInterpreterOut(output)
+        .build();
     pysparkInterpreter.open();
 
     // test traceback msg
@@ -359,7 +375,7 @@ public class LivyInterpreterIT {
       // for livy version >=0.3 , input some erroneous spark code, check the shown result is more
       // than one line
       InterpreterResult result = pysparkInterpreter.interpret(
-              "sc.parallelize(wrongSyntax(1, 2)).count()", context);
+          "sc.parallelize(wrongSyntax(1, 2)).count()", context);
       assertEquals(InterpreterResult.Code.ERROR, result.code());
       assertTrue(result.message().get(0).getData().split("\n").length > 1);
       assertTrue(result.message().get(0).getData().contains("Traceback"));
@@ -465,7 +481,7 @@ public class LivyInterpreterIT {
 
   @Test
   public void testSparkInterpreterWithDisplayAppInfo_StringWithoutTruncation()
-          throws InterpreterException {
+      throws InterpreterException {
     if (!checkPreCondition()) {
       return;
     }
@@ -482,8 +498,12 @@ public class LivyInterpreterIT {
     AuthenticationInfo authInfo = new AuthenticationInfo("user1");
     MyInterpreterOutputListener outputListener = new MyInterpreterOutputListener();
     InterpreterOutput output = new InterpreterOutput(outputListener);
-    InterpreterContext context = new InterpreterContext("noteId", "paragraphId", "livy.spark",
-        "title", "text", authInfo, null, null, null, null, null, null, output);
+    InterpreterContext context = InterpreterContext.builder()
+        .setNoteId("noteId")
+        .setParagraphId("paragraphId")
+        .setAuthenticationInfo(authInfo)
+        .setInterpreterOut(output)
+        .build();
     sparkInterpreter.open();
 
     LivySparkSQLInterpreter sqlInterpreter = new LivySparkSQLInterpreter(properties2);
@@ -515,8 +535,8 @@ public class LivyInterpreterIT {
       if (!isSpark2) {
         result = sparkInterpreter.interpret(
             "val df=sqlContext.createDataFrame(Seq((\"12characters12characters\",20)))"
-                    + ".toDF(\"col_1\", \"col_2\")\n"
-                    + "df.collect()", context);
+                + ".toDF(\"col_1\", \"col_2\")\n"
+                + "df.collect()", context);
         assertEquals(InterpreterResult.Code.SUCCESS, result.code());
         assertEquals(2, result.message().size());
         assertTrue(result.message().get(0).getData()
@@ -524,8 +544,8 @@ public class LivyInterpreterIT {
       } else {
         result = sparkInterpreter.interpret(
             "val df=spark.createDataFrame(Seq((\"12characters12characters\",20)))"
-                    + ".toDF(\"col_1\", \"col_2\")\n"
-                    + "df.collect()", context);
+                + ".toDF(\"col_1\", \"col_2\")\n"
+                + "df.collect()", context);
         assertEquals(InterpreterResult.Code.SUCCESS, result.code());
         assertEquals(2, result.message().size());
         assertTrue(result.message().get(0).getData()
@@ -534,7 +554,7 @@ public class LivyInterpreterIT {
       sparkInterpreter.interpret("df.registerTempTable(\"df\")", context);
       // test LivySparkSQLInterpreter which share the same SparkContext with LivySparkInterpreter
       result = sqlInterpreter.interpret("select * from df where col_1='12characters12characters'",
-              context);
+          context);
       assertEquals(InterpreterResult.Code.SUCCESS, result.code());
       assertEquals(InterpreterResult.Type.TABLE, result.message().get(0).getType());
       assertEquals("col_1\tcol_2\n12characters12characters\t20", result.message().get(0).getData());
@@ -562,8 +582,12 @@ public class LivyInterpreterIT {
     AuthenticationInfo authInfo = new AuthenticationInfo("user1");
     MyInterpreterOutputListener outputListener = new MyInterpreterOutputListener();
     InterpreterOutput output = new InterpreterOutput(outputListener);
-    final InterpreterContext context = new InterpreterContext("noteId", "paragraphId",
-            "livy.sparkr", "title", "text", authInfo, null, null, null, null, null, null, output);
+    final InterpreterContext context = InterpreterContext.builder()
+        .setNoteId("noteId")
+        .setParagraphId("paragraphId")
+        .setAuthenticationInfo(authInfo)
+        .setInterpreterOut(output)
+        .build();
     sparkRInterpreter.open();
 
     try {
@@ -638,8 +662,12 @@ public class LivyInterpreterIT {
       AuthenticationInfo authInfo = new AuthenticationInfo("user1");
       MyInterpreterOutputListener outputListener = new MyInterpreterOutputListener();
       InterpreterOutput output = new InterpreterOutput(outputListener);
-      InterpreterContext context = new InterpreterContext("noteId", "paragraphId", "livy.sql",
-          "title", "text", authInfo, null, null, null, null, null, null, output);
+      InterpreterContext context = InterpreterContext.builder()
+          .setNoteId("noteId")
+          .setParagraphId("paragraphId")
+          .setAuthenticationInfo(authInfo)
+          .setInterpreterOut(output)
+          .build();
 
       String p1 = IOUtils.toString(getClass().getResourceAsStream("/livy_tutorial_1.scala"));
       InterpreterResult result = sparkInterpreter.interpret(p1, context);
@@ -696,15 +724,19 @@ public class LivyInterpreterIT {
       AuthenticationInfo authInfo = new AuthenticationInfo("user1");
       MyInterpreterOutputListener outputListener = new MyInterpreterOutputListener();
       InterpreterOutput output = new InterpreterOutput(outputListener);
-      InterpreterContext context = new InterpreterContext("noteId", "paragraphId", "livy.sql",
-          "title", "text", authInfo, null, null, null, null, null, null, output);
+      InterpreterContext context = InterpreterContext.builder()
+          .setNoteId("noteId")
+          .setParagraphId("paragraphId")
+          .setAuthenticationInfo(authInfo)
+          .setInterpreterOut(output)
+          .build();
       // detect spark version
       InterpreterResult result = sparkInterpreter.interpret("sc.version", context);
       assertEquals(InterpreterResult.Code.SUCCESS, result.code());
       assertEquals(1, result.message().size());
 
       boolean isSpark2 = isSpark2((BaseLivyInterpreter) sparkInterpreter.getInnerInterpreter(),
-              context);
+          context);
 
       if (!isSpark2) {
         result = sparkInterpreter.interpret(
@@ -718,7 +750,7 @@ public class LivyInterpreterIT {
 
         // access table from pyspark
         result = pysparkInterpreter.interpret("sqlContext.sql(\"select * from df\").show()",
-                context);
+            context);
         assertEquals(InterpreterResult.Code.SUCCESS, result.code());
         assertEquals(1, result.message().size());
         assertTrue(result.message().get(0).getData()
@@ -730,7 +762,7 @@ public class LivyInterpreterIT {
 
         // access table from sparkr
         result = sparkRInterpreter.interpret("head(sql(sqlContext, \"select * from df\"))",
-                context);
+            context);
         assertEquals(InterpreterResult.Code.SUCCESS, result.code());
         assertEquals(1, result.message().size());
         assertTrue(result.message().get(0).getData().contains("col_1 col_2\n1 hello    20"));
@@ -765,11 +797,11 @@ public class LivyInterpreterIT {
       // test plotting of python
       result = pysparkInterpreter.interpret(
           "import matplotlib.pyplot as plt\n" +
-          "plt.switch_backend('agg')\n" +
-          "data=[1,2,3,4]\n" +
-          "plt.figure()\n" +
-          "plt.plot(data)\n" +
-          "%matplot plt", context);
+              "plt.switch_backend('agg')\n" +
+              "data=[1,2,3,4]\n" +
+              "plt.figure()\n" +
+              "plt.plot(data)\n" +
+              "%matplot plt", context);
       assertEquals(InterpreterResult.Code.SUCCESS, result.code());
       assertEquals(1, result.message().size());
       assertEquals(InterpreterResult.Type.IMG, result.message().get(0).getType());

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/neo4j/src/test/java/org/apache/zeppelin/graph/neo4j/Neo4jCypherInterpreterTest.java
----------------------------------------------------------------------
diff --git a/neo4j/src/test/java/org/apache/zeppelin/graph/neo4j/Neo4jCypherInterpreterTest.java b/neo4j/src/test/java/org/apache/zeppelin/graph/neo4j/Neo4jCypherInterpreterTest.java
index d7c8b37..24bd513 100644
--- a/neo4j/src/test/java/org/apache/zeppelin/graph/neo4j/Neo4jCypherInterpreterTest.java
+++ b/neo4j/src/test/java/org/apache/zeppelin/graph/neo4j/Neo4jCypherInterpreterTest.java
@@ -16,11 +16,14 @@
  */
 package org.apache.zeppelin.graph.neo4j;
 
-import static org.junit.Assert.assertEquals;
-
 import com.google.gson.Gson;
-
 import org.apache.commons.lang3.StringUtils;
+import org.apache.zeppelin.graph.neo4j.Neo4jConnectionManager.Neo4jAuthType;
+import org.apache.zeppelin.interpreter.InterpreterContext;
+import org.apache.zeppelin.interpreter.InterpreterOutput;
+import org.apache.zeppelin.interpreter.InterpreterResult;
+import org.apache.zeppelin.interpreter.InterpreterResult.Code;
+import org.apache.zeppelin.interpreter.graph.GraphResult;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -32,23 +35,10 @@ import org.neo4j.harness.ServerControls;
 import org.neo4j.harness.TestServerBuilders;
 
 import java.util.Arrays;
-import java.util.HashMap;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Properties;
 
-import org.apache.zeppelin.display.AngularObjectRegistry;
-import org.apache.zeppelin.display.GUI;
-import org.apache.zeppelin.graph.neo4j.Neo4jConnectionManager.Neo4jAuthType;
-import org.apache.zeppelin.interpreter.InterpreterContext;
-import org.apache.zeppelin.interpreter.InterpreterContextRunner;
-import org.apache.zeppelin.interpreter.InterpreterGroup;
-import org.apache.zeppelin.interpreter.InterpreterOutput;
-import org.apache.zeppelin.interpreter.InterpreterResult;
-import org.apache.zeppelin.interpreter.InterpreterResult.Code;
-import org.apache.zeppelin.interpreter.graph.GraphResult;
-import org.apache.zeppelin.resource.LocalResourcePool;
-import org.apache.zeppelin.user.AuthenticationInfo;
+import static org.junit.Assert.assertEquals;
 
 @FixMethodOrder(MethodSorters.NAME_ASCENDING)
 public class Neo4jCypherInterpreterTest {
@@ -91,15 +81,9 @@ public class Neo4jCypherInterpreterTest {
     p.setProperty(Neo4jConnectionManager.NEO4J_AUTH_TYPE, Neo4jAuthType.NONE.toString());
     p.setProperty(Neo4jConnectionManager.NEO4J_MAX_CONCURRENCY, "50");
     interpreter = new Neo4jCypherInterpreter(p);
-    context = new InterpreterContext("note", "id", null, "title", "text",
-            new AuthenticationInfo(),
-            new HashMap<String, Object>(),
-            new GUI(),
-            new GUI(),
-            new AngularObjectRegistry(new InterpreterGroup().getId(), null),
-            new LocalResourcePool("id"),
-            new LinkedList<InterpreterContextRunner>(),
-            new InterpreterOutput(null));
+    context = InterpreterContext.builder()
+        .setInterpreterOut(new InterpreterOutput(null))
+        .build();;
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/pig/src/test/java/org/apache/zeppelin/pig/PigInterpreterSparkTest.java
----------------------------------------------------------------------
diff --git a/pig/src/test/java/org/apache/zeppelin/pig/PigInterpreterSparkTest.java b/pig/src/test/java/org/apache/zeppelin/pig/PigInterpreterSparkTest.java
index d25d6e2..ea1a3f8 100644
--- a/pig/src/test/java/org/apache/zeppelin/pig/PigInterpreterSparkTest.java
+++ b/pig/src/test/java/org/apache/zeppelin/pig/PigInterpreterSparkTest.java
@@ -43,8 +43,7 @@ public class PigInterpreterSparkTest {
     properties.put("zeppelin.pig.includeJobStats", includeJobStats + "");
     pigInterpreter = new PigInterpreter(properties);
     pigInterpreter.open();
-    context = new InterpreterContext(null, "paragraph_id", null, null, null, null, null, null, null,
-        null, null, null, null);
+    context = InterpreterContext.builder().setParagraphId("paragraphId").build();
 
   }
   @After

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/pig/src/test/java/org/apache/zeppelin/pig/PigInterpreterTest.java
----------------------------------------------------------------------
diff --git a/pig/src/test/java/org/apache/zeppelin/pig/PigInterpreterTest.java b/pig/src/test/java/org/apache/zeppelin/pig/PigInterpreterTest.java
index 59ea753..5a21bb3 100644
--- a/pig/src/test/java/org/apache/zeppelin/pig/PigInterpreterTest.java
+++ b/pig/src/test/java/org/apache/zeppelin/pig/PigInterpreterTest.java
@@ -46,8 +46,7 @@ public class PigInterpreterTest {
     properties.put("zeppelin.pig.includeJobStats", includeJobStats + "");
     pigInterpreter = new PigInterpreter(properties);
     pigInterpreter.open();
-    context = new InterpreterContext(null, "paragraph_id", null, null, null,
-        null, null, null, null, null, null, null, null);
+    context = InterpreterContext.builder().setParagraphId("paragraphId").build();
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/pig/src/test/java/org/apache/zeppelin/pig/PigInterpreterTezTest.java
----------------------------------------------------------------------
diff --git a/pig/src/test/java/org/apache/zeppelin/pig/PigInterpreterTezTest.java b/pig/src/test/java/org/apache/zeppelin/pig/PigInterpreterTezTest.java
index 409502b..ec09a88 100644
--- a/pig/src/test/java/org/apache/zeppelin/pig/PigInterpreterTezTest.java
+++ b/pig/src/test/java/org/apache/zeppelin/pig/PigInterpreterTezTest.java
@@ -47,8 +47,7 @@ public class PigInterpreterTezTest {
     properties.put("tez.queue.name", "test");
     pigInterpreter = new PigInterpreter(properties);
     pigInterpreter.open();
-    context = new InterpreterContext(null, "paragraph_id", null, null, null, null, null, null, null,
-        null, null, null, null);
+    context = InterpreterContext.builder().setParagraphId("paragraphId").build();
 
   }
   @After

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/pig/src/test/java/org/apache/zeppelin/pig/PigQueryInterpreterTest.java
----------------------------------------------------------------------
diff --git a/pig/src/test/java/org/apache/zeppelin/pig/PigQueryInterpreterTest.java b/pig/src/test/java/org/apache/zeppelin/pig/PigQueryInterpreterTest.java
index 2ca586b..7243489 100644
--- a/pig/src/test/java/org/apache/zeppelin/pig/PigQueryInterpreterTest.java
+++ b/pig/src/test/java/org/apache/zeppelin/pig/PigQueryInterpreterTest.java
@@ -66,8 +66,7 @@ public class PigQueryInterpreterTest {
     pigInterpreter.open();
     pigQueryInterpreter.open();
 
-    context = new InterpreterContext(null, "paragraph_id", null, null, null, null, null, null, null,
-        null, null, null, null);
+    context = InterpreterContext.builder().setParagraphId("paragraphId").build();
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/python/src/test/java/org/apache/zeppelin/python/BasePythonInterpreterTest.java
----------------------------------------------------------------------
diff --git a/python/src/test/java/org/apache/zeppelin/python/BasePythonInterpreterTest.java b/python/src/test/java/org/apache/zeppelin/python/BasePythonInterpreterTest.java
index 9bedd53..0a3a606 100644
--- a/python/src/test/java/org/apache/zeppelin/python/BasePythonInterpreterTest.java
+++ b/python/src/test/java/org/apache/zeppelin/python/BasePythonInterpreterTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.zeppelin.python;
 
-import org.apache.zeppelin.display.GUI;
 import org.apache.zeppelin.display.ui.CheckBox;
 import org.apache.zeppelin.display.ui.Select;
 import org.apache.zeppelin.display.ui.TextBox;
@@ -28,19 +27,18 @@ import org.apache.zeppelin.interpreter.InterpreterGroup;
 import org.apache.zeppelin.interpreter.InterpreterOutput;
 import org.apache.zeppelin.interpreter.InterpreterResult;
 import org.apache.zeppelin.interpreter.InterpreterResultMessage;
-import org.apache.zeppelin.interpreter.remote.RemoteEventClient;
+import org.apache.zeppelin.interpreter.remote.RemoteInterpreterEventClient;
 import org.apache.zeppelin.interpreter.thrift.InterpreterCompletion;
-import org.apache.zeppelin.user.AuthenticationInfo;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.List;
 
 import static junit.framework.TestCase.assertTrue;
 import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
 
 public abstract class BasePythonInterpreterTest {
 
@@ -307,25 +305,11 @@ public abstract class BasePythonInterpreterTest {
   }
 
   protected InterpreterContext getInterpreterContext() {
-    return new InterpreterContext(
-        "noteId",
-        "paragraphId",
-        "replName",
-        "paragraphTitle",
-        "paragraphText",
-        new AuthenticationInfo(),
-        new HashMap<String, Object>(),
-        new GUI(),
-        new GUI(),
-        null,
-        null,
-        null,
-        new InterpreterOutput(null));
-  }
-
-  protected InterpreterContext getInterpreterContext(RemoteEventClient mockRemoteEventClient) {
-    InterpreterContext context = getInterpreterContext();
-    context.setClient(mockRemoteEventClient);
-    return context;
+    return InterpreterContext.builder()
+        .setNoteId("noteId")
+        .setParagraphId("paragraphId")
+        .setInterpreterOut(new InterpreterOutput(null))
+        .setIntpEventClient(mock(RemoteInterpreterEventClient.class))
+        .build();
   }
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/python/src/test/java/org/apache/zeppelin/python/IPythonInterpreterTest.java
----------------------------------------------------------------------
diff --git a/python/src/test/java/org/apache/zeppelin/python/IPythonInterpreterTest.java b/python/src/test/java/org/apache/zeppelin/python/IPythonInterpreterTest.java
index 9e01d06..be72ca9 100644
--- a/python/src/test/java/org/apache/zeppelin/python/IPythonInterpreterTest.java
+++ b/python/src/test/java/org/apache/zeppelin/python/IPythonInterpreterTest.java
@@ -21,9 +21,11 @@ import org.apache.zeppelin.interpreter.Interpreter;
 import org.apache.zeppelin.interpreter.InterpreterContext;
 import org.apache.zeppelin.interpreter.InterpreterException;
 import org.apache.zeppelin.interpreter.InterpreterGroup;
+import org.apache.zeppelin.interpreter.InterpreterOutput;
 import org.apache.zeppelin.interpreter.InterpreterResult;
 import org.apache.zeppelin.interpreter.InterpreterResultMessage;
 import org.apache.zeppelin.interpreter.LazyOpenInterpreter;
+import org.apache.zeppelin.interpreter.remote.RemoteInterpreterEventClient;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -38,7 +40,6 @@ import static org.mockito.Mockito.mock;
 
 public class IPythonInterpreterTest extends BasePythonInterpreterTest {
 
-
   protected Properties initIntpProperties() {
     Properties properties = new Properties();
     properties.setProperty("zeppelin.python.maxResult", "3");

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/python/src/test/java/org/apache/zeppelin/python/PythonCondaInterpreterTest.java
----------------------------------------------------------------------
diff --git a/python/src/test/java/org/apache/zeppelin/python/PythonCondaInterpreterTest.java b/python/src/test/java/org/apache/zeppelin/python/PythonCondaInterpreterTest.java
index f1be1b9..292580a 100644
--- a/python/src/test/java/org/apache/zeppelin/python/PythonCondaInterpreterTest.java
+++ b/python/src/test/java/org/apache/zeppelin/python/PythonCondaInterpreterTest.java
@@ -18,19 +18,30 @@
 package org.apache.zeppelin.python;
 
 
-import org.apache.zeppelin.display.GUI;
-import org.apache.zeppelin.interpreter.*;
-import org.apache.zeppelin.user.AuthenticationInfo;
+import org.apache.zeppelin.interpreter.InterpreterContext;
+import org.apache.zeppelin.interpreter.InterpreterException;
+import org.apache.zeppelin.interpreter.InterpreterGroup;
+import org.apache.zeppelin.interpreter.InterpreterOutput;
+import org.apache.zeppelin.interpreter.InterpreterResult;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
 import java.util.regex.Matcher;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
 
 public class PythonCondaInterpreterTest {
   private PythonCondaInterpreter conda;
@@ -124,20 +135,9 @@ public class PythonCondaInterpreterTest {
   }
 
   private InterpreterContext getInterpreterContext() {
-    return new InterpreterContext(
-        "noteId",
-        "paragraphId",
-        null,
-        "paragraphTitle",
-        "paragraphText",
-        new AuthenticationInfo(),
-        new HashMap<String, Object>(),
-        new GUI(),
-        new GUI(),
-        null,
-        null,
-        null,
-        new InterpreterOutput(null));
+    return InterpreterContext.builder()
+        .setInterpreterOut(new InterpreterOutput(null))
+        .build();
   }
 
 

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/python/src/test/java/org/apache/zeppelin/python/PythonDockerInterpreterTest.java
----------------------------------------------------------------------
diff --git a/python/src/test/java/org/apache/zeppelin/python/PythonDockerInterpreterTest.java b/python/src/test/java/org/apache/zeppelin/python/PythonDockerInterpreterTest.java
index 17f6cc1..a862a59 100644
--- a/python/src/test/java/org/apache/zeppelin/python/PythonDockerInterpreterTest.java
+++ b/python/src/test/java/org/apache/zeppelin/python/PythonDockerInterpreterTest.java
@@ -1,34 +1,31 @@
 /*
-* 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.
-*/
+ * 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.python;
 
-import org.apache.zeppelin.display.GUI;
 import org.apache.zeppelin.interpreter.InterpreterContext;
 import org.apache.zeppelin.interpreter.InterpreterException;
 import org.apache.zeppelin.interpreter.InterpreterGroup;
 import org.apache.zeppelin.interpreter.InterpreterOutput;
-import org.apache.zeppelin.user.AuthenticationInfo;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
 import java.io.File;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.Properties;
 
 import static org.mockito.Matchers.any;
@@ -80,19 +77,8 @@ public class PythonDockerInterpreterTest {
   }
 
   private InterpreterContext getInterpreterContext() {
-    return new InterpreterContext(
-        "noteId",
-        "paragraphId",
-        "replName",
-        "paragraphTitle",
-        "paragraphText",
-        new AuthenticationInfo(),
-        new HashMap<String, Object>(),
-        new GUI(),
-        new GUI(),
-        null,
-        null,
-        null,
-        new InterpreterOutput(null));
+    return InterpreterContext.builder()
+        .setInterpreterOut(new InterpreterOutput(null))
+        .build();
   }
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/python/src/test/java/org/apache/zeppelin/python/PythonInterpreterMatplotlibTest.java
----------------------------------------------------------------------
diff --git a/python/src/test/java/org/apache/zeppelin/python/PythonInterpreterMatplotlibTest.java b/python/src/test/java/org/apache/zeppelin/python/PythonInterpreterMatplotlibTest.java
index 1ab9cf1..8475331 100644
--- a/python/src/test/java/org/apache/zeppelin/python/PythonInterpreterMatplotlibTest.java
+++ b/python/src/test/java/org/apache/zeppelin/python/PythonInterpreterMatplotlibTest.java
@@ -67,15 +67,9 @@ public class PythonInterpreterMatplotlibTest implements InterpreterOutputListene
 
     out = new InterpreterOutput(this);
 
-    context = new InterpreterContext("note", "id", null, "title", "text",
-        new AuthenticationInfo(),
-        new HashMap<String, Object>(),
-        new GUI(),
-        new GUI(),
-        new AngularObjectRegistry(intpGroup.getId(), null),
-        new LocalResourcePool("id"),
-        new LinkedList<InterpreterContextRunner>(),
-        out);
+    context = InterpreterContext.builder()
+        .setInterpreterOut(out)
+        .build();
     python.open();
   }
 

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/python/src/test/java/org/apache/zeppelin/python/PythonInterpreterPandasSqlTest.java
----------------------------------------------------------------------
diff --git a/python/src/test/java/org/apache/zeppelin/python/PythonInterpreterPandasSqlTest.java b/python/src/test/java/org/apache/zeppelin/python/PythonInterpreterPandasSqlTest.java
index d8f41c4..fbaa695 100644
--- a/python/src/test/java/org/apache/zeppelin/python/PythonInterpreterPandasSqlTest.java
+++ b/python/src/test/java/org/apache/zeppelin/python/PythonInterpreterPandasSqlTest.java
@@ -79,15 +79,9 @@ public class PythonInterpreterPandasSqlTest implements InterpreterOutputListener
     intpGroup = new InterpreterGroup();
 
     out = new InterpreterOutput(this);
-    context = new InterpreterContext("note", "id", null, "title", "text",
-        new AuthenticationInfo(),
-        new HashMap<String, Object>(),
-        new GUI(),
-        new GUI(),
-        new AngularObjectRegistry(intpGroup.getId(), null),
-        new LocalResourcePool("id"),
-        new LinkedList<InterpreterContextRunner>(),
-        out);
+    context = InterpreterContext.builder()
+        .setInterpreterOut(out)
+        .build();
     InterpreterContext.set(context);
 
     python = new PythonInterpreter(p);

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/python/src/test/java/org/apache/zeppelin/python/PythonInterpreterTest.java
----------------------------------------------------------------------
diff --git a/python/src/test/java/org/apache/zeppelin/python/PythonInterpreterTest.java b/python/src/test/java/org/apache/zeppelin/python/PythonInterpreterTest.java
index e750dde..8748c00 100644
--- a/python/src/test/java/org/apache/zeppelin/python/PythonInterpreterTest.java
+++ b/python/src/test/java/org/apache/zeppelin/python/PythonInterpreterTest.java
@@ -36,6 +36,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
+
 public class PythonInterpreterTest extends BasePythonInterpreterTest {
 
   @Override
@@ -78,7 +79,7 @@ public class PythonInterpreterTest extends BasePythonInterpreterTest {
   private class infinityPythonJob implements Runnable {
     @Override
     public void run() {
-      String code = "import time\nwhile True:\n  time.sleep(1)" ;
+      String code = "import time\nwhile True:\n  time.sleep(1)";
       InterpreterResult ret = null;
       try {
         ret = interpreter.interpret(code, getInterpreterContext());

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/scalding/src/test/java/org/apache/zeppelin/scalding/ScaldingInterpreterTest.java
----------------------------------------------------------------------
diff --git a/scalding/src/test/java/org/apache/zeppelin/scalding/ScaldingInterpreterTest.java b/scalding/src/test/java/org/apache/zeppelin/scalding/ScaldingInterpreterTest.java
index 2c94751..992c155 100644
--- a/scalding/src/test/java/org/apache/zeppelin/scalding/ScaldingInterpreterTest.java
+++ b/scalding/src/test/java/org/apache/zeppelin/scalding/ScaldingInterpreterTest.java
@@ -17,9 +17,10 @@
 
 package org.apache.zeppelin.scalding;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
+import org.apache.zeppelin.interpreter.InterpreterContext;
+import org.apache.zeppelin.interpreter.InterpreterResult;
+import org.apache.zeppelin.interpreter.InterpreterResult.Code;
+import org.apache.zeppelin.user.AuthenticationInfo;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.FixMethodOrder;
@@ -27,18 +28,10 @@ import org.junit.Test;
 import org.junit.runners.MethodSorters;
 
 import java.io.File;
-import java.util.HashMap;
-import java.util.LinkedList;
 import java.util.Properties;
 
-import org.apache.zeppelin.display.AngularObjectRegistry;
-import org.apache.zeppelin.display.GUI;
-import org.apache.zeppelin.interpreter.InterpreterContext;
-import org.apache.zeppelin.interpreter.InterpreterContextRunner;
-import org.apache.zeppelin.interpreter.InterpreterGroup;
-import org.apache.zeppelin.interpreter.InterpreterResult;
-import org.apache.zeppelin.interpreter.InterpreterResult.Code;
-import org.apache.zeppelin.user.AuthenticationInfo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Tests for the Scalding interpreter for Zeppelin.
@@ -66,11 +59,11 @@ public class ScaldingInterpreterTest {
       repl.open();
     }
 
-    InterpreterGroup intpGroup = new InterpreterGroup();
-    context = new InterpreterContext("note", "id", null, "title", "text", new AuthenticationInfo(),
-        new HashMap<String, Object>(), new GUI(), new GUI(), new AngularObjectRegistry(
-            intpGroup.getId(), null), null,
-        new LinkedList<InterpreterContextRunner>(), null);
+    context = InterpreterContext.builder()
+        .setNoteId("noteId")
+        .setParagraphId("paragraphId")
+        .setAuthenticationInfo(new AuthenticationInfo())
+        .build();
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/scio/src/test/java/org/apache/zeppelin/scio/ScioInterpreterTest.java
----------------------------------------------------------------------
diff --git a/scio/src/test/java/org/apache/zeppelin/scio/ScioInterpreterTest.java b/scio/src/test/java/org/apache/zeppelin/scio/ScioInterpreterTest.java
index 91b5fa1..2e5c0d9 100644
--- a/scio/src/test/java/org/apache/zeppelin/scio/ScioInterpreterTest.java
+++ b/scio/src/test/java/org/apache/zeppelin/scio/ScioInterpreterTest.java
@@ -40,15 +40,10 @@ public class ScioInterpreterTest {
   private final String newline = "\n";
 
   private InterpreterContext getNewContext() {
-    return new InterpreterContext("note", "id", null, "title", "text",
-        new AuthenticationInfo(),
-        new HashMap<String, Object>(),
-        new GUI(),
-        new GUI(),
-        new AngularObjectRegistry(intpGroup.getId(), null),
-        new LocalResourcePool("id"),
-        new LinkedList<InterpreterContextRunner>(),
-        new InterpreterOutput(null));
+    return InterpreterContext.builder()
+        .setNoteId("noteId")
+        .setParagraphId("paragraphId")
+        .build();
   }
 
   @Before

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/shell/src/test/java/org/apache/zeppelin/shell/ShellInterpreterTest.java
----------------------------------------------------------------------
diff --git a/shell/src/test/java/org/apache/zeppelin/shell/ShellInterpreterTest.java b/shell/src/test/java/org/apache/zeppelin/shell/ShellInterpreterTest.java
index 1b76346..5a8f4b4 100644
--- a/shell/src/test/java/org/apache/zeppelin/shell/ShellInterpreterTest.java
+++ b/shell/src/test/java/org/apache/zeppelin/shell/ShellInterpreterTest.java
@@ -42,8 +42,7 @@ public class ShellInterpreterTest {
     p.setProperty("shell.command.timeout.millisecs", "2000");
     shell = new ShellInterpreter(p);
 
-    context = new InterpreterContext("", "1", null, "", "", null, null, null, null, null, null,
-        null, null);
+    context = InterpreterContext.builder().setParagraphId("paragraphId").build();
     shell.open();
   }
 

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/spark/interpreter/src/main/java/org/apache/zeppelin/spark/NewSparkInterpreter.java
----------------------------------------------------------------------
diff --git a/spark/interpreter/src/main/java/org/apache/zeppelin/spark/NewSparkInterpreter.java b/spark/interpreter/src/main/java/org/apache/zeppelin/spark/NewSparkInterpreter.java
index fcfee27..b60e38b 100644
--- a/spark/interpreter/src/main/java/org/apache/zeppelin/spark/NewSparkInterpreter.java
+++ b/spark/interpreter/src/main/java/org/apache/zeppelin/spark/NewSparkInterpreter.java
@@ -110,15 +110,15 @@ public class NewSparkInterpreter extends AbstractSparkInterpreter {
       }
       sqlContext = this.innerInterpreter.sqlContext();
       sparkSession = this.innerInterpreter.sparkSession();
-      sparkUrl = this.innerInterpreter.sparkUrl();
-      sparkShims = SparkShims.getInstance(sc.version());
-      sparkShims.setupSparkListener(sc.master(), sparkUrl);
-
       hooks = getInterpreterGroup().getInterpreterHookRegistry();
       z = new SparkZeppelinContext(sc, hooks,
           Integer.parseInt(getProperty("zeppelin.spark.maxResult")));
       this.innerInterpreter.bind("z", z.getClass().getCanonicalName(), z,
           Lists.newArrayList("@transient"));
+
+      sparkUrl = this.innerInterpreter.sparkUrl();
+      sparkShims = SparkShims.getInstance(sc.version());
+      sparkShims.setupSparkListener(sc.master(), sparkUrl, InterpreterContext.get());
     } catch (Exception e) {
       LOGGER.error("Fail to open SparkInterpreter", ExceptionUtils.getStackTrace(e));
       throw new InterpreterException("Fail to open SparkInterpreter", e);
@@ -231,10 +231,9 @@ public class NewSparkInterpreter extends AbstractSparkInterpreter {
         infos.put("message", "No spark url defined");
       }
     }
-    if (ctx != null && ctx.getClient() != null) {
+    if (ctx != null) {
       LOGGER.debug("Sending metadata to Zeppelin server: {}", infos.toString());
-      getZeppelinContext().setEventClient(ctx.getClient());
-      ctx.getClient().onMetaInfosReceived(infos);
+      ctx.getIntpEventClient().onMetaInfosReceived(infos);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/spark/interpreter/src/main/java/org/apache/zeppelin/spark/OldSparkInterpreter.java
----------------------------------------------------------------------
diff --git a/spark/interpreter/src/main/java/org/apache/zeppelin/spark/OldSparkInterpreter.java b/spark/interpreter/src/main/java/org/apache/zeppelin/spark/OldSparkInterpreter.java
index 83d3d6a..850b53e 100644
--- a/spark/interpreter/src/main/java/org/apache/zeppelin/spark/OldSparkInterpreter.java
+++ b/spark/interpreter/src/main/java/org/apache/zeppelin/spark/OldSparkInterpreter.java
@@ -162,8 +162,6 @@ public class OldSparkInterpreter extends AbstractSparkInterpreter {
     this(property);
     this.sc = sc;
     env = SparkEnv.get();
-    sparkShims = SparkShims.getInstance(sc.version());
-    sparkShims.setupSparkListener(sc.master(), sparkUrl);
   }
 
   public SparkContext getSparkContext() {
@@ -873,8 +871,7 @@ public class OldSparkInterpreter extends AbstractSparkInterpreter {
 
     sparkUrl = getSparkUIUrl();
     sparkShims = SparkShims.getInstance(sc.version());
-    sparkShims.setupSparkListener(sc.master(), sparkUrl);
-
+    sparkShims.setupSparkListener(sc.master(), sparkUrl, InterpreterContext.get());
     numReferenceOfSparkContext.incrementAndGet();
   }
 
@@ -927,10 +924,9 @@ public class OldSparkInterpreter extends AbstractSparkInterpreter {
         infos.put("message", "No spark url defined");
       }
     }
-    if (ctx != null && ctx.getClient() != null) {
+    if (ctx != null) {
       logger.info("Sending metadata to Zeppelin server: {}", infos.toString());
-      getZeppelinContext().setEventClient(ctx.getClient());
-      ctx.getClient().onMetaInfosReceived(infos);
+      ctx.getIntpEventClient().onMetaInfosReceived(infos);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/7af86168/spark/interpreter/src/main/java/org/apache/zeppelin/spark/SparkRInterpreter.java
----------------------------------------------------------------------
diff --git a/spark/interpreter/src/main/java/org/apache/zeppelin/spark/SparkRInterpreter.java b/spark/interpreter/src/main/java/org/apache/zeppelin/spark/SparkRInterpreter.java
index 6d21450..41c11ee 100644
--- a/spark/interpreter/src/main/java/org/apache/zeppelin/spark/SparkRInterpreter.java
+++ b/spark/interpreter/src/main/java/org/apache/zeppelin/spark/SparkRInterpreter.java
@@ -33,6 +33,7 @@ import org.apache.zeppelin.scheduler.SchedulerFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -76,11 +77,13 @@ public class SparkRInterpreter extends Interpreter {
       // yarn-cluster mode
       sparkRLibPath = "sparkr";
     }
+    if (!new File(sparkRLibPath).exists()) {
+      throw new InterpreterException(String.format("sparkRLib %s doesn't exist", sparkRLibPath));
+    }
 
     this.sparkInterpreter = getSparkInterpreter();
     this.sc = sparkInterpreter.getSparkContext();
     this.jsc = sparkInterpreter.getJavaSparkContext();
-
     // Share the same SparkRBackend across sessions
     SparkVersion sparkVersion = new SparkVersion(sc.version());
     synchronized (SparkRBackend.backend()) {