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/03/06 02:08:30 UTC

zeppelin git commit: ZEPPELIN-3291. Throw exception instead of return null for InterpreterFactory.getInterpreter

Repository: zeppelin
Updated Branches:
  refs/heads/master d6cdd56d8 -> e148c8cff


ZEPPELIN-3291. Throw exception instead of return null for InterpreterFactory.getInterpreter

### What is this PR for?
Minor code refactoring to throw exception when no interpreter is found

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

### Todos
* [ ] - Task

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

### 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 #2836 from zjffdu/ZEPPELIN-3291 and squashes the following commits:

bee319c [Jeff Zhang] ZEPPELIN-3291. Throw exception instead of return null for InterpreterFactory.getInterpreter


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

Branch: refs/heads/master
Commit: e148c8cfff04a7a299b2d626759a662a3f91087b
Parents: d6cdd56
Author: Jeff Zhang <zj...@apache.org>
Authored: Mon Mar 5 12:58:19 2018 +0800
Committer: Jeff Zhang <zj...@apache.org>
Committed: Tue Mar 6 10:08:23 2018 +0800

----------------------------------------------------------------------
 .../zeppelin/helium/ApplicationException.java   |  4 ++
 .../apache/zeppelin/socket/NotebookServer.java  |  7 ++-
 .../java/org/apache/zeppelin/helium/Helium.java |  7 ++-
 .../helium/HeliumApplicationFactory.java        | 24 ++++++---
 .../interpreter/InterpreterFactory.java         | 20 +++----
 .../InterpreterNotFoundException.java           | 22 ++++++++
 .../org/apache/zeppelin/notebook/Paragraph.java | 56 +++++++++++---------
 .../helium/HeliumApplicationFactoryTest.java    | 11 +++-
 .../interpreter/InterpreterFactoryTest.java     | 12 +++--
 .../InterpreterSettingManagerTest.java          |  8 +--
 .../org/apache/zeppelin/notebook/NoteTest.java  | 11 ++--
 .../apache/zeppelin/notebook/NotebookTest.java  | 12 +++--
 12 files changed, 133 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/zeppelin/blob/e148c8cf/zeppelin-interpreter/src/main/java/org/apache/zeppelin/helium/ApplicationException.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/helium/ApplicationException.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/helium/ApplicationException.java
index 4bf0ac2..d3c6488 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/helium/ApplicationException.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/helium/ApplicationException.java
@@ -31,4 +31,8 @@ public class ApplicationException extends Exception {
   public ApplicationException() {
 
   }
+
+  public ApplicationException(String message, Throwable cause) {
+    super(message, cause);
+  }
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/e148c8cf/zeppelin-server/src/main/java/org/apache/zeppelin/socket/NotebookServer.java
----------------------------------------------------------------------
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/socket/NotebookServer.java b/zeppelin-server/src/main/java/org/apache/zeppelin/socket/NotebookServer.java
index 113dfd6..d14a852 100644
--- a/zeppelin-server/src/main/java/org/apache/zeppelin/socket/NotebookServer.java
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/socket/NotebookServer.java
@@ -2442,7 +2442,12 @@ public class NotebookServer extends WebSocketServlet
     Message resp = new Message(OP.EDITOR_SETTING);
     resp.put("paragraphId", paragraphId);
     Interpreter interpreter =
-        notebook().getInterpreterFactory().getInterpreter(user, noteId, replName);
+        null;
+    try {
+      interpreter = notebook().getInterpreterFactory().getInterpreter(user, noteId, replName);
+    } catch (InterpreterNotFoundException e) {
+      throw new IOException("Fail to get interpreter: " + replName, e);
+    }
     resp.put("editor", notebook().getInterpreterSettingManager().
         getEditorSetting(interpreter, user, noteId, replName));
     conn.send(serializeMessage(resp));

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/e148c8cf/zeppelin-zengine/src/main/java/org/apache/zeppelin/helium/Helium.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/helium/Helium.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/helium/Helium.java
index 0af15ab..399aea8 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/helium/Helium.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/helium/Helium.java
@@ -20,6 +20,7 @@ import com.google.gson.Gson;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.zeppelin.interpreter.Interpreter;
+import org.apache.zeppelin.interpreter.InterpreterNotFoundException;
 import org.apache.zeppelin.interpreter.InterpreterSettingManager;
 import org.apache.zeppelin.interpreter.ManagedInterpreterGroup;
 import org.apache.zeppelin.interpreter.remote.RemoteInterpreterProcess;
@@ -340,8 +341,10 @@ public class Helium {
   public HeliumPackageSuggestion suggestApp(Paragraph paragraph) {
     HeliumPackageSuggestion suggestion = new HeliumPackageSuggestion();
 
-    Interpreter intp = paragraph.getBindedInterpreter();
-    if (intp == null) {
+    Interpreter intp = null;
+    try {
+      intp = paragraph.getBindedInterpreter();
+    } catch (InterpreterNotFoundException e) {
       return suggestion;
     }
 

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/e148c8cf/zeppelin-zengine/src/main/java/org/apache/zeppelin/helium/HeliumApplicationFactory.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/helium/HeliumApplicationFactory.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/helium/HeliumApplicationFactory.java
index bdb3614..50928bb 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/helium/HeliumApplicationFactory.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/helium/HeliumApplicationFactory.java
@@ -200,9 +200,11 @@ public class HeliumApplicationFactory implements ApplicationEventListener, Noteb
               "Can't unload application status " + appsToUnload.getStatus());
         }
         appStatusChange(paragraph, appsToUnload.getId(), ApplicationState.Status.UNLOADING);
-        Interpreter intp = paragraph.getBindedInterpreter();
-        if (intp == null) {
-          throw new ApplicationException("No interpreter found");
+        Interpreter intp = null;
+        try {
+          intp = paragraph.getBindedInterpreter();
+        } catch (InterpreterException e) {
+          throw new ApplicationException("No interpreter found", e);
         }
 
         RemoteInterpreterProcess intpProcess =
@@ -280,9 +282,11 @@ public class HeliumApplicationFactory implements ApplicationEventListener, Noteb
               "Can't run application status " + app.getStatus());
         }
 
-        Interpreter intp = paragraph.getBindedInterpreter();
-        if (intp == null) {
-          throw new ApplicationException("No interpreter found");
+        Interpreter intp = null;
+        try {
+          intp = paragraph.getBindedInterpreter();
+        } catch (InterpreterException e) {
+          throw new ApplicationException("No interpreter found", e);
         }
 
         RemoteInterpreterProcess intpProcess =
@@ -417,7 +421,13 @@ public class HeliumApplicationFactory implements ApplicationEventListener, Noteb
   @Override
   public void onUnbindInterpreter(Note note, InterpreterSetting setting) {
     for (Paragraph p : note.getParagraphs()) {
-      Interpreter currentInterpreter = p.getBindedInterpreter();
+      Interpreter currentInterpreter = null;
+      try {
+        currentInterpreter = p.getBindedInterpreter();
+      } catch (InterpreterNotFoundException e) {
+        logger.warn("Not interpreter found", e);
+        return;
+      }
       List<InterpreterInfo> infos = setting.getInterpreterInfos();
       for (InterpreterInfo info : infos) {
         if (currentInterpreter != null &&

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/e148c8cf/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterFactory.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterFactory.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterFactory.java
index 911c285..20b1c2b 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterFactory.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterFactory.java
@@ -50,14 +50,14 @@ public class InterpreterFactory {
     return null;
   }
 
-  public Interpreter getInterpreter(String user, String noteId, String replName) {
+  public Interpreter getInterpreter(String user, String noteId, String replName)
+      throws InterpreterNotFoundException {
     List<InterpreterSetting> settings = interpreterSettingManager.getInterpreterSettings(noteId);
     InterpreterSetting setting;
     Interpreter interpreter;
 
     if (settings == null || settings.size() == 0) {
-      LOGGER.error("No interpreter is binded to this note: " + noteId);
-      return null;
+      throw new InterpreterNotFoundException("No interpreter is binded to this note: " + noteId);
     }
 
     if (StringUtils.isBlank(replName)) {
@@ -76,9 +76,10 @@ public class InterpreterFactory {
         if (null != interpreter) {
           return interpreter;
         }
-        throw new RuntimeException("No such interpreter: " + replName);
+        throw new InterpreterNotFoundException("No such interpreter: " + replName);
       }
-      throw new RuntimeException("Interpreter " + group + " is not binded to this note");
+      throw new InterpreterNotFoundException("Interpreter " + group +
+          " is not binded to this note");
     } else if (replNameSplit.length == 1){
       // first assume replName is 'name' of interpreter. ('groupName' is ommitted)
       // search 'name' from first (default) interpreter group
@@ -97,11 +98,12 @@ public class InterpreterFactory {
       if (null != setting) {
         return setting.getDefaultInterpreter(user, noteId);
       } else {
-        throw new RuntimeException("Either no interpreter named " + replName + " or it is not " +
-            "binded to this note");
+        throw new InterpreterNotFoundException("Either no interpreter named " + replName +
+            " or it is not binded to this note");
       }
     }
-    //TODO(zjffdu) throw InterpreterException instead of return null
-    return null;
+
+    throw new InterpreterNotFoundException("No such interpreter " + replName + " for note "
+        + noteId);
   }
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/e148c8cf/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterNotFoundException.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterNotFoundException.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterNotFoundException.java
new file mode 100644
index 0000000..192e822
--- /dev/null
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterNotFoundException.java
@@ -0,0 +1,22 @@
+package org.apache.zeppelin.interpreter;
+
+/**
+ * Exception for no interpreter is found
+ */
+public class InterpreterNotFoundException extends InterpreterException {
+
+  public InterpreterNotFoundException() {
+  }
+
+  public InterpreterNotFoundException(String message) {
+    super(message);
+  }
+
+  public InterpreterNotFoundException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public InterpreterNotFoundException(Throwable cause) {
+    super(cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/e148c8cf/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Paragraph.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Paragraph.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Paragraph.java
index 76c48c8..23d2d47 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Paragraph.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Paragraph.java
@@ -43,6 +43,7 @@ import org.apache.zeppelin.interpreter.InterpreterContext;
 import org.apache.zeppelin.interpreter.InterpreterContextRunner;
 import org.apache.zeppelin.interpreter.InterpreterException;
 import org.apache.zeppelin.interpreter.InterpreterFactory;
+import org.apache.zeppelin.interpreter.InterpreterNotFoundException;
 import org.apache.zeppelin.interpreter.InterpreterOption;
 import org.apache.zeppelin.interpreter.InterpreterOutput;
 import org.apache.zeppelin.interpreter.InterpreterOutputListener;
@@ -237,7 +238,7 @@ public class Paragraph extends Job implements Cloneable, JsonSerializable {
     return enabled == null || enabled.booleanValue();
   }
 
-  public Interpreter getBindedInterpreter() {
+  public Interpreter getBindedInterpreter() throws InterpreterNotFoundException {
     return this.interpreterFactory.getInterpreter(user, note.getId(), intpText);
   }
 
@@ -253,8 +254,11 @@ public class Paragraph extends Job implements Cloneable, JsonSerializable {
         return note.getInterpreterCompletion();
       }
     }
-    this.interpreter = getBindedInterpreter();
-
+    try {
+      this.interpreter = getBindedInterpreter();
+    } catch (InterpreterNotFoundException e) {
+      return null;
+    }
     setText(buffer);
 
     cursor = calculateCursorPosition(buffer, cursor);
@@ -342,9 +346,27 @@ public class Paragraph extends Job implements Cloneable, JsonSerializable {
     }
 
     clearRuntimeInfo(null);
-    this.interpreter = getBindedInterpreter();
+    try {
+      this.interpreter = getBindedInterpreter();
+      setStatus(Status.READY);
+      if (getConfig().get("enabled") == null || (Boolean) getConfig().get("enabled")) {
+        setAuthenticationInfo(getAuthenticationInfo());
+        interpreter.getScheduler().submit(this);
+      }
 
-    if (interpreter == null) {
+      if (blocking) {
+        while (!getStatus().isCompleted()) {
+          try {
+            Thread.sleep(100);
+          } catch (InterruptedException e) {
+            throw new RuntimeException(e);
+          }
+        }
+        return getStatus() == Status.FINISHED;
+      } else {
+        return true;
+      }
+    } catch (InterpreterNotFoundException e) {
       String intpExceptionMsg =
           getJobName() + "'s Interpreter " + getIntpText() + " not found";
       RuntimeException intpException = new RuntimeException(intpExceptionMsg);
@@ -354,24 +376,6 @@ public class Paragraph extends Job implements Cloneable, JsonSerializable {
       setStatus(Job.Status.ERROR);
       throw intpException;
     }
-    setStatus(Status.READY);
-    if (getConfig().get("enabled") == null || (Boolean) getConfig().get("enabled")) {
-      setAuthenticationInfo(getAuthenticationInfo());
-      interpreter.getScheduler().submit(this);
-    }
-
-    if (blocking) {
-      while (!getStatus().isCompleted()) {
-        try {
-          Thread.sleep(100);
-        } catch (InterruptedException e) {
-          throw new RuntimeException(e);
-        }
-      }
-      return getStatus() == Status.FINISHED;
-    } else {
-      return true;
-    }
   }
 
   @Override
@@ -702,7 +706,11 @@ public class Paragraph extends Job implements Cloneable, JsonSerializable {
   }
 
   public boolean isValidInterpreter(String replName) {
-    return interpreterFactory.getInterpreter(user, note.getId(), replName) != null;
+    try {
+      return interpreterFactory.getInterpreter(user, note.getId(), replName) != null;
+    } catch (InterpreterNotFoundException e) {
+      return false;
+    }
   }
 
   public void updateRuntimeInfos(String label, String tooltip, Map<String, String> infos,

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/e148c8cf/zeppelin-zengine/src/test/java/org/apache/zeppelin/helium/HeliumApplicationFactoryTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/helium/HeliumApplicationFactoryTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/helium/HeliumApplicationFactoryTest.java
index 0952de6..03de533 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/helium/HeliumApplicationFactoryTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/helium/HeliumApplicationFactoryTest.java
@@ -20,6 +20,7 @@ import org.apache.zeppelin.conf.ZeppelinConfiguration;
 import org.apache.zeppelin.interpreter.AbstractInterpreterTest;
 import org.apache.zeppelin.interpreter.Interpreter;
 import org.apache.zeppelin.interpreter.InterpreterException;
+import org.apache.zeppelin.interpreter.InterpreterNotFoundException;
 import org.apache.zeppelin.interpreter.InterpreterResultMessage;
 import org.apache.zeppelin.interpreter.InterpreterSetting;
 import org.apache.zeppelin.notebook.ApplicationState;
@@ -44,6 +45,7 @@ import java.util.LinkedList;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 
 public class HeliumApplicationFactoryTest extends AbstractInterpreterTest implements JobListenerFactory {
@@ -229,8 +231,13 @@ public class HeliumApplicationFactoryTest extends AbstractInterpreterTest implem
     p1.setText("%fake ");
 
     // make sure that p1's repl is null
-    Interpreter intp = p1.getBindedInterpreter();
-    assertEquals(intp, null);
+    Interpreter intp = null;
+    try {
+      intp = p1.getBindedInterpreter();
+      fail("Should throw InterpreterNotFoundException");
+    } catch (InterpreterNotFoundException e) {
+
+    }
 
     // Unbind all interpreter from note
     // NullPointerException shouldn't occur here

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/e148c8cf/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/InterpreterFactoryTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/InterpreterFactoryTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/InterpreterFactoryTest.java
index 760bdf0..2fef05d 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/InterpreterFactoryTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/InterpreterFactoryTest.java
@@ -31,10 +31,14 @@ import static org.junit.Assert.fail;
 public class InterpreterFactoryTest extends AbstractInterpreterTest {
 
   @Test
-  public void testGetFactory() throws IOException {
+  public void testGetFactory() throws IOException, InterpreterException {
     // no default interpreter because there's no interpreter setting binded to this note
-    assertNull(interpreterFactory.getInterpreter("user1", "note1", ""));
+    try {
+      interpreterFactory.getInterpreter("user1", "note1", "");
+      fail("Should throw InterpreterNotFoundException");
+    } catch (InterpreterNotFoundException e) {
 
+    }
     interpreterSettingManager.setInterpreterBinding("user1", "note1", interpreterSettingManager.getSettingIds());
     assertTrue(interpreterFactory.getInterpreter("user1", "note1", "") instanceof RemoteInterpreter);
     RemoteInterpreter remoteInterpreter = (RemoteInterpreter) interpreterFactory.getInterpreter("user1", "note1", "");
@@ -61,7 +65,7 @@ public class InterpreterFactoryTest extends AbstractInterpreterTest {
     try {
       interpreterFactory.getInterpreter("user1", "note1", "test.unknown_repl");
       fail("should fail due to no such interpreter");
-    } catch (RuntimeException e) {
+    } catch (InterpreterNotFoundException e) {
       assertEquals("No such interpreter: test.unknown_repl", e.getMessage());
     }
   }
@@ -72,7 +76,7 @@ public class InterpreterFactoryTest extends AbstractInterpreterTest {
     try {
       interpreterFactory.getInterpreter("user1", "note1", "unknown_repl");
       fail("should fail due to no such interpreter");
-    } catch (RuntimeException e) {
+    } catch (InterpreterNotFoundException e) {
       assertEquals("Either no interpreter named unknown_repl or it is not binded to this note", e.getMessage());
     }
   }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/e148c8cf/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/InterpreterSettingManagerTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/InterpreterSettingManagerTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/InterpreterSettingManagerTest.java
index ec79ada..a52601c 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/InterpreterSettingManagerTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/InterpreterSettingManagerTest.java
@@ -200,7 +200,7 @@ public class InterpreterSettingManagerTest extends AbstractInterpreterTest {
   }
 
   @Test
-  public void testUpdateInterpreterBinding_PerNoteShared() throws IOException {
+  public void testUpdateInterpreterBinding_PerNoteShared() throws IOException, InterpreterNotFoundException {
     InterpreterSetting defaultInterpreterSetting = interpreterSettingManager.get().get(0);
     defaultInterpreterSetting.getOption().setPerNote("shared");
 
@@ -220,7 +220,7 @@ public class InterpreterSettingManagerTest extends AbstractInterpreterTest {
   }
 
   @Test
-  public void testUpdateInterpreterBinding_PerNoteIsolated() throws IOException {
+  public void testUpdateInterpreterBinding_PerNoteIsolated() throws IOException, InterpreterNotFoundException {
     InterpreterSetting defaultInterpreterSetting = interpreterSettingManager.get().get(0);
     defaultInterpreterSetting.getOption().setPerNote("isolated");
 
@@ -241,7 +241,7 @@ public class InterpreterSettingManagerTest extends AbstractInterpreterTest {
   }
 
   @Test
-  public void testUpdateInterpreterBinding_PerNoteScoped() throws IOException {
+  public void testUpdateInterpreterBinding_PerNoteScoped() throws IOException, InterpreterNotFoundException {
     InterpreterSetting defaultInterpreterSetting = interpreterSettingManager.get().get(0);
     defaultInterpreterSetting.getOption().setPerNote("scoped");
 
@@ -266,7 +266,7 @@ public class InterpreterSettingManagerTest extends AbstractInterpreterTest {
   }
 
   @Test
-  public void testGetEditor() throws IOException {
+  public void testGetEditor() throws IOException, InterpreterNotFoundException {
     interpreterSettingManager.setInterpreterBinding("user1", "note1", interpreterSettingManager.getInterpreterSettingIds());
     Interpreter echoInterpreter = interpreterFactory.getInterpreter("user1", "note1", "test.echo");
     // get editor setting from interpreter-setting.json

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/e148c8cf/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NoteTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NoteTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NoteTest.java
index 286affd..824fe10 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NoteTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NoteTest.java
@@ -23,6 +23,7 @@ import org.apache.zeppelin.display.AngularObject;
 import org.apache.zeppelin.display.ui.TextBox;
 import org.apache.zeppelin.interpreter.Interpreter;
 import org.apache.zeppelin.interpreter.InterpreterFactory;
+import org.apache.zeppelin.interpreter.InterpreterNotFoundException;
 import org.apache.zeppelin.interpreter.InterpreterResult;
 import org.apache.zeppelin.interpreter.InterpreterSettingManager;
 import org.apache.zeppelin.notebook.repo.NotebookRepo;
@@ -73,7 +74,7 @@ public class NoteTest {
   private AuthenticationInfo anonymous = new AuthenticationInfo("anonymous");
 
   @Test
-  public void runNormalTest() {
+  public void runNormalTest() throws InterpreterNotFoundException {
     when(interpreterFactory.getInterpreter(anyString(), anyString(), eq("spark"))).thenReturn(interpreter);
     when(interpreter.getScheduler()).thenReturn(scheduler);
 
@@ -101,7 +102,7 @@ public class NoteTest {
   }
 
   @Test
-  public void addParagraphWithLastReplNameTest() {
+  public void addParagraphWithLastReplNameTest() throws InterpreterNotFoundException {
     when(interpreterFactory.getInterpreter(anyString(), anyString(), eq("spark"))).thenReturn(interpreter);
 
     Note note = new Note(repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
@@ -113,7 +114,7 @@ public class NoteTest {
   }
 
   @Test
-  public void insertParagraphWithLastReplNameTest() {
+  public void insertParagraphWithLastReplNameTest() throws InterpreterNotFoundException {
     when(interpreterFactory.getInterpreter(anyString(), anyString(), eq("spark"))).thenReturn(interpreter);
 
     Note note = new Note(repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
@@ -125,7 +126,7 @@ public class NoteTest {
   }
 
   @Test
-  public void insertParagraphWithInvalidReplNameTest() {
+  public void insertParagraphWithInvalidReplNameTest() throws InterpreterNotFoundException {
     when(interpreterFactory.getInterpreter(anyString(), anyString(), eq("invalid"))).thenReturn(null);
 
     Note note = new Note(repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
@@ -144,7 +145,7 @@ public class NoteTest {
   }
 
   @Test
-  public void clearAllParagraphOutputTest() {
+  public void clearAllParagraphOutputTest() throws InterpreterNotFoundException {
     when(interpreterFactory.getInterpreter(anyString(), anyString(), eq("md"))).thenReturn(interpreter);
     when(interpreter.getScheduler()).thenReturn(scheduler);
 

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/e148c8cf/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NotebookTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NotebookTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NotebookTest.java
index 02490ea..122395c 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NotebookTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NotebookTest.java
@@ -26,6 +26,7 @@ import org.apache.zeppelin.interpreter.AbstractInterpreterTest;
 import org.apache.zeppelin.interpreter.InterpreterException;
 import org.apache.zeppelin.interpreter.InterpreterFactory;
 import org.apache.zeppelin.interpreter.InterpreterGroup;
+import org.apache.zeppelin.interpreter.InterpreterNotFoundException;
 import org.apache.zeppelin.interpreter.InterpreterOption;
 import org.apache.zeppelin.interpreter.InterpreterResult;
 import org.apache.zeppelin.interpreter.InterpreterResultMessage;
@@ -145,7 +146,12 @@ public class NotebookTest extends AbstractInterpreterTest implements JobListener
 
     // then interpreter factory should be injected into all the paragraphs
     Note note = notebook.getAllNotes().get(0);
-    assertNull(note.getParagraphs().get(0).getBindedInterpreter());
+    try {
+      note.getParagraphs().get(0).getBindedInterpreter();
+      fail("Should throw InterpreterNotFoundException");
+    } catch (InterpreterNotFoundException e) {
+
+    }
   }
 
   @Test
@@ -523,7 +529,7 @@ public class NotebookTest extends AbstractInterpreterTest implements JobListener
 
 
   @Test
-  public void testAutoRestartInterpreterAfterSchedule() throws InterruptedException, IOException{
+  public void testAutoRestartInterpreterAfterSchedule() throws InterruptedException, IOException, InterpreterNotFoundException {
     // create a note and a paragraph
     Note note = notebook.createNote(anonymous);
     interpreterSettingManager.setInterpreterBinding(anonymous.getUser(), note.getId(), interpreterSettingManager.getInterpreterSettingIds());
@@ -573,7 +579,7 @@ public class NotebookTest extends AbstractInterpreterTest implements JobListener
 
   @Test
   public void testCronWithReleaseResourceClosesOnlySpecificInterpreters()
-          throws IOException, InterruptedException {
+      throws IOException, InterruptedException, InterpreterNotFoundException {
     // create a cron scheduled note.
     Note cronNote = notebook.createNote(anonymous);
     interpreterSettingManager.setInterpreterBinding(anonymous.getUser(), cronNote.getId(),