You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zeppelin.apache.org by pd...@apache.org on 2022/01/21 08:03:48 UTC

[zeppelin] branch master updated: [ZEPPELIN-5559] Add note cache

This is an automated email from the ASF dual-hosted git repository.

pdallig pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/zeppelin.git


The following commit(s) were added to refs/heads/master by this push:
     new f6e8224  [ZEPPELIN-5559] Add note cache
f6e8224 is described below

commit f6e822434bdfefb2ac2f1a89721c3b3fe5695e8f
Author: Philipp Dallig <ph...@gmail.com>
AuthorDate: Mon Jan 17 18:14:29 2022 +0100

    [ZEPPELIN-5559] Add note cache
    
    ### What is this PR for?
    The following changes are made with this PR:
     - a note cache that holds only a certain number of notes in memory
     - a lock for each `Note`
       - No unloading of notes when in use
       - removes methods that return a Note object so that the lock is not bypassed
    
    ### What type of PR is it?
    - Bug Fix/Feature
    
    ### What is the Jira issue?
    * https://issues.apache.org/jira/browse/ZEPPELIN-5559
    
    ### How should this be tested?
    * CI
    
    ### Questions:
    * Does the licenses files need update? No
    * Is there breaking changes for older versions? No
    * Does this needs documentation? Yes, a new configuration option has been added
    
    Author: Philipp Dallig <ph...@gmail.com>
    
    Closes #4252 from Reamer/noteUnloader and squashes the following commits:
    
    cde297672 [Philipp Dallig] Adjustment based on the review
    381b11ac8 [Philipp Dallig] Remove Exception from Listerner
    9e76337ee [Philipp Dallig] Add note cache
---
 docs/setup/operation/configuration.md              |    6 +
 .../zeppelin/integration/FlinkIntegrationTest.java |    6 -
 .../zeppelin/integration/ShellIntegrationTest.java |   62 +-
 .../zeppelin/integration/SparkIntegrationTest.java |    1 -
 .../integration/ZSessionIntegrationTest.java       |   42 +-
 .../integration/ZeppelinClientIntegrationTest.java |   58 +-
 .../integration/ZeppelinFlinkClusterTest.java      |  176 ++-
 .../integration/ZeppelinSparkClusterTest.java      | 1670 +++++++++++---------
 .../zeppelin/conf/ZeppelinConfiguration.java       |    5 +
 .../org/apache/zeppelin/rest/HeliumRestApi.java    |   75 +-
 .../org/apache/zeppelin/rest/NotebookRestApi.java  |  508 +++---
 .../org/apache/zeppelin/server/ZeppelinServer.java |    4 +
 .../apache/zeppelin/service/JobManagerService.java |   38 +-
 .../apache/zeppelin/service/NotebookService.java   | 1122 +++++++------
 .../zeppelin/service/SessionManagerService.java    |    4 +-
 .../org/apache/zeppelin/socket/NotebookServer.java |  577 ++++---
 .../apache/zeppelin/cluster/ClusterEventTest.java  |   80 +-
 .../org/apache/zeppelin/recovery/RecoveryTest.java |  197 ++-
 .../zeppelin/rest/InterpreterRestApiTest.java      |  154 +-
 .../apache/zeppelin/rest/NotebookRestApiTest.java  |  494 +++---
 .../zeppelin/rest/NotebookSecurityRestApiTest.java |   21 +-
 .../apache/zeppelin/rest/ZeppelinRestApiTest.java  |  797 ++++++----
 .../zeppelin/service/NotebookServiceTest.java      |  233 ++-
 .../apache/zeppelin/socket/NotebookServerTest.java |  403 ++---
 .../zeppelin/helium/HeliumApplicationFactory.java  |   47 +-
 .../zeppelin/interpreter/InterpreterSetting.java   |   19 +-
 .../interpreter/InterpreterSettingManager.java     |   48 +-
 .../interpreter/RemoteInterpreterEventServer.java  |   48 +-
 .../java/org/apache/zeppelin/notebook/Note.java    |   75 +-
 .../zeppelin/notebook/NoteEventAsyncListener.java  |  120 +-
 .../zeppelin/notebook/NoteEventListener.java       |   16 +-
 .../org/apache/zeppelin/notebook/NoteManager.java  |  329 ++--
 .../org/apache/zeppelin/notebook/Notebook.java     |  360 +++--
 .../org/apache/zeppelin/notebook/Paragraph.java    |    9 +-
 .../zeppelin/notebook/scheduler/CronJob.java       |   63 +-
 .../notebook/scheduler/CronJobListener.java        |   48 +-
 .../notebook/scheduler/QuartzSchedulerService.java |  139 +-
 .../notebook/scheduler/SchedulerService.java       |    3 +-
 .../org/apache/zeppelin/search/LuceneSearch.java   |  133 +-
 .../apache/zeppelin/search/NoSearchService.java    |   36 +-
 .../org/apache/zeppelin/search/SearchService.java  |   47 +-
 .../helium/HeliumApplicationFactoryTest.java       |   50 +-
 .../interpreter/AbstractInterpreterTest.java       |   18 +-
 .../interpreter/InterpreterSettingManagerTest.java |   61 +-
 .../interpreter/InterpreterSettingTest.java        |  139 +-
 .../recovery/FileSystemRecoveryStorageTest.java    |   20 +-
 .../recovery/LocalRecoveryStorageTest.java         |   23 +-
 .../remote/RemoteAngularObjectTest.java            |   12 +-
 .../interpreter/remote/RemoteInterpreterTest.java  |   54 +-
 .../apache/zeppelin/notebook/NoteManagerTest.java  |   56 +-
 .../org/apache/zeppelin/notebook/NotebookTest.java | 1597 +++++++++++--------
 .../notebook/repo/NotebookRepoSyncTest.java        |  135 +-
 .../resource/DistributedResourcePoolTest.java      |   31 +-
 .../zeppelin/scheduler/RemoteSchedulerTest.java    |   13 +-
 .../apache/zeppelin/search/LuceneSearchTest.java   |  146 +-
 55 files changed, 6051 insertions(+), 4577 deletions(-)

diff --git a/docs/setup/operation/configuration.md b/docs/setup/operation/configuration.md
index 5627cdd..026581d 100644
--- a/docs/setup/operation/configuration.md
+++ b/docs/setup/operation/configuration.md
@@ -463,6 +463,12 @@ Sources descending by priority:
     <td></td>
     <td>comma-separated list of folder, where cron is allowed</td>
   </tr>
+  <tr>
+    <td><h6 class="properties">ZEPPELIN_NOTE_CACHE_THRESHOLD</h6></td>
+    <td><h6 class="properties">zeppelin.note.cache.threshold</h6></td>
+    <td>50</td>
+    <td>Threshold for the number of notes in the cache before an eviction occurs.</td>
+  </tr>
 </table>
 
 
diff --git a/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/FlinkIntegrationTest.java b/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/FlinkIntegrationTest.java
index 2dc14f8..7240181 100644
--- a/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/FlinkIntegrationTest.java
+++ b/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/FlinkIntegrationTest.java
@@ -28,7 +28,6 @@ import org.apache.zeppelin.interpreter.Interpreter;
 import org.apache.zeppelin.interpreter.InterpreterContext;
 import org.apache.zeppelin.interpreter.InterpreterException;
 import org.apache.zeppelin.interpreter.InterpreterFactory;
-import org.apache.zeppelin.interpreter.InterpreterOutput;
 import org.apache.zeppelin.interpreter.InterpreterResult;
 import org.apache.zeppelin.interpreter.InterpreterSetting;
 import org.apache.zeppelin.interpreter.InterpreterSettingManager;
@@ -36,16 +35,11 @@ import org.apache.zeppelin.interpreter.integration.DownloadUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.EnumSet;
-import java.util.List;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
diff --git a/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ShellIntegrationTest.java b/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ShellIntegrationTest.java
index bfeda40..823c2ed 100644
--- a/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ShellIntegrationTest.java
+++ b/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ShellIntegrationTest.java
@@ -18,7 +18,6 @@
 package org.apache.zeppelin.integration;
 
 import org.apache.zeppelin.conf.ZeppelinConfiguration;
-import org.apache.zeppelin.notebook.Note;
 import org.apache.zeppelin.notebook.Notebook;
 import org.apache.zeppelin.notebook.Paragraph;
 import org.apache.zeppelin.rest.AbstractTestRestApi;
@@ -51,40 +50,45 @@ public class ShellIntegrationTest extends AbstractTestRestApi {
 
   @Test
   public void testBasicShell() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", AuthenticationInfo.ANONYMOUS);
-      Paragraph p = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", AuthenticationInfo.ANONYMOUS);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
 
-      // test correct shell command
-      p.setText("%sh echo 'hello world'");
-      note.run(p.getId(), true);
-      assertEquals(Job.Status.FINISHED, p.getStatus());
-      assertEquals("hello world\n", p.getReturn().message().get(0).getData());
+          // test correct shell command
+          p.setText("%sh echo 'hello world'");
+          note.run(p.getId(), true);
+          assertEquals(Job.Status.FINISHED, p.getStatus());
+          assertEquals("hello world\n", p.getReturn().message().get(0).getData());
 
-      // test invalid shell command
-      p.setText("%sh invalid_cmd");
-      note.run(p.getId(), true);
-      assertEquals(Job.Status.ERROR, p.getStatus());
-      assertTrue(p.getReturn().toString(),
-              p.getReturn().message().get(0).getData().contains("command not found"));
+          // test invalid shell command
+          p.setText("%sh invalid_cmd");
+          note.run(p.getId(), true);
+          assertEquals(Job.Status.ERROR, p.getStatus());
+          assertTrue(p.getReturn().toString(),
+                  p.getReturn().message().get(0).getData().contains("command not found"));
 
-      // test shell environment variable
-      p.setText("%sh a='hello world'\n" +
-              "echo ${a}");
-      note.run(p.getId(), true);
-      assertEquals(Job.Status.FINISHED, p.getStatus());
-      assertEquals("hello world\n", p.getReturn().message().get(0).getData());
+          // test shell environment variable
+          p.setText("%sh a='hello world'\n" +
+                  "echo ${a}");
+          note.run(p.getId(), true);
+          assertEquals(Job.Status.FINISHED, p.getStatus());
+          assertEquals("hello world\n", p.getReturn().message().get(0).getData());
+
+          // use dynamic form via local property
+          p.setText("%sh(form=simple) a='hello world'\n" +
+                  "echo ${a}");
+          note.run(p.getId(), true);
+          assertEquals(Job.Status.FINISHED, p.getStatus());
+          assertEquals(0, p.getReturn().message().size());
+          return null;
+        });
 
-      // use dynamic form via local property
-      p.setText("%sh(form=simple) a='hello world'\n" +
-              "echo ${a}");
-      note.run(p.getId(), true);
-      assertEquals(Job.Status.FINISHED, p.getStatus());
-      assertEquals(0, p.getReturn().message().size());
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, AuthenticationInfo.ANONYMOUS);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, AuthenticationInfo.ANONYMOUS);
       }
     }
   }
diff --git a/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/SparkIntegrationTest.java b/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/SparkIntegrationTest.java
index 2c4c278..647f844 100644
--- a/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/SparkIntegrationTest.java
+++ b/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/SparkIntegrationTest.java
@@ -31,7 +31,6 @@ import org.apache.zeppelin.interpreter.Interpreter;
 import org.apache.zeppelin.interpreter.InterpreterContext;
 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.InterpreterResult;
 import org.apache.zeppelin.interpreter.InterpreterSetting;
diff --git a/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZSessionIntegrationTest.java b/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZSessionIntegrationTest.java
index 18d5b73..88f0f88 100644
--- a/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZSessionIntegrationTest.java
+++ b/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZSessionIntegrationTest.java
@@ -26,7 +26,6 @@ import org.apache.zeppelin.client.ZSession;
 import org.apache.zeppelin.conf.ZeppelinConfiguration;
 import org.apache.zeppelin.interpreter.integration.DownloadUtils;
 import org.apache.zeppelin.interpreter.lifecycle.TimeoutLifecycleManager;
-import org.apache.zeppelin.notebook.Note;
 import org.apache.zeppelin.notebook.Notebook;
 import org.apache.zeppelin.rest.AbstractTestRestApi;
 import org.apache.zeppelin.utils.TestUtils;
@@ -89,9 +88,13 @@ public class ZSessionIntegrationTest extends AbstractTestRestApi {
       assertNull(session.getWeburl());
       assertNotNull(session.getNoteId());
 
-      Note note = notebook.getNote(session.getNoteId());
-      assertEquals(2, note.getParagraphCount());
-      assertTrue(note.getParagraph(0).getText(), note.getParagraph(0).getText().startsWith("%sh.conf"));
+      notebook.processNote(session.getNoteId(),
+        note -> {
+          assertEquals(2, note.getParagraphCount());
+          assertTrue(note.getParagraph(0).getText(), note.getParagraph(0).getText().startsWith("%sh.conf"));
+          return null;
+        });
+
 
       ExecuteResult result = session.execute("pwd");
       assertEquals(result.toString(), Status.FINISHED, result.getStatus());
@@ -106,9 +109,12 @@ public class ZSessionIntegrationTest extends AbstractTestRestApi {
       assertEquals("TEXT", result.getResults().get(1).getType());
       assertTrue(result.getResults().get(1).getData(), result.getResults().get(1).getData().contains("ExitValue"));
 
-      assertEquals(4, note.getParagraphCount());
-      assertEquals("%sh invalid_command", note.getParagraph(3).getText());
-
+      notebook.processNote(session.getNoteId(),
+        note -> {
+          assertEquals(4, note.getParagraphCount());
+          assertEquals("%sh invalid_command", note.getParagraph(3).getText());
+          return null;
+        });
     } finally {
       session.stop();
     }
@@ -126,9 +132,12 @@ public class ZSessionIntegrationTest extends AbstractTestRestApi {
       assertNull(session.getWeburl());
       assertNotNull(session.getNoteId());
 
-      Note note = notebook.getNote(session.getNoteId());
-      assertEquals(2, note.getParagraphCount());
-      assertTrue(note.getParagraph(0).getText(), note.getParagraph(0).getText().startsWith("%sh.conf"));
+      notebook.processNote(session.getNoteId(),
+        note -> {
+          assertEquals(2, note.getParagraphCount());
+          assertTrue(note.getParagraph(0).getText(), note.getParagraph(0).getText().startsWith("%sh.conf"));
+          return null;
+        });
 
       ExecuteResult result = session.submit("sleep 10\npwd");
       assertFalse("Status is: " + result.getStatus().toString(), result.getStatus().isCompleted());
@@ -146,9 +155,12 @@ public class ZSessionIntegrationTest extends AbstractTestRestApi {
       assertEquals("TEXT", result.getResults().get(1).getType());
       assertTrue(result.getResults().get(1).getData(), result.getResults().get(1).getData().contains("ExitValue"));
 
-      assertEquals(4, note.getParagraphCount());
-      assertEquals("%sh invalid_command", note.getParagraph(3).getText());
-
+      notebook.processNote(session.getNoteId(),
+        note -> {
+          assertEquals(4, note.getParagraphCount());
+          assertEquals("%sh invalid_command", note.getParagraph(3).getText());
+          return null;
+        });
     } finally {
       session.stop();
     }
@@ -456,10 +468,10 @@ public class ZSessionIntegrationTest extends AbstractTestRestApi {
       assertNull(session.getWeburl());
       assertNotNull(session.getNoteId());
 
-      assertTrue(notebook.getAllNotes().size() > 0);
+      assertTrue(notebook.getNotesInfo().size() > 0);
 
       Thread.sleep(30 * 1000);
-      assertEquals(0, notebook.getAllNotes().size());
+      assertEquals(0, notebook.getNotesInfo().size());
 
       try {
         session.execute("1/0");
diff --git a/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZeppelinClientIntegrationTest.java b/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZeppelinClientIntegrationTest.java
index 5cc5a74..af3507e 100644
--- a/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZeppelinClientIntegrationTest.java
+++ b/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZeppelinClientIntegrationTest.java
@@ -27,7 +27,6 @@ import org.apache.zeppelin.client.ZeppelinClient;
 
 import org.apache.zeppelin.conf.ZeppelinConfiguration;
 import org.apache.zeppelin.common.SessionInfo;
-import org.apache.zeppelin.notebook.Note;
 import org.apache.zeppelin.notebook.Notebook;
 import org.apache.zeppelin.rest.AbstractTestRestApi;
 import org.apache.zeppelin.utils.TestUtils;
@@ -110,7 +109,11 @@ public class ZeppelinClientIntegrationTest extends AbstractTestRestApi {
   @Test
   public void testNoteOperation() throws Exception {
     String noteId = zeppelinClient.createNote("/project_1/note1");
-    assertNotNull(notebook.getNote(noteId));
+    notebook.processNote(noteId,
+      note -> {
+        assertNotNull(note);
+        return null;
+      });
 
     // create duplicated note
     try {
@@ -149,34 +152,53 @@ public class ZeppelinClientIntegrationTest extends AbstractTestRestApi {
   @Test
   public void testCloneNote() throws Exception {
     String noteId = zeppelinClient.createNote("/clone_note_test/note1");
-    Note note1 = notebook.getNote(noteId);
-    assertNotNull(note1);
-
+    notebook.processNote(noteId,
+      note1 -> {
+        assertNotNull(note1);
+        return null;
+      });
     zeppelinClient.addParagraph(noteId, "title_1", "text_1");
-    assertEquals(1, note1.getParagraphCount());
+    notebook.processNote(noteId,
+      note1 -> {
+        assertEquals(1, note1.getParagraphCount());
+        return null;
+      });
 
     String clonedNoteId = zeppelinClient.cloneNote(noteId, "/clone_note_test/cloned_note1");
-    Note clonedNote = notebook.getNote(clonedNoteId);
-    assertEquals(1, clonedNote.getParagraphCount());
-    assertEquals("title_1", clonedNote.getParagraph(0).getTitle());
-    assertEquals("text_1", clonedNote.getParagraph(0).getText());
+    notebook.processNote(clonedNoteId,
+      clonedNote -> {
+        assertEquals(1, clonedNote.getParagraphCount());
+        assertEquals("title_1", clonedNote.getParagraph(0).getTitle());
+        assertEquals("text_1", clonedNote.getParagraph(0).getText());
+        return null;
+      });
   }
 
   @Test
   public void testRenameNote() throws Exception {
     String noteId = zeppelinClient.createNote("/rename_note_test/note1");
-    Note note1 = notebook.getNote(noteId);
-    assertNotNull(note1);
+    notebook.processNote(noteId,
+      note1 -> {
+        assertNotNull(note1);
+        return null;
+      });
 
     zeppelinClient.addParagraph(noteId, "title_1", "text_1");
-    assertEquals(1, note1.getParagraphCount());
+    notebook.processNote(noteId,
+      note1 -> {
+        assertEquals(1, note1.getParagraphCount());
+        return null;
+      });
 
     zeppelinClient.renameNote(noteId, "/rename_note_test/note1_renamed");
-    Note renamedNote = notebook.getNote(noteId);
-    assertEquals("/rename_note_test/note1_renamed", renamedNote.getPath());
-    assertEquals(1, renamedNote.getParagraphCount());
-    assertEquals("title_1", renamedNote.getParagraph(0).getTitle());
-    assertEquals("text_1", renamedNote.getParagraph(0).getText());
+    notebook.processNote(noteId,
+      renamedNote -> {
+        assertEquals("/rename_note_test/note1_renamed", renamedNote.getPath());
+        assertEquals(1, renamedNote.getParagraphCount());
+        assertEquals("title_1", renamedNote.getParagraph(0).getTitle());
+        assertEquals("text_1", renamedNote.getParagraph(0).getText());
+        return null;
+      });
   }
 
   @Test
diff --git a/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZeppelinFlinkClusterTest.java b/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZeppelinFlinkClusterTest.java
index f43c34b..28a9eda 100644
--- a/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZeppelinFlinkClusterTest.java
+++ b/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZeppelinFlinkClusterTest.java
@@ -29,7 +29,6 @@ import org.apache.zeppelin.user.AuthenticationInfo;
 import org.apache.zeppelin.utils.TestUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,6 +38,7 @@ import java.nio.file.Files;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public abstract class ZeppelinFlinkClusterTest extends AbstractTestRestApi {
 
@@ -69,56 +69,65 @@ public abstract class ZeppelinFlinkClusterTest extends AbstractTestRestApi {
   //@Test
   public void testResumeFromCheckpoint() throws Exception {
 
-    Note note = null;
+    String noteId = null;
     try {
       // create new note
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", AuthenticationInfo.ANONYMOUS);
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", AuthenticationInfo.ANONYMOUS);
 
       // run p0 for %flink.conf
       String checkpointPath = Files.createTempDirectory("checkpoint").toAbsolutePath().toString();
-      Paragraph p0 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      StringBuilder builder = new StringBuilder("%flink.conf\n");
-      builder.append("FLINK_HOME " + flinkHome + "\n");
-      builder.append("flink.execution.mode local\n");
-      builder.append("state.checkpoints.dir file://" + checkpointPath + "\n");
-      builder.append("execution.checkpointing.externalized-checkpoint-retention RETAIN_ON_CANCELLATION");
-      p0.setText(builder.toString());
-      note.run(p0.getId(), true);
-      assertEquals(Job.Status.FINISHED, p0.getStatus());
-
-      // run p1 for creating flink table via scala
-      Paragraph p1 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p1.setText("%flink " + getInitStreamScript(2000));
-      note.run(p1.getId(), true);
-      assertEquals(Job.Status.FINISHED, p0.getStatus());
-
-      // run p2 for flink streaming sql
-      Paragraph p2 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p2.setText("%flink.ssql(type=single, template=<h1>Total: {0}</h1>, resumeFromLatestCheckpoint=true)\n" +
-              "select count(1) from log;");
-      note.run(p2.getId(), false);
-      p2.waitUntilRunning();
-
-      Thread.sleep(60 * 1000);
-      p2.abort();
-
-      // Sleep 5 seconds to ensure checkpoint info is written to note file
-      Thread.sleep(5 * 1000);
-      assertTrue(p2.getConfig().toString(), p2.getConfig().get("latest_checkpoint_path").toString().contains(checkpointPath));
-
-      // run it again
-      note.run(p0.getId(), true);
-      note.run(p1.getId(), true);
-      note.run(p2.getId(), false);
-      p2.waitUntilFinished();
-      assertEquals(p2.getReturn().toString(), Job.Status.FINISHED, p2.getStatus());
-
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p0 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          StringBuilder builder = new StringBuilder("%flink.conf\n");
+          builder.append("FLINK_HOME " + flinkHome + "\n");
+          builder.append("flink.execution.mode local\n");
+          builder.append("state.checkpoints.dir file://" + checkpointPath + "\n");
+          builder.append("execution.checkpointing.externalized-checkpoint-retention RETAIN_ON_CANCELLATION");
+          p0.setText(builder.toString());
+          note.run(p0.getId(), true);
+          assertEquals(Job.Status.FINISHED, p0.getStatus());
+
+          // run p1 for creating flink table via scala
+          Paragraph p1 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p1.setText("%flink " + getInitStreamScript(2000));
+          note.run(p1.getId(), true);
+          assertEquals(Job.Status.FINISHED, p0.getStatus());
+
+          // run p2 for flink streaming sql
+          Paragraph p2 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p2.setText("%flink.ssql(type=single, template=<h1>Total: {0}</h1>, resumeFromLatestCheckpoint=true)\n" +
+                  "select count(1) from log;");
+          note.run(p2.getId(), false);
+          try {
+            p2.waitUntilRunning();
+            Thread.sleep(60 * 1000);
+            p2.abort();
+            // Sleep 5 seconds to ensure checkpoint info is written to note file
+            Thread.sleep(5 * 1000);
+            assertTrue(p2.getConfig().toString(), p2.getConfig().get("latest_checkpoint_path").toString().contains(checkpointPath));
+          } catch (InterruptedException e) {
+            fail();
+          }
+
+          // run it again
+          note.run(p0.getId(), true);
+          note.run(p1.getId(), true);
+          note.run(p2.getId(), false);
+          try {
+            p2.waitUntilFinished();
+          } catch (InterruptedException e) {
+            fail();
+          }
+          assertEquals(p2.getReturn().toString(), Job.Status.FINISHED, p2.getStatus());
+          return null;
+        });
     } catch (Exception e) {
       e.printStackTrace();
       throw e;
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, AuthenticationInfo.ANONYMOUS);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, AuthenticationInfo.ANONYMOUS);
       }
     }
   }
@@ -126,50 +135,63 @@ public abstract class ZeppelinFlinkClusterTest extends AbstractTestRestApi {
   //@Test
   public void testResumeFromInvalidCheckpoint() throws Exception {
 
-    Note note = null;
+    String noteId = null;
     try {
       // create new note
-      note = TestUtils.getInstance(Notebook.class).createNote("note2", AuthenticationInfo.ANONYMOUS);
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note2", AuthenticationInfo.ANONYMOUS);
 
       // run p0 for %flink.conf
       String checkpointPath = Files.createTempDirectory("checkpoint").toAbsolutePath().toString();
-      Paragraph p0 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      StringBuilder builder = new StringBuilder("%flink.conf\n");
-      builder.append("FLINK_HOME " + flinkHome + "\n");
-      builder.append("flink.execution.mode local\n");
-      builder.append("state.checkpoints.dir file://" + checkpointPath + "\n");
-      builder.append("execution.checkpointing.externalized-checkpoint-retention RETAIN_ON_CANCELLATION");
-      p0.setText(builder.toString());
-      note.run(p0.getId(), true);
-      assertEquals(Job.Status.FINISHED, p0.getStatus());
-
-      // run p1 for creating flink table via scala
-      Paragraph p1 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p1.setText("%flink " + getInitStreamScript(500));
-      note.run(p1.getId(), true);
-      assertEquals(Job.Status.FINISHED, p0.getStatus());
-
-      // run p2 for flink streaming sql
-      Paragraph p2 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p2.setText("%flink.ssql(type=single, template=<h1>Total: {0}</h1>, resumeFromLatestCheckpoint=true)\n" +
-              "select count(1) from log;");
-      p2.getConfig().put("latest_checkpoint_path", "file:///invalid_checkpoint");
-      note.run(p2.getId(), false);
-      p2.waitUntilFinished();
-      assertEquals(p2.getReturn().toString(), Job.Status.ERROR, p2.getStatus());
-      assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("Cannot find checkpoint"));
-
-      p2.setText("%flink.ssql(type=single, template=<h1>Total: {0}</h1>, resumeFromLatestCheckpoint=false)\n" +
-              "select count(1) from log;");
-      note.run(p2.getId(), false);
-      p2.waitUntilFinished();
-      assertEquals(p2.getReturn().toString(), Job.Status.FINISHED, p2.getStatus());
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p0 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          StringBuilder builder = new StringBuilder("%flink.conf\n");
+          builder.append("FLINK_HOME " + flinkHome + "\n");
+          builder.append("flink.execution.mode local\n");
+          builder.append("state.checkpoints.dir file://" + checkpointPath + "\n");
+          builder.append("execution.checkpointing.externalized-checkpoint-retention RETAIN_ON_CANCELLATION");
+          p0.setText(builder.toString());
+          note.run(p0.getId(), true);
+          assertEquals(Job.Status.FINISHED, p0.getStatus());
+
+          // run p1 for creating flink table via scala
+          Paragraph p1 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p1.setText("%flink " + getInitStreamScript(500));
+          note.run(p1.getId(), true);
+          assertEquals(Job.Status.FINISHED, p0.getStatus());
+
+          // run p2 for flink streaming sql
+          Paragraph p2 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p2.setText("%flink.ssql(type=single, template=<h1>Total: {0}</h1>, resumeFromLatestCheckpoint=true)\n" +
+                  "select count(1) from log;");
+          p2.getConfig().put("latest_checkpoint_path", "file:///invalid_checkpoint");
+          note.run(p2.getId(), false);
+          try {
+            p2.waitUntilFinished();
+          } catch (InterruptedException e) {
+            fail();
+          }
+          assertEquals(p2.getReturn().toString(), Job.Status.ERROR, p2.getStatus());
+          assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("Cannot find checkpoint"));
+
+          p2.setText("%flink.ssql(type=single, template=<h1>Total: {0}</h1>, resumeFromLatestCheckpoint=false)\n" +
+                  "select count(1) from log;");
+          note.run(p2.getId(), false);
+          try {
+            p2.waitUntilFinished();
+          } catch (InterruptedException e) {
+            fail();
+          }
+          assertEquals(p2.getReturn().toString(), Job.Status.FINISHED, p2.getStatus());
+          return null;
+        });
+
     } catch (Exception e) {
       e.printStackTrace();
       throw e;
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, AuthenticationInfo.ANONYMOUS);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, AuthenticationInfo.ANONYMOUS);
       }
     }
   }
diff --git a/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZeppelinSparkClusterTest.java b/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZeppelinSparkClusterTest.java
index 9246bf5..24a7be6 100644
--- a/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZeppelinSparkClusterTest.java
+++ b/zeppelin-interpreter-integration/src/test/java/org/apache/zeppelin/integration/ZeppelinSparkClusterTest.java
@@ -59,6 +59,7 @@ import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Test against spark cluster.
@@ -154,102 +155,114 @@ public abstract class ZeppelinSparkClusterTest extends AbstractTestRestApi {
 
   @Test
   public void scalaOutputTest() throws IOException, InterruptedException {
-    Note note = null;
+    String noteId = null;
     try {
       // create new note
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p = note.addNewParagraph(anonymous);
-      p.setText("%spark import java.util.Date\n" +
-          "import java.net.URL\n" +
-          "println(\"hello\")\n"
-      );
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertEquals("hello\n" +
-          "import java.util.Date\n" +
-          "import java.net.URL\n",
-          p.getReturn().message().get(0).getData());
-
-      // check spark weburl in zeppelin-server side
-      InterpreterSettingManager interpreterSettingManager = TestUtils.getInstance(InterpreterSettingManager.class);
-      InterpreterSetting sparkInterpreterSetting = interpreterSettingManager.getByName("spark");
-      assertEquals(1, sparkInterpreterSetting.getAllInterpreterGroups().size());
-      assertNotNull(sparkInterpreterSetting.getAllInterpreterGroups().get(0).getWebUrl());
-
-      p.setText("%spark invalid_code");
-      note.run(p.getId(), true);
-      assertEquals(Status.ERROR, p.getStatus());
-      assertTrue(p.getReturn().message().get(0).getData().contains("error: "));
-
-      // test local properties
-      p.setText("%spark(p1=v1,p2=v2) print(z.getInterpreterContext().getLocalProperties().size())");
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertEquals("2", p.getReturn().message().get(0).getData());
-
-      // test code completion
-      List<InterpreterCompletion> completions = note.completion(p.getId(), "sc.", 2, AuthenticationInfo.ANONYMOUS);
-      assertTrue(completions.size() > 0);
-
-      // test cancel
-      p.setText("%spark sc.range(1,10).map(e=>{Thread.sleep(1000); e}).collect()");
-      note.run(p.getId(), false);
-      waitForRunning(p);
-      p.abort();
-      waitForFinish(p);
-      assertEquals(Status.ABORT, p.getStatus());
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(anonymous);
+          p.setText("%spark import java.util.Date\n" +
+              "import java.net.URL\n" +
+              "println(\"hello\")\n"
+          );
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertEquals("hello\n" +
+              "import java.util.Date\n" +
+              "import java.net.URL\n",
+              p.getReturn().message().get(0).getData());
+
+          // check spark weburl in zeppelin-server side
+          InterpreterSettingManager interpreterSettingManager = TestUtils.getInstance(InterpreterSettingManager.class);
+          InterpreterSetting sparkInterpreterSetting = interpreterSettingManager.getByName("spark");
+          assertEquals(1, sparkInterpreterSetting.getAllInterpreterGroups().size());
+          assertNotNull(sparkInterpreterSetting.getAllInterpreterGroups().get(0).getWebUrl());
+
+          p.setText("%spark invalid_code");
+          note.run(p.getId(), true);
+          assertEquals(Status.ERROR, p.getStatus());
+          assertTrue(p.getReturn().message().get(0).getData().contains("error: "));
+
+          // test local properties
+          p.setText("%spark(p1=v1,p2=v2) print(z.getInterpreterContext().getLocalProperties().size())");
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertEquals("2", p.getReturn().message().get(0).getData());
+
+          // test code completion
+          List<InterpreterCompletion> completions = note.completion(p.getId(), "sc.", 2, AuthenticationInfo.ANONYMOUS);
+          assertTrue(completions.size() > 0);
+
+          // test cancel
+          p.setText("%spark sc.range(1,10).map(e=>{Thread.sleep(1000); e}).collect()");
+          note.run(p.getId(), false);
+          waitForRunning(p);
+          p.abort();
+          waitForFinish(p);
+          assertEquals(Status.ABORT, p.getStatus());
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void basicRDDTransformationAndActionTest() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p = note.addNewParagraph(anonymous);
-      p.setText("%spark print(sc.parallelize(1 to 10).reduce(_ + _))");
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertEquals("55", p.getReturn().message().get(0).getData());
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(anonymous);
+          p.setText("%spark print(sc.parallelize(1 to 10).reduce(_ + _))");
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertEquals("55", p.getReturn().message().get(0).getData());
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void sparkReadJSONTest() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p = note.addNewParagraph(anonymous);
-      File tmpJsonFile = File.createTempFile("test", ".json");
-      FileWriter jsonFileWriter = new FileWriter(tmpJsonFile);
-      IOUtils.copy(new StringReader("{\"metadata\": { \"key\": 84896, \"value\": 54 }}\n"),
-              jsonFileWriter);
-      jsonFileWriter.close();
-      if (isSpark2() || isSpark3()) {
-        p.setText("%spark spark.read.json(\"file://" + tmpJsonFile.getAbsolutePath() + "\")");
-      } else {
-        p.setText("%spark sqlContext.read.json(\"file://" + tmpJsonFile.getAbsolutePath() + "\")");
-      }
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-      if (isSpark2() || isSpark3()) {
-        assertTrue(p.getReturn().message().get(0).getData().contains(
-                "org.apache.spark.sql.DataFrame = [metadata: struct<key: bigint, value: bigint>]"));
-      } else {
-        assertTrue(p.getReturn().message().get(0).getData().contains(
-                "org.apache.spark.sql.DataFrame = [metadata: struct<key:bigint,value:bigint>]"));
-      }
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(anonymous);
+          File tmpJsonFile = File.createTempFile("test", ".json");
+          FileWriter jsonFileWriter = new FileWriter(tmpJsonFile);
+          IOUtils.copy(new StringReader("{\"metadata\": { \"key\": 84896, \"value\": 54 }}\n"),
+                  jsonFileWriter);
+          jsonFileWriter.close();
+          if (isSpark2() || isSpark3()) {
+            p.setText("%spark spark.read.json(\"file://" + tmpJsonFile.getAbsolutePath() + "\")");
+          } else {
+            p.setText("%spark sqlContext.read.json(\"file://" + tmpJsonFile.getAbsolutePath() + "\")");
+          }
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+          if (isSpark2() || isSpark3()) {
+            assertTrue(p.getReturn().message().get(0).getData().contains(
+                    "org.apache.spark.sql.DataFrame = [metadata: struct<key: bigint, value: bigint>]"));
+          } else {
+            assertTrue(p.getReturn().message().get(0).getData().contains(
+                    "org.apache.spark.sql.DataFrame = [metadata: struct<key:bigint,value:bigint>]"));
+          }
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
@@ -261,152 +274,164 @@ public abstract class ZeppelinSparkClusterTest extends AbstractTestRestApi {
       return;
     }
 
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p = note.addNewParagraph(anonymous);
-      File tmpCSVFile = File.createTempFile("test", ".csv");
-      FileWriter csvFileWriter = new FileWriter(tmpCSVFile);
-      IOUtils.copy(new StringReader("84896,54"), csvFileWriter);
-      csvFileWriter.close();
-      p.setText("%spark spark.read.csv(\"file://" + tmpCSVFile.getAbsolutePath() + "\")");
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertTrue(p.getReturn().message().get(0).getData().contains(
-              "org.apache.spark.sql.DataFrame = [_c0: string, _c1: string]\n"));
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(anonymous);
+          File tmpCSVFile = File.createTempFile("test", ".csv");
+          FileWriter csvFileWriter = new FileWriter(tmpCSVFile);
+          IOUtils.copy(new StringReader("84896,54"), csvFileWriter);
+          csvFileWriter.close();
+          p.setText("%spark spark.read.csv(\"file://" + tmpCSVFile.getAbsolutePath() + "\")");
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertTrue(p.getReturn().message().get(0).getData().contains(
+                  "org.apache.spark.sql.DataFrame = [_c0: string, _c1: string]\n"));
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void sparkSQLTest() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      // test basic dataframe api
-      Paragraph p = note.addNewParagraph(anonymous);
-      if (isSpark2() || isSpark3()) {
-        p.setText("%spark val df=spark.createDataFrame(Seq((\"hello\",20)))" +
-                ".toDF(\"name\", \"age\")\n" +
-                "df.collect()");
-      } else {
-        p.setText("%spark val df=sqlContext.createDataFrame(Seq((\"hello\",20)))" +
-                ".toDF(\"name\", \"age\")\n" +
-                "df.collect()");
-      }
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertTrue(p.getReturn().message().get(0).getData().contains(
-              "Array[org.apache.spark.sql.Row] = Array([hello,20])"));
-
-      // test display DataFrame
-      p = note.addNewParagraph(anonymous);
-      if (isSpark2() || isSpark3()) {
-        p.setText("%spark val df=spark.createDataFrame(Seq((\"hello\",20)))" +
-                ".toDF(\"name\", \"age\")\n" +
-                "df.createOrReplaceTempView(\"test_table\")\n" +
-                "z.show(df)");
-      } else {
-        p.setText("%spark val df=sqlContext.createDataFrame(Seq((\"hello\",20)))" +
-                ".toDF(\"name\", \"age\")\n" +
-                "df.registerTempTable(\"test_table\")\n" +
-                "z.show(df)");
-      }
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertEquals(InterpreterResult.Type.TABLE, p.getReturn().message().get(0).getType());
-      assertEquals("name\tage\nhello\t20\n", p.getReturn().message().get(0).getData());
-
-      // run sql and save it into resource pool
-      p = note.addNewParagraph(anonymous);
-      p.setText("%spark.sql(saveAs=table_result) select * from test_table");
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertEquals(InterpreterResult.Type.TABLE, p.getReturn().message().get(0).getType());
-      assertEquals("name\tage\nhello\t20\n", p.getReturn().message().get(0).getData());
-
-      // get resource from spark
-      p = note.addNewParagraph(anonymous);
-      p.setText("%spark val df=z.getAsDataFrame(\"table_result\")\nz.show(df)");
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertEquals(InterpreterResult.Type.TABLE, p.getReturn().message().get(0).getType());
-      assertEquals("name\tage\nhello\t20\n", p.getReturn().message().get(0).getData());
-
-      // get resource from pyspark
-      p = note.addNewParagraph(anonymous);
-      p.setText("%spark.pyspark df=z.getAsDataFrame('table_result')\nz.show(df)");
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertEquals(InterpreterResult.Type.TABLE, p.getReturn().message().get(0).getType());
-      assertEquals("name\tage\nhello\t20\n", p.getReturn().message().get(0).getData());
-
-      // get resource from ipyspark
-      p = note.addNewParagraph(anonymous);
-      p.setText("%spark.ipyspark df=z.getAsDataFrame('table_result')\nz.show(df)");
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertEquals(InterpreterResult.Type.TABLE, p.getReturn().message().get(0).getType());
-      assertEquals("name\tage\nhello\t20\n", p.getReturn().message().get(0).getData());
-
-      // get resource from sparkr
-      p = note.addNewParagraph(anonymous);
-      p.setText("%spark.r df=z.getAsDataFrame('table_result')\ndf");
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertEquals(InterpreterResult.Type.TEXT, p.getReturn().message().get(0).getType());
-      assertTrue(p.getReturn().toString(),
-              p.getReturn().message().get(0).getData().contains("name age\n1 hello  20"));
-
-      // test display DataSet
-      if (isSpark2() || isSpark3()) {
-        p = note.addNewParagraph(anonymous);
-        p.setText("%spark val ds=spark.createDataset(Seq((\"hello\",20)))\n" +
-            "z.show(ds)");
-        note.run(p.getId(), true);
-        assertEquals(Status.FINISHED, p.getStatus());
-        assertEquals(InterpreterResult.Type.TABLE, p.getReturn().message().get(0).getType());
-        assertEquals("_1\t_2\nhello\t20\n", p.getReturn().message().get(0).getData());
-      }
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          // test basic dataframe api
+          Paragraph p = note.addNewParagraph(anonymous);
+          if (isSpark2() || isSpark3()) {
+            p.setText("%spark val df=spark.createDataFrame(Seq((\"hello\",20)))" +
+                    ".toDF(\"name\", \"age\")\n" +
+                    "df.collect()");
+          } else {
+            p.setText("%spark val df=sqlContext.createDataFrame(Seq((\"hello\",20)))" +
+                    ".toDF(\"name\", \"age\")\n" +
+                    "df.collect()");
+          }
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertTrue(p.getReturn().message().get(0).getData().contains(
+                  "Array[org.apache.spark.sql.Row] = Array([hello,20])"));
+
+          // test display DataFrame
+          p = note.addNewParagraph(anonymous);
+          if (isSpark2() || isSpark3()) {
+            p.setText("%spark val df=spark.createDataFrame(Seq((\"hello\",20)))" +
+                    ".toDF(\"name\", \"age\")\n" +
+                    "df.createOrReplaceTempView(\"test_table\")\n" +
+                    "z.show(df)");
+          } else {
+            p.setText("%spark val df=sqlContext.createDataFrame(Seq((\"hello\",20)))" +
+                    ".toDF(\"name\", \"age\")\n" +
+                    "df.registerTempTable(\"test_table\")\n" +
+                    "z.show(df)");
+          }
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertEquals(InterpreterResult.Type.TABLE, p.getReturn().message().get(0).getType());
+          assertEquals("name\tage\nhello\t20\n", p.getReturn().message().get(0).getData());
+
+          // run sql and save it into resource pool
+          p = note.addNewParagraph(anonymous);
+          p.setText("%spark.sql(saveAs=table_result) select * from test_table");
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertEquals(InterpreterResult.Type.TABLE, p.getReturn().message().get(0).getType());
+          assertEquals("name\tage\nhello\t20\n", p.getReturn().message().get(0).getData());
+
+          // get resource from spark
+          p = note.addNewParagraph(anonymous);
+          p.setText("%spark val df=z.getAsDataFrame(\"table_result\")\nz.show(df)");
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertEquals(InterpreterResult.Type.TABLE, p.getReturn().message().get(0).getType());
+          assertEquals("name\tage\nhello\t20\n", p.getReturn().message().get(0).getData());
+
+          // get resource from pyspark
+          p = note.addNewParagraph(anonymous);
+          p.setText("%spark.pyspark df=z.getAsDataFrame('table_result')\nz.show(df)");
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertEquals(InterpreterResult.Type.TABLE, p.getReturn().message().get(0).getType());
+          assertEquals("name\tage\nhello\t20\n", p.getReturn().message().get(0).getData());
+
+          // get resource from ipyspark
+          p = note.addNewParagraph(anonymous);
+          p.setText("%spark.ipyspark df=z.getAsDataFrame('table_result')\nz.show(df)");
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertEquals(InterpreterResult.Type.TABLE, p.getReturn().message().get(0).getType());
+          assertEquals("name\tage\nhello\t20\n", p.getReturn().message().get(0).getData());
+
+          // get resource from sparkr
+          p = note.addNewParagraph(anonymous);
+          p.setText("%spark.r df=z.getAsDataFrame('table_result')\ndf");
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertEquals(InterpreterResult.Type.TEXT, p.getReturn().message().get(0).getType());
+          assertTrue(p.getReturn().toString(),
+                  p.getReturn().message().get(0).getData().contains("name age\n1 hello  20"));
+
+          // test display DataSet
+          if (isSpark2() || isSpark3()) {
+            p = note.addNewParagraph(anonymous);
+            p.setText("%spark val ds=spark.createDataset(Seq((\"hello\",20)))\n" +
+                "z.show(ds)");
+            note.run(p.getId(), true);
+            assertEquals(Status.FINISHED, p.getStatus());
+            assertEquals(InterpreterResult.Type.TABLE, p.getReturn().message().get(0).getType());
+            assertEquals("_1\t_2\nhello\t20\n", p.getReturn().message().get(0).getData());
+          }
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void sparkRTest() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p = note.addNewParagraph(anonymous);
-
-      if (isSpark3()) {
-        p.setText("%spark.r localDF <- data.frame(name=c(\"a\", \"b\", \"c\"), age=c(19, 23, 18))\n" +
-                "df <- createDataFrame(localDF)\n" +
-                "count(df)"
-        );
-      } else {
-        String sqlContextName = "sqlContext";
-        if (isSpark2() || isSpark3()) {
-          sqlContextName = "spark";
-        }
-        p.setText("%spark.r localDF <- data.frame(name=c(\"a\", \"b\", \"c\"), age=c(19, 23, 18))\n" +
-                "df <- createDataFrame(" + sqlContextName + ", localDF)\n" +
-                "count(df)"
-        );
-      }
-
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertEquals("[1] 3", p.getReturn().message().get(0).getData().trim());
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(anonymous);
+
+          if (isSpark3()) {
+            p.setText("%spark.r localDF <- data.frame(name=c(\"a\", \"b\", \"c\"), age=c(19, 23, 18))\n" +
+                    "df <- createDataFrame(localDF)\n" +
+                    "count(df)"
+            );
+          } else {
+            String sqlContextName = "sqlContext";
+            if (isSpark2() || isSpark3()) {
+              sqlContextName = "spark";
+            }
+            p.setText("%spark.r localDF <- data.frame(name=c(\"a\", \"b\", \"c\"), age=c(19, 23, 18))\n" +
+                    "df <- createDataFrame(" + sqlContextName + ", localDF)\n" +
+                    "count(df)"
+            );
+          }
+
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertEquals("[1] 3", p.getReturn().message().get(0).getData().trim());
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
@@ -414,309 +439,338 @@ public abstract class ZeppelinSparkClusterTest extends AbstractTestRestApi {
   @Test
   public void pySparkTest() throws IOException {
     // create new note
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-
-      // run markdown paragraph, again
-      Paragraph p = note.addNewParagraph(anonymous);
-      p.setText("%spark.pyspark sc.parallelize(range(1, 11)).reduce(lambda a, b: a + b)");
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertEquals("55\n", p.getReturn().message().get(0).getData());
-
-      // simple form via local properties
-      p = note.addNewParagraph(anonymous);
-      p.setText("%spark.pyspark(form=simple) print('name_' + '${name=abc}')");
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertEquals("name_abc\n", p.getReturn().message().get(0).getData());
-
-      // test code completion
-      String code = "%spark.pyspark spark.";
-      List<InterpreterCompletion> completions = note.completion(p.getId(), code, code.length(), AuthenticationInfo.ANONYMOUS);
-      assertTrue(completions.size() > 0);
-
-      if (isSpark1()) {
-        // run sqlContext test
-        p = note.addNewParagraph(anonymous);
-        p.setText("%pyspark from pyspark.sql import Row\n" +
-            "df=sqlContext.createDataFrame([Row(id=1, age=20)])\n" +
-            "df.collect()");
-        note.run(p.getId(), true);
-        assertEquals(Status.FINISHED, p.getStatus());
-        assertEquals("[Row(age=20, id=1)]\n", p.getReturn().message().get(0).getData());
-
-        // test display Dataframe
-        p = note.addNewParagraph(anonymous);
-        p.setText("%pyspark from pyspark.sql import Row\n" +
-            "df=sqlContext.createDataFrame([Row(id=1, age=20)])\n" +
-            "z.show(df)");
-        note.run(p.getId(), true);
-        waitForFinish(p);
-        assertEquals(Status.FINISHED, p.getStatus());
-        assertEquals(InterpreterResult.Type.TABLE, p.getReturn().message().get(0).getType());
-        // TODO(zjffdu), one more \n is appended, need to investigate why.
-        assertEquals("age\tid\n20\t1\n", p.getReturn().message().get(0).getData());
-
-        // test udf
-        p = note.addNewParagraph(anonymous);
-        p.setText("%pyspark sqlContext.udf.register(\"f1\", lambda x: len(x))\n" +
-            "sqlContext.sql(\"select f1(\\\"abc\\\") as len\").collect()");
-        note.run(p.getId(), true);
-        assertEquals(Status.FINISHED, p.getStatus());
-        assertTrue("[Row(len=u'3')]\n".equals(p.getReturn().message().get(0).getData()) ||
-            "[Row(len='3')]\n".equals(p.getReturn().message().get(0).getData()));
-
-        // test exception
-        p = note.addNewParagraph(anonymous);
-        /*
-         %pyspark
-         a=1
-
-         print(a2)
-         */
-        p.setText("%pyspark a=1\n\nprint(a2)");
-        note.run(p.getId(), true);
-        assertEquals(Status.ERROR, p.getStatus());
-        assertTrue(p.getReturn().message().get(0).getData()
-            .contains("Fail to execute line 3: print(a2)"));
-        assertTrue(p.getReturn().message().get(0).getData()
-            .contains("name 'a2' is not defined"));
-      } else if (isSpark2()){
-        // run SparkSession test
-        p = note.addNewParagraph(anonymous);
-        p.setText("%pyspark from pyspark.sql import Row\n" +
-            "df=sqlContext.createDataFrame([Row(id=1, age=20)])\n" +
-            "df.collect()");
-        note.run(p.getId(), true);
-        assertEquals(Status.FINISHED, p.getStatus());
-        assertEquals("[Row(age=20, id=1)]\n", p.getReturn().message().get(0).getData());
-
-        // test udf
-        p = note.addNewParagraph(anonymous);
-        // use SQLContext to register UDF but use this UDF through SparkSession
-        p.setText("%pyspark sqlContext.udf.register(\"f1\", lambda x: len(x))\n" +
-            "spark.sql(\"select f1(\\\"abc\\\") as len\").collect()");
-        note.run(p.getId(), true);
-        assertEquals(Status.FINISHED, p.getStatus());
-        assertTrue("[Row(len=u'3')]\n".equals(p.getReturn().message().get(0).getData()) ||
-            "[Row(len='3')]\n".equals(p.getReturn().message().get(0).getData()));
-      } else {
-        // run SparkSession test
-        p = note.addNewParagraph(anonymous);
-        p.setText("%pyspark from pyspark.sql import Row\n" +
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          // run markdown paragraph, again
+          Paragraph p = note.addNewParagraph(anonymous);
+          p.setText("%spark.pyspark sc.parallelize(range(1, 11)).reduce(lambda a, b: a + b)");
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertEquals("55\n", p.getReturn().message().get(0).getData());
+
+          // simple form via local properties
+          p = note.addNewParagraph(anonymous);
+          p.setText("%spark.pyspark(form=simple) print('name_' + '${name=abc}')");
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertEquals("name_abc\n", p.getReturn().message().get(0).getData());
+
+          // test code completion
+          String code = "%spark.pyspark spark.";
+          List<InterpreterCompletion> completions = note.completion(p.getId(), code, code.length(), AuthenticationInfo.ANONYMOUS);
+          assertTrue(completions.size() > 0);
+
+          if (isSpark1()) {
+            // run sqlContext test
+            p = note.addNewParagraph(anonymous);
+            p.setText("%pyspark from pyspark.sql import Row\n" +
+                "df=sqlContext.createDataFrame([Row(id=1, age=20)])\n" +
+                "df.collect()");
+            note.run(p.getId(), true);
+            assertEquals(Status.FINISHED, p.getStatus());
+            assertEquals("[Row(age=20, id=1)]\n", p.getReturn().message().get(0).getData());
+
+            // test display Dataframe
+            p = note.addNewParagraph(anonymous);
+            p.setText("%pyspark from pyspark.sql import Row\n" +
+                "df=sqlContext.createDataFrame([Row(id=1, age=20)])\n" +
+                "z.show(df)");
+            note.run(p.getId(), true);
+            waitForFinish(p);
+            assertEquals(Status.FINISHED, p.getStatus());
+            assertEquals(InterpreterResult.Type.TABLE, p.getReturn().message().get(0).getType());
+            // TODO(zjffdu), one more \n is appended, need to investigate why.
+            assertEquals("age\tid\n20\t1\n", p.getReturn().message().get(0).getData());
+
+            // test udf
+            p = note.addNewParagraph(anonymous);
+            p.setText("%pyspark sqlContext.udf.register(\"f1\", lambda x: len(x))\n" +
+                "sqlContext.sql(\"select f1(\\\"abc\\\") as len\").collect()");
+            note.run(p.getId(), true);
+            assertEquals(Status.FINISHED, p.getStatus());
+            assertTrue("[Row(len=u'3')]\n".equals(p.getReturn().message().get(0).getData()) ||
+                "[Row(len='3')]\n".equals(p.getReturn().message().get(0).getData()));
+
+            // test exception
+            p = note.addNewParagraph(anonymous);
+            /*
+             %pyspark
+             a=1
+
+             print(a2)
+             */
+            p.setText("%pyspark a=1\n\nprint(a2)");
+            note.run(p.getId(), true);
+            assertEquals(Status.ERROR, p.getStatus());
+            assertTrue(p.getReturn().message().get(0).getData()
+                .contains("Fail to execute line 3: print(a2)"));
+            assertTrue(p.getReturn().message().get(0).getData()
+                .contains("name 'a2' is not defined"));
+          } else if (isSpark2()){
+            // run SparkSession test
+            p = note.addNewParagraph(anonymous);
+            p.setText("%pyspark from pyspark.sql import Row\n" +
                 "df=sqlContext.createDataFrame([Row(id=1, age=20)])\n" +
                 "df.collect()");
-        note.run(p.getId(), true);
-        assertEquals(Status.FINISHED, p.getStatus());
-        assertEquals("[Row(id=1, age=20)]\n", p.getReturn().message().get(0).getData());
-
-        // test udf
-        p = note.addNewParagraph(anonymous);
-        // use SQLContext to register UDF but use this UDF through SparkSession
-        p.setText("%pyspark sqlContext.udf.register(\"f1\", lambda x: len(x))\n" +
+            note.run(p.getId(), true);
+            assertEquals(Status.FINISHED, p.getStatus());
+            assertEquals("[Row(age=20, id=1)]\n", p.getReturn().message().get(0).getData());
+
+            // test udf
+            p = note.addNewParagraph(anonymous);
+            // use SQLContext to register UDF but use this UDF through SparkSession
+            p.setText("%pyspark sqlContext.udf.register(\"f1\", lambda x: len(x))\n" +
                 "spark.sql(\"select f1(\\\"abc\\\") as len\").collect()");
-        note.run(p.getId(), true);
-        assertEquals(Status.FINISHED, p.getStatus());
-        assertTrue("[Row(len=u'3')]\n".equals(p.getReturn().message().get(0).getData()) ||
+            note.run(p.getId(), true);
+            assertEquals(Status.FINISHED, p.getStatus());
+            assertTrue("[Row(len=u'3')]\n".equals(p.getReturn().message().get(0).getData()) ||
                 "[Row(len='3')]\n".equals(p.getReturn().message().get(0).getData()));
-      }
+          } else {
+            // run SparkSession test
+            p = note.addNewParagraph(anonymous);
+            p.setText("%pyspark from pyspark.sql import Row\n" +
+                    "df=sqlContext.createDataFrame([Row(id=1, age=20)])\n" +
+                    "df.collect()");
+            note.run(p.getId(), true);
+            assertEquals(Status.FINISHED, p.getStatus());
+            assertEquals("[Row(id=1, age=20)]\n", p.getReturn().message().get(0).getData());
+
+            // test udf
+            p = note.addNewParagraph(anonymous);
+            // use SQLContext to register UDF but use this UDF through SparkSession
+            p.setText("%pyspark sqlContext.udf.register(\"f1\", lambda x: len(x))\n" +
+                    "spark.sql(\"select f1(\\\"abc\\\") as len\").collect()");
+            note.run(p.getId(), true);
+            assertEquals(Status.FINISHED, p.getStatus());
+            assertTrue("[Row(len=u'3')]\n".equals(p.getReturn().message().get(0).getData()) ||
+                    "[Row(len='3')]\n".equals(p.getReturn().message().get(0).getData()));
+          }
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void zRunTest() throws IOException, InterruptedException {
-    Note note = null;
-    Note note2 = null;
+    String noteId = null;
+    String note2Id = null;
     try {
       // create new note
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p0 = note.addNewParagraph(anonymous);
-      // z.run(paragraphIndex)
-      p0.setText("%spark z.run(1)");
-      Paragraph p1 = note.addNewParagraph(anonymous);
-      p1.setText("%spark val a=10");
-      Paragraph p2 = note.addNewParagraph(anonymous);
-      p2.setText("%spark print(a)");
-
-      note.run(p0.getId(), true);
-      assertEquals(Status.FINISHED, p0.getStatus());
-
-      // z.run is not blocking call. So p1 may not be finished when p0 is done.
-      waitForFinish(p1);
-      assertEquals(Status.FINISHED, p1.getStatus());
-      note.run(p2.getId(), true);
-      assertEquals(Status.FINISHED, p2.getStatus());
-      assertEquals("10", p2.getReturn().message().get(0).getData());
-
-      Paragraph p3 = note.addNewParagraph(anonymous);
-      p3.setText("%spark println(new java.util.Date())");
-
-      // run current Node, z.runNote(noteId)
-      p0.setText(String.format("%%spark z.runNote(\"%s\")", note.getId()));
-      note.run(p0.getId());
-      waitForFinish(p0);
-      waitForFinish(p1);
-      waitForFinish(p2);
-      waitForFinish(p3);
-
-      assertEquals(Status.FINISHED, p3.getStatus());
-      String p3result = p3.getReturn().message().get(0).getData();
-      assertTrue(p3result.length() > 0);
-
-      // z.run(noteId, paragraphId)
-      p0.setText(String.format("%%spark z.run(\"%s\", \"%s\")", note.getId(), p3.getId()));
-      p3.setText("%spark println(\"END\")");
-
-      note.run(p0.getId(), true);
-      // Sleep 1 second to ensure p3 start running
-      Thread.sleep(1000);
-      waitForFinish(p3);
-      assertEquals(Status.FINISHED, p3.getStatus());
-      assertEquals("END\n", p3.getReturn().message().get(0).getData());
-
-      // run paragraph in note2 via paragraph in note1
-      note2 = TestUtils.getInstance(Notebook.class).createNote("note2", anonymous);
-      Paragraph p20 = note2.addNewParagraph(anonymous);
-      p20.setText("%spark val a = 1");
-      Paragraph p21 = note2.addNewParagraph(anonymous);
-      p21.setText("%spark print(a)");
-
-      // run p20 of note2 via paragraph in note1
-      p0.setText(String.format("%%spark.pyspark z.run(\"%s\", \"%s\")", note2.getId(), p20.getId()));
-      note.run(p0.getId(), true);
-      waitForFinish(p20);
-      assertEquals(Status.FINISHED, p20.getStatus());
-      assertEquals(Status.READY, p21.getStatus());
-
-      p0.setText(String.format("%%spark z.runNote(\"%s\")", note2.getId()));
-      note.run(p0.getId(), true);
-      waitForFinish(p20);
-      waitForFinish(p21);
-      assertEquals(Status.FINISHED, p20.getStatus());
-      assertEquals(Status.FINISHED, p21.getStatus());
-      assertEquals("1", p21.getReturn().message().get(0).getData());
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      note2Id = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p0 = note.addNewParagraph(anonymous);
+          // z.run(paragraphIndex)
+          p0.setText("%spark z.run(1)");
+          Paragraph p1 = note.addNewParagraph(anonymous);
+          p1.setText("%spark val a=10");
+          Paragraph p2 = note.addNewParagraph(anonymous);
+          p2.setText("%spark print(a)");
+
+          note.run(p0.getId(), true);
+          assertEquals(Status.FINISHED, p0.getStatus());
+
+          // z.run is not blocking call. So p1 may not be finished when p0 is done.
+          waitForFinish(p1);
+          assertEquals(Status.FINISHED, p1.getStatus());
+          note.run(p2.getId(), true);
+          assertEquals(Status.FINISHED, p2.getStatus());
+          assertEquals("10", p2.getReturn().message().get(0).getData());
+
+          Paragraph p3 = note.addNewParagraph(anonymous);
+          p3.setText("%spark println(new java.util.Date())");
+
+          // run current Node, z.runNote(noteId)
+          p0.setText(String.format("%%spark z.runNote(\"%s\")", note.getId()));
+          note.run(p0.getId());
+          waitForFinish(p0);
+          waitForFinish(p1);
+          waitForFinish(p2);
+          waitForFinish(p3);
+
+          assertEquals(Status.FINISHED, p3.getStatus());
+          String p3result = p3.getReturn().message().get(0).getData();
+          assertTrue(p3result.length() > 0);
+
+          // z.run(noteId, paragraphId)
+          p0.setText(String.format("%%spark z.run(\"%s\", \"%s\")", note.getId(), p3.getId()));
+          p3.setText("%spark println(\"END\")");
+
+          note.run(p0.getId(), true);
+          // Sleep 1 second to ensure p3 start running
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException e) {
+            fail();
+          }
+          waitForFinish(p3);
+          assertEquals(Status.FINISHED, p3.getStatus());
+          assertEquals("END\n", p3.getReturn().message().get(0).getData());
+
+          // run paragraph in note2 via paragraph in note1
+          String noteId2 = TestUtils.getInstance(Notebook.class).createNote("note2", anonymous);
+          TestUtils.getInstance(Notebook.class).processNote(noteId2,
+            note2 -> {
+              Paragraph p20 = note2.addNewParagraph(anonymous);
+              p20.setText("%spark val a = 1");
+              Paragraph p21 = note2.addNewParagraph(anonymous);
+              p21.setText("%spark print(a)");
+
+              // run p20 of note2 via paragraph in note1
+              p0.setText(String.format("%%spark.pyspark z.run(\"%s\", \"%s\")", note2.getId(), p20.getId()));
+              note.run(p0.getId(), true);
+              waitForFinish(p20);
+              assertEquals(Status.FINISHED, p20.getStatus());
+              assertEquals(Status.READY, p21.getStatus());
+
+              p0.setText(String.format("%%spark z.runNote(\"%s\")", note2.getId()));
+              note.run(p0.getId(), true);
+              waitForFinish(p20);
+              waitForFinish(p21);
+              assertEquals(Status.FINISHED, p20.getStatus());
+              assertEquals(Status.FINISHED, p21.getStatus());
+              assertEquals("1", p21.getReturn().message().get(0).getData());
+              return null;
+            });
+          return noteId2;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
-      if (null != note2) {
-        TestUtils.getInstance(Notebook.class).removeNote(note2, anonymous);
+      if (null != note2Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note2Id, anonymous);
       }
     }
   }
 
   @Test
   public void testZeppelinContextResource() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-
-      Paragraph p1 = note.addNewParagraph(anonymous);
-      p1.setText("%spark z.put(\"var_1\", \"hello world\")");
-
-      Paragraph p2 = note.addNewParagraph(anonymous);
-      p2.setText("%spark println(z.get(\"var_1\"))");
-
-      Paragraph p3 = note.addNewParagraph(anonymous);
-      p3.setText("%spark.pyspark print(z.get(\"var_1\"))");
-
-      Paragraph p4 = note.addNewParagraph(anonymous);
-      p4.setText("%spark.r z.get(\"var_1\")");
-
-      // resources across interpreter processes (via DistributedResourcePool)
-      Paragraph p5 = note.addNewParagraph(anonymous);
-      p5.setText("%python print(z.get('var_1'))");
-
-      note.run(p1.getId(), true);
-      note.run(p2.getId(), true);
-      note.run(p3.getId(), true);
-      note.run(p4.getId(), true);
-      note.run(p5.getId(), true);
-
-      assertEquals(Status.FINISHED, p1.getStatus());
-      assertEquals(Status.FINISHED, p2.getStatus());
-      assertEquals("hello world\n", p2.getReturn().message().get(0).getData());
-      assertEquals(Status.FINISHED, p3.getStatus());
-      assertEquals("hello world\n", p3.getReturn().message().get(0).getData());
-      assertEquals(Status.FINISHED, p4.getStatus());
-      assertTrue(p4.getReturn().toString(),
-              p4.getReturn().message().get(0).getData().contains("hello world"));
-      assertEquals(Status.FINISHED, p5.getStatus());
-      assertEquals("hello world\n", p5.getReturn().message().get(0).getData());
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p1 = note.addNewParagraph(anonymous);
+          p1.setText("%spark z.put(\"var_1\", \"hello world\")");
+
+          Paragraph p2 = note.addNewParagraph(anonymous);
+          p2.setText("%spark println(z.get(\"var_1\"))");
+
+          Paragraph p3 = note.addNewParagraph(anonymous);
+          p3.setText("%spark.pyspark print(z.get(\"var_1\"))");
+
+          Paragraph p4 = note.addNewParagraph(anonymous);
+          p4.setText("%spark.r z.get(\"var_1\")");
+
+          // resources across interpreter processes (via DistributedResourcePool)
+          Paragraph p5 = note.addNewParagraph(anonymous);
+          p5.setText("%python print(z.get('var_1'))");
+
+          note.run(p1.getId(), true);
+          note.run(p2.getId(), true);
+          note.run(p3.getId(), true);
+          note.run(p4.getId(), true);
+          note.run(p5.getId(), true);
+
+          assertEquals(Status.FINISHED, p1.getStatus());
+          assertEquals(Status.FINISHED, p2.getStatus());
+          assertEquals("hello world\n", p2.getReturn().message().get(0).getData());
+          assertEquals(Status.FINISHED, p3.getStatus());
+          assertEquals("hello world\n", p3.getReturn().message().get(0).getData());
+          assertEquals(Status.FINISHED, p4.getStatus());
+          assertTrue(p4.getReturn().toString(),
+                  p4.getReturn().message().get(0).getData().contains("hello world"));
+          assertEquals(Status.FINISHED, p5.getStatus());
+          assertEquals("hello world\n", p5.getReturn().message().get(0).getData());
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testZeppelinContextHook() throws IOException {
-    Note note = null;
-    Note note2 = null;
+    String noteId = null;
+    String note2Id = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-
-      // register global hook & note1 hook
-      Paragraph p1 = note.addNewParagraph(anonymous);
-      p1.setText("%python from __future__ import print_function\n" +
-          "z.registerHook('pre_exec', 'print(1)')\n" +
-          "z.registerHook('post_exec', 'print(2)')\n" +
-          "z.registerNoteHook('pre_exec', 'print(3)', '" + note.getId() + "')\n" +
-          "z.registerNoteHook('post_exec', 'print(4)', '" + note.getId() + "')\n");
-
-      Paragraph p2 = note.addNewParagraph(anonymous);
-      p2.setText("%python print(5)");
-
-      note.run(p1.getId(), true);
-      note.run(p2.getId(), true);
-
-      assertEquals(Status.FINISHED, p1.getStatus());
-      assertEquals(Status.FINISHED, p2.getStatus());
-      assertEquals("1\n3\n5\n4\n2\n", p2.getReturn().message().get(0).getData());
-
-      note2 = TestUtils.getInstance(Notebook.class).createNote("note2", anonymous);
-      Paragraph p3 = note2.addNewParagraph(anonymous);
-      p3.setText("%python print(6)");
-      note2.run(p3.getId(), true);
-      assertEquals("1\n6\n2\n", p3.getReturn().message().get(0).getData());
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      note2Id = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          // register global hook & note1 hook
+          Paragraph p1 = note.addNewParagraph(anonymous);
+          p1.setText("%python from __future__ import print_function\n" +
+              "z.registerHook('pre_exec', 'print(1)')\n" +
+              "z.registerHook('post_exec', 'print(2)')\n" +
+              "z.registerNoteHook('pre_exec', 'print(3)', '" + note.getId() + "')\n" +
+              "z.registerNoteHook('post_exec', 'print(4)', '" + note.getId() + "')\n");
+
+          Paragraph p2 = note.addNewParagraph(anonymous);
+          p2.setText("%python print(5)");
+
+          note.run(p1.getId(), true);
+          note.run(p2.getId(), true);
+
+          assertEquals(Status.FINISHED, p1.getStatus());
+          assertEquals(Status.FINISHED, p2.getStatus());
+          assertEquals("1\n3\n5\n4\n2\n", p2.getReturn().message().get(0).getData());
+
+          String note2Tmp = TestUtils.getInstance(Notebook.class).createNote("note2", anonymous);
+          TestUtils.getInstance(Notebook.class).processNote(note2Tmp,
+            note2 -> {
+              Paragraph p3 = note2.addNewParagraph(anonymous);
+              p3.setText("%python print(6)");
+              note2.run(p3.getId(), true);
+              assertEquals("1\n6\n2\n", p3.getReturn().message().get(0).getData());
+              return null;
+            });
+          return note2Tmp;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
-      if (null != note2) {
-        TestUtils.getInstance(Notebook.class).removeNote(note2, anonymous);
+      if (null != note2Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note2Id, anonymous);
       }
     }
   }
 
   private void verifySparkVersionNumber() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p = note.addNewParagraph(anonymous);
-
-      p.setText("%spark print(sc.version)");
-      note.run(p.getId());
-      waitForFinish(p);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertEquals(sparkVersion, p.getReturn().message().get(0).getData());
-
-      p.setText("%spark.pyspark sc.version");
-      note.run(p.getId());
-      waitForFinish(p);
-      assertEquals(Status.FINISHED, p.getStatus());
-      assertTrue(p.getReturn().toString(),
-              p.getReturn().message().get(0).getData().contains(sparkVersion));
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(anonymous);
+
+          p.setText("%spark print(sc.version)");
+          note.run(p.getId());
+          waitForFinish(p);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertEquals(sparkVersion, p.getReturn().message().get(0).getData());
+
+          p.setText("%spark.pyspark sc.version");
+          note.run(p.getId());
+          waitForFinish(p);
+          assertEquals(Status.FINISHED, p.getStatus());
+          assertTrue(p.getReturn().toString(),
+                  p.getReturn().message().get(0).getData().contains(sparkVersion));
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
@@ -735,370 +789,442 @@ public abstract class ZeppelinSparkClusterTest extends AbstractTestRestApi {
 
   @Test
   public void testSparkZeppelinContextDynamicForms() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p = note.addNewParagraph(anonymous);
-      String code = "%spark println(z.textbox(\"my_input\", \"default_name\"))\n" +
-          "println(z.password(\"my_pwd\"))\n" +
-          "println(z.select(\"my_select\", \"1\"," +
-          "Seq((\"1\", \"select_1\"), (\"2\", \"select_2\"))))\n" +
-          "val items=z.checkbox(\"my_checkbox\", " +
-          "Seq((\"1\", \"check_1\"), (\"2\", \"check_2\")), Seq(\"2\"))\n" +
-          "println(items(0))";
-      p.setText(code);
-      note.run(p.getId());
-      waitForFinish(p);
-
-      assertEquals(Status.FINISHED, p.getStatus());
-      Iterator<String> formIter = p.settings.getForms().keySet().iterator();
-      assertEquals("my_input", formIter.next());
-      assertEquals("my_pwd", formIter.next());
-      assertEquals("my_select", formIter.next());
-      assertEquals("my_checkbox", formIter.next());
-
-      // check dynamic forms values
-      String[] result = p.getReturn().message().get(0).getData().split("\n");
-      assertEquals(5, result.length);
-      assertEquals("default_name", result[0]);
-      assertEquals("null", result[1]);
-      assertEquals("1", result[2]);
-      assertEquals("2", result[3]);
-      assertEquals("items: Seq[Any] = Buffer(2)", result[4]);
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(anonymous);
+          String code = "%spark println(z.textbox(\"my_input\", \"default_name\"))\n" +
+              "println(z.password(\"my_pwd\"))\n" +
+              "println(z.select(\"my_select\", \"1\"," +
+              "Seq((\"1\", \"select_1\"), (\"2\", \"select_2\"))))\n" +
+              "val items=z.checkbox(\"my_checkbox\", " +
+              "Seq((\"1\", \"check_1\"), (\"2\", \"check_2\")), Seq(\"2\"))\n" +
+              "println(items(0))";
+          p.setText(code);
+          note.run(p.getId());
+          waitForFinish(p);
+
+          assertEquals(Status.FINISHED, p.getStatus());
+          Iterator<String> formIter = p.settings.getForms().keySet().iterator();
+          assertEquals("my_input", formIter.next());
+          assertEquals("my_pwd", formIter.next());
+          assertEquals("my_select", formIter.next());
+          assertEquals("my_checkbox", formIter.next());
+
+          // check dynamic forms values
+          String[] result = p.getReturn().message().get(0).getData().split("\n");
+          assertEquals(5, result.length);
+          assertEquals("default_name", result[0]);
+          assertEquals("null", result[1]);
+          assertEquals("1", result[2]);
+          assertEquals("2", result[3]);
+          assertEquals("items: Seq[Any] = Buffer(2)", result[4]);
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testPySparkZeppelinContextDynamicForms() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p = note.addNewParagraph(anonymous);
-      String code = "%spark.pyspark print(z.input('my_input', 'default_name'))\n" +
-          "print(z.password('my_pwd'))\n" +
-          "print(z.select('my_select', " +
-          "[('1', 'select_1'), ('2', 'select_2')], defaultValue='1'))\n" +
-          "items=z.checkbox('my_checkbox', " +
-          "[('1', 'check_1'), ('2', 'check_2')], defaultChecked=['2'])\n" +
-          "print(items[0])";
-      p.setText(code);
-      note.run(p.getId(), true);
-
-      assertEquals(Status.FINISHED, p.getStatus());
-      Iterator<String> formIter = p.settings.getForms().keySet().iterator();
-      assertEquals("my_input", formIter.next());
-      assertEquals("my_pwd", formIter.next());
-      assertEquals("my_select", formIter.next());
-      assertEquals("my_checkbox", formIter.next());
-
-      // check dynamic forms values
-      String[] result = p.getReturn().message().get(0).getData().split("\n");
-      assertEquals(4, result.length);
-      assertEquals("default_name", result[0]);
-      assertEquals("None", result[1]);
-      assertEquals("1", result[2]);
-      assertEquals("2", result[3]);
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(anonymous);
+          String code = "%spark.pyspark print(z.input('my_input', 'default_name'))\n" +
+              "print(z.password('my_pwd'))\n" +
+              "print(z.select('my_select', " +
+              "[('1', 'select_1'), ('2', 'select_2')], defaultValue='1'))\n" +
+              "items=z.checkbox('my_checkbox', " +
+              "[('1', 'check_1'), ('2', 'check_2')], defaultChecked=['2'])\n" +
+              "print(items[0])";
+          p.setText(code);
+          note.run(p.getId(), true);
+
+          assertEquals(Status.FINISHED, p.getStatus());
+          Iterator<String> formIter = p.settings.getForms().keySet().iterator();
+          assertEquals("my_input", formIter.next());
+          assertEquals("my_pwd", formIter.next());
+          assertEquals("my_select", formIter.next());
+          assertEquals("my_checkbox", formIter.next());
+
+          // check dynamic forms values
+          String[] result = p.getReturn().message().get(0).getData().split("\n");
+          assertEquals(4, result.length);
+          assertEquals("default_name", result[0]);
+          assertEquals("None", result[1]);
+          assertEquals("1", result[2]);
+          assertEquals("2", result[3]);
+      return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testAngularObjects() throws IOException, InterpreterNotFoundException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p1 = note.addNewParagraph(anonymous);
-
-      // add local angular object
-      p1.setText("%spark z.angularBind(\"name\", \"world\")");
-      note.run(p1.getId(), true);
-      assertEquals(Status.FINISHED, p1.getStatus());
-      // angular object is saved to InterpreterGroup's AngularObjectRegistry
-      List<AngularObject> angularObjects = p1.getBindedInterpreter().getInterpreterGroup()
-              .getAngularObjectRegistry().getAll(note.getId(), null);
-      assertEquals(1, angularObjects.size());
-      assertEquals("name", angularObjects.get(0).getName());
-      assertEquals("world", angularObjects.get(0).get());
-
-      // angular object is saved to note as well.
-      angularObjects = note.getAngularObjects(p1.getBindedInterpreter().getInterpreterGroup().getId());
-      assertEquals(1, angularObjects.size());
-      assertEquals("name", angularObjects.get(0).getName());
-      assertEquals("world", angularObjects.get(0).get());
-
-      // remove local angular object
-      Paragraph p2 = note.addNewParagraph(anonymous);
-      p2.setText("%spark z.angularUnbind(\"name\")");
-      note.run(p2.getId(), true);
-      assertEquals(Status.FINISHED, p2.getStatus());
-      angularObjects = p1.getBindedInterpreter().getInterpreterGroup().getAngularObjectRegistry()
-              .getAll(note.getId(), null);
-      assertEquals(0, angularObjects.size());
-
-      angularObjects = note.getAngularObjects(p1.getBindedInterpreter().getInterpreterGroup().getId());
-      assertEquals(0, angularObjects.size());
-
-      // add global angular object
-      Paragraph p3 = note.addNewParagraph(anonymous);
-      p3.setText("%spark z.angularBindGlobal(\"name2\", \"world2\")");
-      note.run(p3.getId(), true);
-      assertEquals(Status.FINISHED, p3.getStatus());
-      List<AngularObject> globalAngularObjects = p3.getBindedInterpreter().getInterpreterGroup()
-              .getAngularObjectRegistry().getAll(null, null);
-      assertEquals(1, globalAngularObjects.size());
-      assertEquals("name2", globalAngularObjects.get(0).getName());
-      assertEquals("world2", globalAngularObjects.get(0).get());
-
-      // global angular object is not saved to note
-      angularObjects = note.getAngularObjects(p1.getBindedInterpreter().getInterpreterGroup().getId());
-      assertEquals(0, angularObjects.size());
-
-      // remove global angular object
-      Paragraph p4 = note.addNewParagraph(anonymous);
-      p4.setText("%spark z.angularUnbindGlobal(\"name2\")");
-      note.run(p4.getId(), true);
-      assertEquals(Status.FINISHED, p4.getStatus());
-      globalAngularObjects = p4.getBindedInterpreter().getInterpreterGroup()
-              .getAngularObjectRegistry().getAll(note.getId(), null);
-      assertEquals(0, globalAngularObjects.size());
-
-      // global angular object is not saved to note
-      angularObjects = note.getAngularObjects(p1.getBindedInterpreter().getInterpreterGroup().getId());
-      assertEquals(0, angularObjects.size());
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p1 = note.addNewParagraph(anonymous);
+
+          // add local angular object
+          p1.setText("%spark z.angularBind(\"name\", \"world\")");
+          note.run(p1.getId(), true);
+          assertEquals(Status.FINISHED, p1.getStatus());
+          // angular object is saved to InterpreterGroup's AngularObjectRegistry
+          List<AngularObject> angularObjects;
+          try {
+            angularObjects = p1.getBindedInterpreter().getInterpreterGroup()
+                    .getAngularObjectRegistry().getAll(note.getId(), null);
+
+            assertEquals(1, angularObjects.size());
+            assertEquals("name", angularObjects.get(0).getName());
+            assertEquals("world", angularObjects.get(0).get());
+          } catch (InterpreterNotFoundException e) {
+            fail();
+          }
+          // angular object is saved to note as well.
+          try {
+            angularObjects = note.getAngularObjects(p1.getBindedInterpreter().getInterpreterGroup().getId());
+
+            assertEquals(1, angularObjects.size());
+            assertEquals("name", angularObjects.get(0).getName());
+            assertEquals("world", angularObjects.get(0).get());
+          } catch (InterpreterNotFoundException e) {
+            fail();
+          }
+
+          // remove local angular object
+          Paragraph p2 = note.addNewParagraph(anonymous);
+          p2.setText("%spark z.angularUnbind(\"name\")");
+          note.run(p2.getId(), true);
+          assertEquals(Status.FINISHED, p2.getStatus());
+          try {
+            angularObjects = p1.getBindedInterpreter().getInterpreterGroup().getAngularObjectRegistry()
+                    .getAll(note.getId(), null);
+            assertEquals(0, angularObjects.size());
+          } catch (InterpreterNotFoundException e) {
+            fail();
+          }
+
+
+          try {
+            angularObjects = note.getAngularObjects(p1.getBindedInterpreter().getInterpreterGroup().getId());
+            assertEquals(0, angularObjects.size());
+          } catch (InterpreterNotFoundException e) {
+            fail();
+          }
+
+
+          // add global angular object
+          Paragraph p3 = note.addNewParagraph(anonymous);
+          p3.setText("%spark z.angularBindGlobal(\"name2\", \"world2\")");
+          note.run(p3.getId(), true);
+          assertEquals(Status.FINISHED, p3.getStatus());
+          List<AngularObject> globalAngularObjects;
+          try {
+            globalAngularObjects = p3.getBindedInterpreter().getInterpreterGroup()
+                    .getAngularObjectRegistry().getAll(null, null);
+            assertEquals(1, globalAngularObjects.size());
+            assertEquals("name2", globalAngularObjects.get(0).getName());
+            assertEquals("world2", globalAngularObjects.get(0).get());
+          } catch (InterpreterNotFoundException e) {
+            fail();
+          }
+
+
+          // global angular object is not saved to note
+          try {
+            angularObjects = note.getAngularObjects(p1.getBindedInterpreter().getInterpreterGroup().getId());
+            assertEquals(0, angularObjects.size());
+          } catch (InterpreterNotFoundException e) {
+            fail();
+          }
+
+
+          // remove global angular object
+          Paragraph p4 = note.addNewParagraph(anonymous);
+          p4.setText("%spark z.angularUnbindGlobal(\"name2\")");
+          note.run(p4.getId(), true);
+          assertEquals(Status.FINISHED, p4.getStatus());
+          try {
+            globalAngularObjects = p4.getBindedInterpreter().getInterpreterGroup()
+                    .getAngularObjectRegistry().getAll(note.getId(), null);
+            assertEquals(0, globalAngularObjects.size());
+          } catch (InterpreterNotFoundException e) {
+            fail();
+          }
+
+
+          // global angular object is not saved to note
+          try {
+            angularObjects = note.getAngularObjects(p1.getBindedInterpreter().getInterpreterGroup().getId());
+            assertEquals(0, angularObjects.size());
+          } catch (InterpreterNotFoundException e) {
+            fail();
+          }
+
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testScalaNoteDynamicForms() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p1 = note.addNewParagraph(anonymous);
-
-      // create TextBox
-      p1.setText("%spark z.noteTextbox(\"name\", \"world\")");
-      note.run(p1.getId(), true);
-      assertEquals(Status.FINISHED, p1.getStatus());
-      Input input = p1.getNote().getNoteForms().get("name");
-      assertTrue(input instanceof TextBox);
-      TextBox inputTextBox = (TextBox) input;
-      assertEquals("name", inputTextBox.getDisplayName());
-      assertEquals("world", inputTextBox.getDefaultValue());
-      assertEquals("world", p1.getNote().getNoteParams().get("name"));
-
-      Paragraph p2 = note.addNewParagraph(anonymous);
-      p2.setText("%md hello $${name}");
-      note.run(p2.getId(), true);
-      assertEquals(Status.FINISHED, p2.getStatus());
-      assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("hello world"));
-
-      // create Select
-      p1.setText("%spark z.noteSelect(\"language\", Seq((\"java\" -> \"JAVA\"), (\"scala\" -> \"SCALA\")), \"java\")");
-      note.run(p1.getId(), true);
-      assertEquals(Status.FINISHED, p1.getStatus());
-      input = p1.getNote().getNoteForms().get("language");
-      assertTrue(input instanceof Select);
-      Select select = (Select) input;
-      assertEquals("language", select.getDisplayName());
-      assertEquals("java", select.getDefaultValue());
-      assertEquals("java", p1.getNote().getNoteParams().get("language"));
-
-      p2 = note.addNewParagraph(anonymous);
-      p2.setText("%md hello $${language}");
-      note.run(p2.getId(), true);
-      assertEquals(Status.FINISHED, p2.getStatus());
-      assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("hello java"));
-
-      // create Checkbox
-      p1.setText("%spark z.noteCheckbox(\"languages\", Seq((\"java\" -> \"JAVA\"), (\"scala\" -> \"SCALA\")), Seq(\"java\", \"scala\"))");
-      note.run(p1.getId(), true);
-      assertEquals(Status.FINISHED, p1.getStatus());
-      input = p1.getNote().getNoteForms().get("languages");
-      assertTrue(input instanceof CheckBox);
-      CheckBox checkbox = (CheckBox) input;
-      assertEquals("languages", checkbox.getDisplayName());
-      assertArrayEquals(new Object[]{"java", "scala"}, checkbox.getDefaultValue());
-      assertEquals(Arrays.asList("java", "scala"), p1.getNote().getNoteParams().get("languages"));
-
-      p2 = note.addNewParagraph(anonymous);
-      p2.setText("%md hello $${checkbox:languages}");
-      note.run(p2.getId(), true);
-      assertEquals(Status.FINISHED, p2.getStatus());
-      assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("hello java,scala"));
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p1 = note.addNewParagraph(anonymous);
+
+          // create TextBox
+          p1.setText("%spark z.noteTextbox(\"name\", \"world\")");
+          note.run(p1.getId(), true);
+          assertEquals(Status.FINISHED, p1.getStatus());
+          Input input = p1.getNote().getNoteForms().get("name");
+          assertTrue(input instanceof TextBox);
+          TextBox inputTextBox = (TextBox) input;
+          assertEquals("name", inputTextBox.getDisplayName());
+          assertEquals("world", inputTextBox.getDefaultValue());
+          assertEquals("world", p1.getNote().getNoteParams().get("name"));
+
+          Paragraph p2 = note.addNewParagraph(anonymous);
+          p2.setText("%md hello $${name}");
+          note.run(p2.getId(), true);
+          assertEquals(Status.FINISHED, p2.getStatus());
+          assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("hello world"));
+
+          // create Select
+          p1.setText("%spark z.noteSelect(\"language\", Seq((\"java\" -> \"JAVA\"), (\"scala\" -> \"SCALA\")), \"java\")");
+          note.run(p1.getId(), true);
+          assertEquals(Status.FINISHED, p1.getStatus());
+          input = p1.getNote().getNoteForms().get("language");
+          assertTrue(input instanceof Select);
+          Select select = (Select) input;
+          assertEquals("language", select.getDisplayName());
+          assertEquals("java", select.getDefaultValue());
+          assertEquals("java", p1.getNote().getNoteParams().get("language"));
+
+          p2 = note.addNewParagraph(anonymous);
+          p2.setText("%md hello $${language}");
+          note.run(p2.getId(), true);
+          assertEquals(Status.FINISHED, p2.getStatus());
+          assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("hello java"));
+
+          // create Checkbox
+          p1.setText("%spark z.noteCheckbox(\"languages\", Seq((\"java\" -> \"JAVA\"), (\"scala\" -> \"SCALA\")), Seq(\"java\", \"scala\"))");
+          note.run(p1.getId(), true);
+          assertEquals(Status.FINISHED, p1.getStatus());
+          input = p1.getNote().getNoteForms().get("languages");
+          assertTrue(input instanceof CheckBox);
+          CheckBox checkbox = (CheckBox) input;
+          assertEquals("languages", checkbox.getDisplayName());
+          assertArrayEquals(new Object[]{"java", "scala"}, checkbox.getDefaultValue());
+          assertEquals(Arrays.asList("java", "scala"), p1.getNote().getNoteParams().get("languages"));
+
+          p2 = note.addNewParagraph(anonymous);
+          p2.setText("%md hello $${checkbox:languages}");
+          note.run(p2.getId(), true);
+          assertEquals(Status.FINISHED, p2.getStatus());
+          assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("hello java,scala"));
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testPythonNoteDynamicForms() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p1 = note.addNewParagraph(anonymous);
-
-      // create TextBox
-      p1.setText("%spark.pyspark z.noteTextbox(\"name\", \"world\")");
-      note.run(p1.getId(), true);
-      assertEquals(Status.FINISHED, p1.getStatus());
-      Input input = p1.getNote().getNoteForms().get("name");
-      assertTrue(input instanceof TextBox);
-      TextBox inputTextBox = (TextBox) input;
-      assertEquals("name", inputTextBox.getDisplayName());
-      assertEquals("world", inputTextBox.getDefaultValue());
-      assertEquals("world", p1.getNote().getNoteParams().get("name"));
-
-      Paragraph p2 = note.addNewParagraph(anonymous);
-      p2.setText("%md hello $${name}");
-      note.run(p2.getId(), true);
-      assertEquals(Status.FINISHED, p2.getStatus());
-      assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("hello world"));
-
-      // create Select
-      p1.setText("%spark.pyspark z.noteSelect('language', [('java', 'JAVA'), ('scala', 'SCALA')], 'java')");
-      note.run(p1.getId(), true);
-      assertEquals(Status.FINISHED, p1.getStatus());
-      input = p1.getNote().getNoteForms().get("language");
-      assertTrue(input instanceof Select);
-      Select select = (Select) input;
-      assertEquals("language", select.getDisplayName());
-      assertEquals("java", select.getDefaultValue());
-      assertEquals("java", p1.getNote().getNoteParams().get("language"));
-
-      p2 = note.addNewParagraph(anonymous);
-      p2.setText("%md hello $${language}");
-      note.run(p2.getId(), true);
-      assertEquals(Status.FINISHED, p2.getStatus());
-      assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("hello java"));
-
-      // create Checkbox
-      p1.setText("%spark.pyspark z.noteCheckbox('languages', [('java', 'JAVA'), ('scala', 'SCALA')], ['java', 'scala'])");
-      note.run(p1.getId(), true);
-      assertEquals(Status.FINISHED, p1.getStatus());
-      input = p1.getNote().getNoteForms().get("languages");
-      assertTrue(input instanceof CheckBox);
-      CheckBox checkbox = (CheckBox) input;
-      assertEquals("languages", checkbox.getDisplayName());
-      assertArrayEquals(new Object[]{"java", "scala"}, checkbox.getDefaultValue());
-      assertEquals(Arrays.asList("java", "scala"), p1.getNote().getNoteParams().get("languages"));
-
-      p2 = note.addNewParagraph(anonymous);
-      p2.setText("%md hello $${checkbox:languages}");
-      note.run(p2.getId(), true);
-      assertEquals(Status.FINISHED, p2.getStatus());
-      assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("hello java,scala"));
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p1 = note.addNewParagraph(anonymous);
+
+          // create TextBox
+          p1.setText("%spark.pyspark z.noteTextbox(\"name\", \"world\")");
+          note.run(p1.getId(), true);
+          assertEquals(Status.FINISHED, p1.getStatus());
+          Input input = p1.getNote().getNoteForms().get("name");
+          assertTrue(input instanceof TextBox);
+          TextBox inputTextBox = (TextBox) input;
+          assertEquals("name", inputTextBox.getDisplayName());
+          assertEquals("world", inputTextBox.getDefaultValue());
+          assertEquals("world", p1.getNote().getNoteParams().get("name"));
+
+          Paragraph p2 = note.addNewParagraph(anonymous);
+          p2.setText("%md hello $${name}");
+          note.run(p2.getId(), true);
+          assertEquals(Status.FINISHED, p2.getStatus());
+          assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("hello world"));
+
+          // create Select
+          p1.setText("%spark.pyspark z.noteSelect('language', [('java', 'JAVA'), ('scala', 'SCALA')], 'java')");
+          note.run(p1.getId(), true);
+          assertEquals(Status.FINISHED, p1.getStatus());
+          input = p1.getNote().getNoteForms().get("language");
+          assertTrue(input instanceof Select);
+          Select select = (Select) input;
+          assertEquals("language", select.getDisplayName());
+          assertEquals("java", select.getDefaultValue());
+          assertEquals("java", p1.getNote().getNoteParams().get("language"));
+
+          p2 = note.addNewParagraph(anonymous);
+          p2.setText("%md hello $${language}");
+          note.run(p2.getId(), true);
+          assertEquals(Status.FINISHED, p2.getStatus());
+          assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("hello java"));
+
+          // create Checkbox
+          p1.setText("%spark.pyspark z.noteCheckbox('languages', [('java', 'JAVA'), ('scala', 'SCALA')], ['java', 'scala'])");
+          note.run(p1.getId(), true);
+          assertEquals(Status.FINISHED, p1.getStatus());
+          input = p1.getNote().getNoteForms().get("languages");
+          assertTrue(input instanceof CheckBox);
+          CheckBox checkbox = (CheckBox) input;
+          assertEquals("languages", checkbox.getDisplayName());
+          assertArrayEquals(new Object[]{"java", "scala"}, checkbox.getDefaultValue());
+          assertEquals(Arrays.asList("java", "scala"), p1.getNote().getNoteParams().get("languages"));
+
+          p2 = note.addNewParagraph(anonymous);
+          p2.setText("%md hello $${checkbox:languages}");
+          note.run(p2.getId(), true);
+          assertEquals(Status.FINISHED, p2.getStatus());
+          assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("hello java,scala"));
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testRNoteDynamicForms() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p1 = note.addNewParagraph(anonymous);
-
-      // create TextBox
-      p1.setText("%spark.r z.noteTextbox(\"name\", \"world\")");
-      note.run(p1.getId(), true);
-      assertEquals(Status.FINISHED, p1.getStatus());
-      Input input = p1.getNote().getNoteForms().get("name");
-      assertTrue(input instanceof TextBox);
-      TextBox inputTextBox = (TextBox) input;
-      assertEquals("name", inputTextBox.getDisplayName());
-      assertEquals("world", inputTextBox.getDefaultValue());
-      assertEquals("world", p1.getNote().getNoteParams().get("name"));
-
-      Paragraph p2 = note.addNewParagraph(anonymous);
-      p2.setText("%md hello $${name}");
-      note.run(p2.getId(), true);
-      assertEquals(Status.FINISHED, p2.getStatus());
-      assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("hello world"));
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p1 = note.addNewParagraph(anonymous);
+
+          // create TextBox
+          p1.setText("%spark.r z.noteTextbox(\"name\", \"world\")");
+          note.run(p1.getId(), true);
+          assertEquals(Status.FINISHED, p1.getStatus());
+          Input input = p1.getNote().getNoteForms().get("name");
+          assertTrue(input instanceof TextBox);
+          TextBox inputTextBox = (TextBox) input;
+          assertEquals("name", inputTextBox.getDisplayName());
+          assertEquals("world", inputTextBox.getDefaultValue());
+          assertEquals("world", p1.getNote().getNoteParams().get("name"));
+
+          Paragraph p2 = note.addNewParagraph(anonymous);
+          p2.setText("%md hello $${name}");
+          note.run(p2.getId(), true);
+          assertEquals(Status.FINISHED, p2.getStatus());
+          assertTrue(p2.getReturn().toString(), p2.getReturn().toString().contains("hello world"));
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testConfInterpreter() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
       TestUtils.getInstance(Notebook.class).getInterpreterSettingManager().close();
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p = note.addNewParagraph(anonymous);
-      p.setText("%spark.conf spark.jars.packages\tcom.databricks:spark-csv_2.11:1.2.0");
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-
-      Paragraph p1 = note.addNewParagraph(anonymous);
-      p1.setText("%spark\nimport com.databricks.spark.csv._");
-      note.run(p1.getId(), true);
-      assertEquals(Status.FINISHED, p1.getStatus());
-
-      // test pyspark imports path
-      Paragraph p2 = note.addNewParagraph(anonymous);
-      p2.setText("%spark.pyspark\nimport sys\nsys.path");
-      note.run(p2.getId(), true);
-      assertEquals(Status.FINISHED, p2.getStatus());
-      assertTrue(p2.getReturn().toString().contains("databricks_spark"));
-
-      Paragraph p3 = note.addNewParagraph(anonymous);
-      p3.setText("%spark.ipyspark\nimport sys\nsys.path");
-      note.run(p3.getId(), true);
-      assertEquals(Status.FINISHED, p3.getStatus());
-      assertTrue(p3.getReturn().toString().contains("databricks_spark"));
-
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(anonymous);
+          p.setText("%spark.conf spark.jars.packages\tcom.databricks:spark-csv_2.11:1.2.0");
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+
+          Paragraph p1 = note.addNewParagraph(anonymous);
+          p1.setText("%spark\nimport com.databricks.spark.csv._");
+          note.run(p1.getId(), true);
+          assertEquals(Status.FINISHED, p1.getStatus());
+
+          // test pyspark imports path
+          Paragraph p2 = note.addNewParagraph(anonymous);
+          p2.setText("%spark.pyspark\nimport sys\nsys.path");
+          note.run(p2.getId(), true);
+          assertEquals(Status.FINISHED, p2.getStatus());
+          assertTrue(p2.getReturn().toString().contains("databricks_spark"));
+
+          Paragraph p3 = note.addNewParagraph(anonymous);
+          p3.setText("%spark.ipyspark\nimport sys\nsys.path");
+          note.run(p3.getId(), true);
+          assertEquals(Status.FINISHED, p3.getStatus());
+          assertTrue(p3.getReturn().toString().contains("databricks_spark"));
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testFailtoLaunchSpark() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
       TestUtils.getInstance(Notebook.class).getInterpreterSettingManager().close();
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p = note.addNewParagraph(anonymous);
-      p.setText("%spark.conf SPARK_HOME invalid_spark_home");
-      note.run(p.getId(), true);
-      assertEquals(Status.FINISHED, p.getStatus());
-
-      Paragraph p1 = note.addNewParagraph(anonymous);
-      p1.setText("%spark\nsc.version");
-      note.run(p1.getId(), true);
-      assertEquals(Status.ERROR, p1.getStatus());
-      assertTrue("Actual error message: " + p1.getReturn().message().get(0).getData(),
-              p1.getReturn().message().get(0).getData().contains("No such file or directory"));
-
-      // run it again, and get the same error
-      note.run(p1.getId(), true);
-      assertEquals(Status.ERROR, p1.getStatus());
-      assertTrue("Actual error message: " + p1.getReturn().message().get(0).getData(),
-              p1.getReturn().message().get(0).getData().contains("No such file or directory"));
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(anonymous);
+          p.setText("%spark.conf SPARK_HOME invalid_spark_home");
+          note.run(p.getId(), true);
+          assertEquals(Status.FINISHED, p.getStatus());
+
+          Paragraph p1 = note.addNewParagraph(anonymous);
+          p1.setText("%spark\nsc.version");
+          note.run(p1.getId(), true);
+          assertEquals(Status.ERROR, p1.getStatus());
+          assertTrue("Actual error message: " + p1.getReturn().message().get(0).getData(),
+                  p1.getReturn().message().get(0).getData().contains("No such file or directory"));
+
+          // run it again, and get the same error
+          note.run(p1.getId(), true);
+          assertEquals(Status.ERROR, p1.getStatus());
+          assertTrue("Actual error message: " + p1.getReturn().message().get(0).getData(),
+                  p1.getReturn().message().get(0).getData().contains("No such file or directory"));
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
       // reset SPARK_HOME, otherwise it will cause the following test fail
       InterpreterSetting sparkIntpSetting = TestUtils.getInstance(Notebook.class).getInterpreterSettingManager()
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
index 6c44495..8021e20 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
@@ -608,6 +608,10 @@ public class ZeppelinConfiguration {
     return StringUtils.split(getString(ConfVars.ZEPPELIN_NOTE_FILE_EXCLUDE_FIELDS), (","));
   }
 
+  public int getNoteCacheThreshold() {
+    return getInt(ConfVars.ZEPPELIN_NOTE_CACHE_THRESHOLD);
+  }
+
   public String getInterpreterPortRange() {
     return getString(ConfVars.ZEPPELIN_INTERPRETER_RPC_PORTRANGE);
   }
@@ -1080,6 +1084,7 @@ public class ZeppelinConfiguration {
     ZEPPELIN_JOBMANAGER_ENABLE("zeppelin.jobmanager.enable", false),
     ZEPPELIN_SPARK_ONLY_YARN_CLUSTER("zeppelin.spark.only_yarn_cluster", false),
     ZEPPELIN_SESSION_CHECK_INTERVAL("zeppelin.session.check_interval", 60 * 10 * 1000),
+    ZEPPELIN_NOTE_CACHE_THRESHOLD("zeppelin.note.cache.threshold", 50),
     ZEPPELIN_NOTE_FILE_EXCLUDE_FIELDS("zeppelin.note.file.exclude.fields", "");
 
     private String varName;
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/rest/HeliumRestApi.java b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/HeliumRestApi.java
index d3106a9..b34c532 100644
--- a/zeppelin-server/src/main/java/org/apache/zeppelin/rest/HeliumRestApi.java
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/HeliumRestApi.java
@@ -121,59 +121,62 @@ public class HeliumRestApi {
   @Path("suggest/{noteId}/{paragraphId}")
   public Response suggest(@PathParam("noteId") String noteId,
           @PathParam("paragraphId") String paragraphId) {
-    Note note;
     try {
-      note = notebook.getNote(noteId);
+      return notebook.processNote(noteId,
+        note -> {
+          if (note == null) {
+            return new JsonResponse<>(Response.Status.NOT_FOUND, "Note " + noteId + " not found").build();
+          }
+
+          Paragraph paragraph = note.getParagraph(paragraphId);
+          if (paragraph == null) {
+            return new JsonResponse<>(Response.Status.NOT_FOUND, "Paragraph " + paragraphId + " not found")
+                .build();
+          }
+          try {
+            return new JsonResponse<>(Response.Status.OK, "", helium.suggestApp(paragraph)).build();
+          } catch (RuntimeException e) {
+            logger.error(e.getMessage(), e);
+            return new JsonResponse<>(Response.Status.INTERNAL_SERVER_ERROR, e.getMessage()).build();
+          }
+        });
     } catch (IOException e) {
       return new JsonResponse<>(Response.Status.NOT_FOUND,
               "Fail to get note: " + noteId + "\n" + ExceptionUtils.getStackTrace(e)).build();
     }
-    if (note == null) {
-      return new JsonResponse<>(Response.Status.NOT_FOUND, "Note " + noteId + " not found").build();
-    }
-
-    Paragraph paragraph = note.getParagraph(paragraphId);
-    if (paragraph == null) {
-      return new JsonResponse<>(Response.Status.NOT_FOUND, "Paragraph " + paragraphId + " not found")
-          .build();
-    }
-    try {
-      return new JsonResponse<>(Response.Status.OK, "", helium.suggestApp(paragraph)).build();
-    } catch (RuntimeException e) {
-      logger.error(e.getMessage(), e);
-      return new JsonResponse<>(Response.Status.INTERNAL_SERVER_ERROR, e.getMessage()).build();
-    }
-
   }
 
   @POST
   @Path("load/{noteId}/{paragraphId}")
   public Response load(@PathParam("noteId") String noteId,
           @PathParam("paragraphId") String paragraphId, String heliumPackage) {
-    Note note;
     try {
-      note = notebook.getNote(noteId);
+      return notebook.processNote(noteId,
+        note -> {
+          if (note == null) {
+            return new JsonResponse<>(Response.Status.NOT_FOUND, "Note " + noteId + " not found").build();
+          }
+          Paragraph paragraph = note.getParagraph(paragraphId);
+          if (paragraph == null) {
+            return new JsonResponse<>(Response.Status.NOT_FOUND, "Paragraph " + paragraphId + " not found")
+                .build();
+          }
+          HeliumPackage pkg = HeliumPackage.fromJson(heliumPackage);
+          try {
+            return new JsonResponse<>(Response.Status.OK, "",
+                    helium.getApplicationFactory().loadAndRun(pkg, paragraph)).build();
+          } catch (RuntimeException e) {
+            logger.error(e.getMessage(), e);
+            return new JsonResponse<>(Response.Status.INTERNAL_SERVER_ERROR, e.getMessage()).build();
+          }
+        });
     } catch (IOException e) {
       return new JsonResponse<>(Response.Status.NOT_FOUND,
               "Fail to get note: " + noteId + "\n" + ExceptionUtils.getStackTrace(e)).build();
     }
-    if (note == null) {
-      return new JsonResponse<>(Response.Status.NOT_FOUND, "Note " + noteId + " not found").build();
-    }
 
-    Paragraph paragraph = note.getParagraph(paragraphId);
-    if (paragraph == null) {
-      return new JsonResponse<>(Response.Status.NOT_FOUND, "Paragraph " + paragraphId + " not found")
-          .build();
-    }
-    HeliumPackage pkg = HeliumPackage.fromJson(heliumPackage);
-    try {
-      return new JsonResponse<>(Response.Status.OK, "",
-              helium.getApplicationFactory().loadAndRun(pkg, paragraph)).build();
-    } catch (RuntimeException e) {
-      logger.error(e.getMessage(), e);
-      return new JsonResponse<>(Response.Status.INTERNAL_SERVER_ERROR, e.getMessage()).build();
-    }
+
+
   }
 
   @GET
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/rest/NotebookRestApi.java b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/NotebookRestApi.java
index 3d6bd26..a4b2004 100644
--- a/zeppelin-server/src/main/java/org/apache/zeppelin/rest/NotebookRestApi.java
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/NotebookRestApi.java
@@ -29,6 +29,7 @@ import java.util.Set;
 import javax.inject.Inject;
 import javax.inject.Singleton;
 import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
 import javax.ws.rs.GET;
 import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
@@ -253,57 +254,58 @@ public class NotebookRestApi extends AbstractRestApi {
     HashMap<String, HashSet<String>> permMap =
             GSON.fromJson(req, new TypeToken<HashMap<String, HashSet<String>>>() {
             }.getType());
-    Note note = notebook.getNote(noteId);
-    checkIfNoteIsNotNull(note, noteId);
-
-    HashSet<String> readers = permMap.get("readers");
-    HashSet<String> runners = permMap.get("runners");
-    HashSet<String> owners = permMap.get("owners");
-    HashSet<String> writers = permMap.get("writers");
-
-    LOGGER.info("Set permissions to note: {} with current user:{}, owners:{}, readers:{}, runners:{}, writers:{}",
-            noteId, principal, owners, readers, runners, writers);
-
-    // Set readers, if runners, writers and owners is empty -> set to user requesting the change
-    if (readers != null && !readers.isEmpty()) {
-      if (runners.isEmpty()) {
-        runners = new HashSet<>(Arrays.asList(authenticationService.getPrincipal()));
-      }
-      if (writers.isEmpty()) {
-        writers = new HashSet<>(Arrays.asList(authenticationService.getPrincipal()));
-      }
-      if (owners.isEmpty()) {
-        owners = new HashSet<>(Arrays.asList(authenticationService.getPrincipal()));
-      }
-    }
-    // Set runners, if writers and owners is empty -> set to user requesting the change
-    if (runners != null && !runners.isEmpty()) {
-      if (writers.isEmpty()) {
-        writers = new HashSet<>(Arrays.asList(authenticationService.getPrincipal()));
-      }
-      if (owners.isEmpty()) {
-        owners = new HashSet<>(Arrays.asList(authenticationService.getPrincipal()));
-      }
-    }
-    // Set writers, if owners is empty -> set to user requesting the change
-    if (writers != null && !writers.isEmpty()) {
-      if (owners.isEmpty()) {
-        owners = new HashSet<>(Arrays.asList(authenticationService.getPrincipal()));
-      }
-    }
-
-    authorizationService.setReaders(noteId, readers);
-    authorizationService.setRunners(noteId, runners);
-    authorizationService.setWriters(noteId, writers);
-    authorizationService.setOwners(noteId, owners);
-    LOGGER.debug("After set permissions {} {} {} {}", authorizationService.getOwners(noteId),
-            authorizationService.getReaders(noteId), authorizationService.getRunners(noteId),
-            authorizationService.getWriters(noteId));
-    AuthenticationInfo subject = new AuthenticationInfo(authenticationService.getPrincipal());
-    authorizationService.saveNoteAuth(noteId, subject);
-    notebookServer.broadcastNote(note);
-    notebookServer.broadcastNoteList(subject, userAndRoles);
-    return new JsonResponse<>(Status.OK).build();
+    return notebook.processNote(noteId,
+      note -> {
+        checkIfNoteIsNotNull(note, noteId);
+        HashSet<String> readers = permMap.get("readers");
+        HashSet<String> runners = permMap.get("runners");
+        HashSet<String> owners = permMap.get("owners");
+        HashSet<String> writers = permMap.get("writers");
+
+        LOGGER.info("Set permissions to note: {} with current user:{}, owners:{}, readers:{}, runners:{}, writers:{}",
+                noteId, principal, owners, readers, runners, writers);
+
+        // Set readers, if runners, writers and owners is empty -> set to user requesting the change
+        if (readers != null && !readers.isEmpty()) {
+          if (runners.isEmpty()) {
+            runners = new HashSet<>(Arrays.asList(authenticationService.getPrincipal()));
+          }
+          if (writers.isEmpty()) {
+            writers = new HashSet<>(Arrays.asList(authenticationService.getPrincipal()));
+          }
+          if (owners.isEmpty()) {
+            owners = new HashSet<>(Arrays.asList(authenticationService.getPrincipal()));
+          }
+        }
+        // Set runners, if writers and owners is empty -> set to user requesting the change
+        if (runners != null && !runners.isEmpty()) {
+          if (writers.isEmpty()) {
+            writers = new HashSet<>(Arrays.asList(authenticationService.getPrincipal()));
+          }
+          if (owners.isEmpty()) {
+            owners = new HashSet<>(Arrays.asList(authenticationService.getPrincipal()));
+          }
+        }
+        // Set writers, if owners is empty -> set to user requesting the change
+        if (writers != null && !writers.isEmpty()) {
+          if (owners.isEmpty()) {
+            owners = new HashSet<>(Arrays.asList(authenticationService.getPrincipal()));
+          }
+        }
+
+        authorizationService.setReaders(noteId, readers);
+        authorizationService.setRunners(noteId, runners);
+        authorizationService.setWriters(noteId, writers);
+        authorizationService.setOwners(noteId, owners);
+        LOGGER.debug("After set permissions {} {} {} {}", authorizationService.getOwners(noteId),
+                authorizationService.getReaders(noteId), authorizationService.getRunners(noteId),
+                authorizationService.getWriters(noteId));
+        AuthenticationInfo subject = new AuthenticationInfo(authenticationService.getPrincipal());
+        authorizationService.saveNoteAuth(noteId, subject);
+        notebookServer.broadcastNote(note);
+        notebookServer.broadcastNoteList(subject, userAndRoles);
+        return new JsonResponse<>(Status.OK).build();
+      });
   }
 
   /**
@@ -333,9 +335,8 @@ public class NotebookRestApi extends AbstractRestApi {
   @ZeppelinApi
   public Response getNote(@PathParam("noteId") String noteId,
                           @QueryParam("reload") boolean reload) throws IOException {
-    Note note =
-            notebookService.getNote(noteId, reload, getServiceContext(), new RestServiceCallback<>());
-    return new JsonResponse<>(Status.OK, "", note).build();
+    return notebookService.getNote(noteId, reload,  getServiceContext(), new RestServiceCallback<>(),
+      note -> new JsonResponse<>(Status.OK, "", note).build());
   }
 
   /**
@@ -366,9 +367,9 @@ public class NotebookRestApi extends AbstractRestApi {
   @Path("import")
   @ZeppelinApi
   public Response importNote(@QueryParam("notePath") String notePath, String noteJson) throws IOException {
-    Note note = notebookService.importNote(notePath, noteJson, getServiceContext(),
+    String noteId = notebookService.importNote(notePath, noteJson, getServiceContext(),
             new RestServiceCallback<>());
-    return new JsonResponse<>(Status.OK, "", note.getId()).build();
+    return new JsonResponse<>(Status.OK, "", noteId).build();
   }
 
   /**
@@ -388,20 +389,23 @@ public class NotebookRestApi extends AbstractRestApi {
     if (StringUtils.isBlank(defaultInterpreterGroup)) {
       defaultInterpreterGroup = zConf.getString(ZeppelinConfiguration.ConfVars.ZEPPELIN_INTERPRETER_GROUP_DEFAULT);
     }
-    Note note = notebookService.createNote(
+    String noteId = notebookService.createNote(
             request.getName(),
             defaultInterpreterGroup,
             request.getAddingEmptyParagraph(),
             getServiceContext(),
             new RestServiceCallback<>());
-    AuthenticationInfo subject = new AuthenticationInfo(authenticationService.getPrincipal());
-    if (request.getParagraphs() != null) {
-      for (NewParagraphRequest paragraphRequest : request.getParagraphs()) {
-        Paragraph p = note.addNewParagraph(subject);
-        initParagraph(p, paragraphRequest, user);
-      }
-    }
-    return new JsonResponse<>(Status.OK, "", note.getId()).build();
+    return notebook.processNote(noteId,
+      note -> {
+        AuthenticationInfo subject = new AuthenticationInfo(authenticationService.getPrincipal());
+        if (request.getParagraphs() != null) {
+          for (NewParagraphRequest paragraphRequest : request.getParagraphs()) {
+            Paragraph p = note.addNewParagraph(subject);
+            initParagraph(p, paragraphRequest, user);
+          }
+        }
+        return new JsonResponse<>(Status.OK, "", note.getId()).build();
+      });
   }
 
   /**
@@ -451,7 +455,7 @@ public class NotebookRestApi extends AbstractRestApi {
       newNoteName = request.getName();
     }
     AuthenticationInfo subject = new AuthenticationInfo(authenticationService.getPrincipal());
-    Note newNote = notebookService.cloneNote(noteId, newNoteName, getServiceContext(),
+    String newNoteId = notebookService.cloneNote(noteId, newNoteName, getServiceContext(),
             new RestServiceCallback<Note>() {
               @Override
               public void onSuccess(Note newNote, ServiceContext context) throws IOException {
@@ -459,7 +463,7 @@ public class NotebookRestApi extends AbstractRestApi {
                 notebookServer.broadcastNoteList(subject, context.getUserAndRoles());
               }
             });
-    return new JsonResponse<>(Status.OK, "", newNote.getId()).build();
+    return new JsonResponse<>(Status.OK, "", newNoteId).build();
   }
 
   /**
@@ -508,24 +512,24 @@ public class NotebookRestApi extends AbstractRestApi {
 
     String user = authenticationService.getPrincipal();
     LOGGER.info("Insert paragraph {} {}", noteId, message);
-
-    Note note = notebook.getNote(noteId);
-    checkIfNoteIsNotNull(note, noteId);
-    checkIfUserCanWrite(noteId, "Insufficient privileges you cannot add paragraph to this note");
-
-    NewParagraphRequest request = NewParagraphRequest.fromJson(message);
     AuthenticationInfo subject = new AuthenticationInfo(user);
-    Paragraph p;
-    Double indexDouble = request.getIndex();
-    if (indexDouble == null) {
-      p = note.addNewParagraph(subject);
-    } else {
-      p = note.insertNewParagraph(indexDouble.intValue(), subject);
-    }
-    initParagraph(p, request, user);
-    notebook.saveNote(note, subject);
-    notebookServer.broadcastNote(note);
-    return new JsonResponse<>(Status.OK, "", p.getId()).build();
+    return notebook.processNote(noteId,
+      note -> {
+        checkIfNoteIsNotNull(note, noteId);
+        checkIfUserCanWrite(noteId, "Insufficient privileges you cannot add paragraph to this note");
+        NewParagraphRequest request = NewParagraphRequest.fromJson(message);
+        Paragraph p;
+        Double indexDouble = request.getIndex();
+        if (indexDouble == null) {
+          p = note.addNewParagraph(subject);
+        } else {
+          p = note.insertNewParagraph(indexDouble.intValue(), subject);
+        }
+        initParagraph(p, request, user);
+        notebook.saveNote(note, subject);
+        notebookServer.broadcastNote(note);
+        return new JsonResponse<>(Status.OK, "", p.getId()).build();
+      });
   }
 
   /**
@@ -540,12 +544,15 @@ public class NotebookRestApi extends AbstractRestApi {
   @ZeppelinApi
   public Response getParagraph(@PathParam("noteId") String noteId,
                                @PathParam("paragraphId") String paragraphId) throws IOException {
-    Note note = notebook.getNote(noteId);
-    checkIfNoteIsNotNull(note, noteId);
-    checkIfUserCanRead(noteId, "Insufficient privileges you cannot get this paragraph");
-    Paragraph p = note.getParagraph(paragraphId);
-    checkIfParagraphIsNotNull(p, paragraphId);
-    return new JsonResponse<>(Status.OK, "", p).build();
+
+    return notebook.processNote(noteId,
+      note -> {
+        checkIfNoteIsNotNull(note, noteId);
+        checkIfUserCanRead(noteId, "Insufficient privileges you cannot get this paragraph");
+        Paragraph p = note.getParagraph(paragraphId);
+        checkIfParagraphIsNotNull(p, paragraphId);
+        return new JsonResponse<>(Status.OK, "", p).build();
+      });
   }
 
   /**
@@ -563,23 +570,25 @@ public class NotebookRestApi extends AbstractRestApi {
 
     String user = authenticationService.getPrincipal();
     LOGGER.info("{} will update paragraph {} {}", user, noteId, paragraphId);
-    Note note = notebook.getNote(noteId);
-    checkIfNoteIsNotNull(note, noteId);
-    checkIfUserCanWrite(noteId, "Insufficient privileges you cannot update this paragraph");
-    Paragraph p = note.getParagraph(paragraphId);
-    checkIfParagraphIsNotNull(p, paragraphId);
+    return notebook.processNote(noteId,
+      note -> {
+        checkIfNoteIsNotNull(note, noteId);
+        checkIfUserCanWrite(noteId, "Insufficient privileges you cannot update this paragraph");
+        Paragraph p = note.getParagraph(paragraphId);
+        checkIfParagraphIsNotNull(p, paragraphId);
 
-    UpdateParagraphRequest updatedParagraph = GSON.fromJson(message, UpdateParagraphRequest.class);
-    p.setText(updatedParagraph.getText());
+        UpdateParagraphRequest updatedParagraph = GSON.fromJson(message, UpdateParagraphRequest.class);
+        p.setText(updatedParagraph.getText());
 
-    if (updatedParagraph.getTitle() != null) {
-      p.setTitle(updatedParagraph.getTitle());
-    }
+        if (updatedParagraph.getTitle() != null) {
+          p.setTitle(updatedParagraph.getTitle());
+        }
 
-    AuthenticationInfo subject = new AuthenticationInfo(user);
-    notebook.saveNote(note, subject);
-    notebookServer.broadcastParagraph(note, p, MSG_ID_NOT_DEFINED);
-    return new JsonResponse<>(Status.OK, "").build();
+        AuthenticationInfo subject = new AuthenticationInfo(user);
+        notebook.saveNote(note, subject);
+        notebookServer.broadcastParagraph(note, p, MSG_ID_NOT_DEFINED);
+        return new JsonResponse<>(Status.OK, "").build();
+      });
   }
 
   /**
@@ -600,18 +609,19 @@ public class NotebookRestApi extends AbstractRestApi {
 
     String user = authenticationService.getPrincipal();
     LOGGER.info("{} will update paragraph config {} {}", user, noteId, paragraphId);
-
-    Note note = notebook.getNote(noteId);
-    checkIfNoteIsNotNull(note, noteId);
-    checkIfUserCanWrite(noteId, "Insufficient privileges you cannot update this paragraph config");
-    Paragraph p = note.getParagraph(paragraphId);
-    checkIfParagraphIsNotNull(p, paragraphId);
-
-    Map<String, Object> newConfig = GSON.fromJson(message, HashMap.class);
-    configureParagraph(p, newConfig, user);
-    AuthenticationInfo subject = new AuthenticationInfo(user);
-    notebook.saveNote(note, subject);
-    return new JsonResponse<>(Status.OK, "", p).build();
+    return notebook.processNote(noteId,
+      note -> {
+        checkIfNoteIsNotNull(note, noteId);
+        checkIfUserCanWrite(noteId, "Insufficient privileges you cannot update this paragraph config");
+        Paragraph p = note.getParagraph(paragraphId);
+        checkIfParagraphIsNotNull(p, paragraphId);
+
+        Map<String, Object> newConfig = GSON.fromJson(message, HashMap.class);
+        configureParagraph(p, newConfig, user);
+        AuthenticationInfo subject = new AuthenticationInfo(user);
+        notebook.saveNote(note, subject);
+        return new JsonResponse<>(Status.OK, "", p).build();
+      });
   }
 
   /**
@@ -671,10 +681,10 @@ public class NotebookRestApi extends AbstractRestApi {
   public Response nextSessionParagraph(@PathParam("noteId") String noteId,
                                        @QueryParam("maxParagraph") int maxParagraph) throws IOException {
 
-    Paragraph p = notebookService.getNextSessionParagraph(noteId, maxParagraph,
+    String paragraphId = notebookService.getNextSessionParagraphId(noteId, maxParagraph,
             getServiceContext(),
             new RestServiceCallback<>());
-    return new JsonResponse<>(Status.OK, p.getId()).build();
+    return new JsonResponse<>(Status.OK, paragraphId).build();
   }
 
   /**
@@ -709,34 +719,34 @@ public class NotebookRestApi extends AbstractRestApi {
   @Path("job/{noteId}")
   @ZeppelinApi
   public Response runNoteJobs(@PathParam("noteId") String noteId,
-                              @QueryParam("blocking") Boolean blocking,
-                              @QueryParam("isolated") Boolean isolated,
+                              @DefaultValue("false") @QueryParam("blocking") boolean blocking,
+                              @DefaultValue("false") @QueryParam("isolated") boolean isolated,
                               String message)
       throws Exception, IllegalArgumentException {
 
-    if (blocking == null) {
-      blocking = false;
-    }
-    if (isolated == null) {
-      isolated = false;
-    }
     Map<String, Object> params = new HashMap<>();
     if (!StringUtils.isEmpty(message)) {
       ParametersRequest request =
               ParametersRequest.fromJson(message);
-      params = request.getParams();
+      params.putAll(request.getParams());
     }
 
     LOGGER.info("Run note jobs, noteId: {}, blocking: {}, isolated: {}, params: {}", noteId, blocking, isolated, params);
-    Note note = notebook.getNote(noteId);
-    AuthenticationInfo subject = new AuthenticationInfo(authenticationService.getPrincipal());
-    subject.setRoles(authenticationService.getAssociatedRoles());
-    checkIfNoteIsNotNull(note, noteId);
-    checkIfUserCanRun(noteId, "Insufficient privileges you cannot run job for this note");
+    return notebook.processNote(noteId,
+      note -> {
+        AuthenticationInfo subject = new AuthenticationInfo(authenticationService.getPrincipal());
+        subject.setRoles(authenticationService.getAssociatedRoles());
+        checkIfNoteIsNotNull(note, noteId);
+        checkIfUserCanRun(noteId, "Insufficient privileges you cannot run job for this note");
+        //TODO(zjffdu), can we run a note via rest api when cron is enabled ?
+        try {
+          note.runAll(subject, blocking, isolated, params);
+          return new JsonResponse<>(Status.OK).build();
+        } catch (Exception e) {
+          return new JsonResponse<>(Status.INTERNAL_SERVER_ERROR, "Fail to run note").build();
+        }
+      });
 
-    //TODO(zjffdu), can we run a note via rest api when cron is enabled ?
-    note.runAll(subject, blocking, isolated, params);
-    return new JsonResponse<>(Status.OK).build();
   }
 
   /**
@@ -754,16 +764,17 @@ public class NotebookRestApi extends AbstractRestApi {
       throws IOException, IllegalArgumentException {
 
     LOGGER.info("Stop note jobs {} ", noteId);
-    Note note = notebook.getNote(noteId);
-    checkIfNoteIsNotNull(note, noteId);
-    checkIfUserCanRun(noteId, "Insufficient privileges you cannot stop this job for this note");
-
-    for (Paragraph p : note.getParagraphs()) {
-      if (!p.isTerminated()) {
-        p.abort();
-      }
-    }
-    return new JsonResponse<>(Status.OK).build();
+    return notebook.processNote(noteId,
+      note -> {
+        checkIfNoteIsNotNull(note, noteId);
+        checkIfUserCanRun(noteId, "Insufficient privileges you cannot stop this job for this note");
+        for (Paragraph p : note.getParagraphs()) {
+          if (!p.isTerminated()) {
+            p.abort();
+          }
+        }
+        return new JsonResponse<>(Status.OK).build();
+      });
   }
 
   /**
@@ -781,11 +792,12 @@ public class NotebookRestApi extends AbstractRestApi {
       throws IOException, IllegalArgumentException {
 
     LOGGER.info("Get note job status.");
-    Note note = notebook.getNote(noteId);
-    checkIfNoteIsNotNull(note, noteId);
-    checkIfUserCanRead(noteId, "Insufficient privileges you cannot get job status");
-
-    return new JsonResponse<>(Status.OK, null, new NoteJobStatus(note)).build();
+    return notebook.processNote(noteId,
+      note -> {
+        checkIfNoteIsNotNull(note, noteId);
+        checkIfUserCanRead(noteId, "Insufficient privileges you cannot get job status");
+        return new JsonResponse<>(Status.OK, null, new NoteJobStatus(note)).build();
+      });
   }
 
   /**
@@ -805,14 +817,15 @@ public class NotebookRestApi extends AbstractRestApi {
       throws IOException, IllegalArgumentException {
 
     LOGGER.info("Get note paragraph job status.");
-    Note note = notebook.getNote(noteId);
-    checkIfNoteIsNotNull(note, noteId);
-    checkIfUserCanRead(noteId, "Insufficient privileges you cannot get job status");
-
-    Paragraph paragraph = note.getParagraph(paragraphId);
-    checkIfParagraphIsNotNull(paragraph, paragraphId);
-
-    return new JsonResponse<>(Status.OK, null, new ParagraphJobStatus(paragraph)).build();
+    return notebook.processNote(noteId,
+      note -> {
+        checkIfNoteIsNotNull(note, noteId);
+        checkIfUserCanRead(noteId, "Insufficient privileges you cannot get job status");
+
+        Paragraph paragraph = note.getParagraph(paragraphId);
+        checkIfParagraphIsNotNull(paragraph, paragraphId);
+        return new JsonResponse<>(Status.OK, null, new ParagraphJobStatus(paragraph)).build();
+      });
   }
 
   /**
@@ -835,21 +848,23 @@ public class NotebookRestApi extends AbstractRestApi {
 
     LOGGER.info("Run paragraph job asynchronously {} {} {}", noteId, paragraphId, message);
 
-    Note note = notebook.getNote(noteId);
-    checkIfNoteIsNotNull(note, noteId);
-    Paragraph paragraph = note.getParagraph(paragraphId);
-    checkIfParagraphIsNotNull(paragraph, paragraphId);
-
-    Map<String, Object> params = new HashMap<>();
-    if (!StringUtils.isEmpty(message)) {
-      ParametersRequest request =
-              ParametersRequest.fromJson(message);
-      params = request.getParams();
-    }
-    notebookService.runParagraph(noteId, paragraphId, paragraph.getTitle(),
-            paragraph.getText(), params, new HashMap<>(), sessionId,
-            false, false, getServiceContext(), new RestServiceCallback<>());
-    return new JsonResponse<>(Status.OK).build();
+    return notebook.processNote(noteId,
+      note -> {
+        checkIfNoteIsNotNull(note, noteId);
+        Paragraph paragraph = note.getParagraph(paragraphId);
+        checkIfParagraphIsNotNull(paragraph, paragraphId);
+
+        Map<String, Object> params = new HashMap<>();
+        if (!StringUtils.isEmpty(message)) {
+          ParametersRequest request =
+                  ParametersRequest.fromJson(message);
+          params = request.getParams();
+        }
+        notebookService.runParagraph(note, paragraphId, paragraph.getTitle(),
+                paragraph.getText(), params, new HashMap<>(), sessionId,
+                false, false, getServiceContext(), new RestServiceCallback<>());
+        return new JsonResponse<>(Status.OK).build();
+      });
   }
 
   /**
@@ -873,28 +888,32 @@ public class NotebookRestApi extends AbstractRestApi {
       throws IOException, IllegalArgumentException {
     LOGGER.info("Run paragraph synchronously {} {} {}", noteId, paragraphId, message);
 
-    Note note = notebook.getNote(noteId);
-    checkIfNoteIsNotNull(note, noteId);
-    Paragraph paragraph = note.getParagraph(paragraphId);
-    checkIfParagraphIsNotNull(paragraph, paragraphId);
-
-    Map<String, Object> params = new HashMap<>();
-    if (!StringUtils.isEmpty(message)) {
-      ParametersRequest request =
-              ParametersRequest.fromJson(message);
-      params = request.getParams();
-    }
-
-    if (notebookService.runParagraph(noteId, paragraphId, paragraph.getTitle(),
-            paragraph.getText(), params,
-            new HashMap<>(), sessionId, false, true, getServiceContext(), new RestServiceCallback<>())) {
-      note = notebookService.getNote(noteId, getServiceContext(), new RestServiceCallback<>());
-      Paragraph p = note.getParagraph(paragraphId);
-      InterpreterResult result = p.getReturn();
-      return new JsonResponse<>(Status.OK, result).build();
-    } else {
-      return new JsonResponse<>(Status.INTERNAL_SERVER_ERROR, "Fail to run paragraph").build();
-    }
+    return notebook.processNote(noteId,
+      note -> {
+        checkIfNoteIsNotNull(note, noteId);
+        Paragraph paragraph = note.getParagraph(paragraphId);
+        checkIfParagraphIsNotNull(paragraph, paragraphId);
+
+        Map<String, Object> params = new HashMap<>();
+        if (!StringUtils.isEmpty(message)) {
+          ParametersRequest request =
+                  ParametersRequest.fromJson(message);
+          params = request.getParams();
+        }
+
+        if (notebookService.runParagraph(note, paragraphId, paragraph.getTitle(),
+                paragraph.getText(), params,
+                new HashMap<>(), sessionId, false, true, getServiceContext(), new RestServiceCallback<>())) {
+          return notebookService.getNote(noteId, getServiceContext(), new RestServiceCallback<>(),
+            noteRun -> {
+              Paragraph p = noteRun.getParagraph(paragraphId);
+              InterpreterResult result = p.getReturn();
+              return new JsonResponse<>(Status.OK, result).build();
+            });
+        } else {
+          return new JsonResponse<>(Status.INTERNAL_SERVER_ERROR, "Fail to run paragraph").build();
+        }
+      });
   }
 
   /**
@@ -936,22 +955,25 @@ public class NotebookRestApi extends AbstractRestApi {
 
     CronRequest request = CronRequest.fromJson(message);
 
-    Note note = notebook.getNote(noteId);
-    checkIfNoteIsNotNull(note, noteId);
-    checkIfUserCanRun(noteId, "Insufficient privileges you cannot set a cron job for this note");
-    checkIfNoteSupportsCron(note);
-
-    if (!CronExpression.isValidExpression(request.getCronString())) {
-      return new JsonResponse<>(Status.BAD_REQUEST, "wrong cron expressions.").build();
-    }
-
-    Map<String, Object> config = note.getConfig();
-    config.put("cron", request.getCronString());
-    config.put("releaseresource", request.getReleaseResource());
-    note.setConfig(config);
-    schedulerService.refreshCron(note.getId());
-
-    return new JsonResponse<>(Status.OK).build();
+    // use write lock, because config is overwritten
+    return notebook.processNote(noteId,
+      note -> {
+        checkIfNoteIsNotNull(note, noteId);
+        checkIfUserCanRun(noteId, "Insufficient privileges you cannot set a cron job for this note");
+        checkIfNoteSupportsCron(note);
+
+        if (!CronExpression.isValidExpression(request.getCronString())) {
+          return new JsonResponse<>(Status.BAD_REQUEST, "wrong cron expressions.").build();
+        }
+
+        Map<String, Object> config = note.getConfig();
+        config.put("cron", request.getCronString());
+        config.put("releaseresource", request.getReleaseResource());
+        note.setConfig(config);
+        schedulerService.refreshCron(note.getId());
+
+        return new JsonResponse<>(Status.OK).build();
+      });
   }
 
   /**
@@ -970,19 +992,22 @@ public class NotebookRestApi extends AbstractRestApi {
 
     LOGGER.info("Remove cron job note {}", noteId);
 
-    Note note = notebook.getNote(noteId);
-    checkIfNoteIsNotNull(note, noteId);
-    checkIfUserIsOwner(noteId,
-            "Insufficient privileges you cannot remove this cron job from this note");
-    checkIfNoteSupportsCron(note);
-
-    Map<String, Object> config = note.getConfig();
-    config.remove("cron");
-    config.remove("releaseresource");
-    note.setConfig(config);
-    schedulerService.refreshCron(note.getId());
-
-    return new JsonResponse<>(Status.OK).build();
+    // use write lock because config is overwritten
+    return notebook.processNote(noteId,
+      note -> {
+        checkIfNoteIsNotNull(note, noteId);
+        checkIfUserIsOwner(noteId,
+                "Insufficient privileges you cannot remove this cron job from this note");
+        checkIfNoteSupportsCron(note);
+
+        Map<String, Object> config = note.getConfig();
+        config.remove("cron");
+        config.remove("releaseresource");
+        note.setConfig(config);
+        schedulerService.refreshCron(note.getId());
+
+        return new JsonResponse<>(Status.OK).build();
+      });
   }
 
   /**
@@ -1001,15 +1026,17 @@ public class NotebookRestApi extends AbstractRestApi {
 
     LOGGER.info("Get cron job note {}", noteId);
 
-    Note note = notebook.getNote(noteId);
-    checkIfNoteIsNotNull(note, noteId);
-    checkIfUserCanRead(noteId, "Insufficient privileges you cannot get cron information");
-    checkIfNoteSupportsCron(note);
-    Map<String, Object> response = new HashMap<>();
-    response.put("cron", note.getConfig().get("cron"));
-    response.put("releaseResource", note.getConfig().get("releaseresource"));
-
-    return new JsonResponse<>(Status.OK, response).build();
+    return notebook.processNote(noteId,
+      note -> {
+        checkIfNoteIsNotNull(note, noteId);
+        checkIfUserCanRead(noteId, "Insufficient privileges you cannot get cron information");
+        checkIfNoteSupportsCron(note);
+        Map<String, Object> response = new HashMap<>();
+        response.put("cron", note.getConfig().get("cron"));
+        response.put("releaseResource", note.getConfig().get("releaseresource"));
+
+        return new JsonResponse<>(Status.OK, response).build();
+      });
   }
 
   /**
@@ -1086,21 +1113,6 @@ public class NotebookRestApi extends AbstractRestApi {
   }
 
 
-  private void handleParagraphParams(String message, Note note, Paragraph paragraph)
-      throws IOException {
-    // handle params if presented
-    if (!StringUtils.isEmpty(message)) {
-      ParametersRequest request =
-          ParametersRequest.fromJson(message);
-      Map<String, Object> paramsForUpdating = request.getParams();
-      if (paramsForUpdating != null) {
-        paragraph.settings.getParams().putAll(paramsForUpdating);
-        AuthenticationInfo subject = new AuthenticationInfo(authenticationService.getPrincipal());
-        notebook.saveNote(note, subject);
-      }
-    }
-  }
-
   private void initParagraph(Paragraph p, NewParagraphRequest request, String user) {
     LOGGER.info("Init Paragraph for user {}", user);
     checkIfParagraphIsNotNull(p, "");
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/server/ZeppelinServer.java b/zeppelin-server/src/main/java/org/apache/zeppelin/server/ZeppelinServer.java
index be1aca7..428cdd9 100644
--- a/zeppelin-server/src/main/java/org/apache/zeppelin/server/ZeppelinServer.java
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/server/ZeppelinServer.java
@@ -276,6 +276,10 @@ public class ZeppelinServer extends ResourceConfig {
     // Lazy loading will cause paragraph recovery and cron job initialization is delayed.
     Notebook notebook = ServiceLocatorUtilities.getService(
             sharedServiceLocator, Notebook.class.getName());
+    ServiceLocatorUtilities.getService(
+      sharedServiceLocator, SearchService.class.getName());
+    ServiceLocatorUtilities.getService(
+      sharedServiceLocator, SchedulerService.class.getName());
     // Try to recover here, don't do it in constructor of Notebook, because it would cause deadlock.
     notebook.recoveryIfNecessary();
 
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/service/JobManagerService.java b/zeppelin-server/src/main/java/org/apache/zeppelin/service/JobManagerService.java
index 84d3594..47f9ed3 100644
--- a/zeppelin-server/src/main/java/org/apache/zeppelin/service/JobManagerService.java
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/service/JobManagerService.java
@@ -22,6 +22,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.zeppelin.conf.ZeppelinConfiguration;
 import org.apache.zeppelin.notebook.AuthorizationService;
 import org.apache.zeppelin.notebook.Note;
+import org.apache.zeppelin.notebook.NoteInfo;
 import org.apache.zeppelin.notebook.Notebook;
 import org.apache.zeppelin.notebook.Paragraph;
 import org.apache.zeppelin.scheduler.Job;
@@ -31,6 +32,7 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Date;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.stream.Collectors;
 
@@ -61,15 +63,18 @@ public class JobManagerService {
     if (!conf.isJobManagerEnabled()) {
       return new ArrayList<>();
     }
-    List<NoteJobInfo> notesJobInfo = new ArrayList<>();
-    Note jobNote = notebook.getNote(noteId);
-    if (jobNote == null) {
-      callback.onFailure(new IOException("Note " + noteId + " not found"), context);
-    } else {
-      notesJobInfo.add(new NoteJobInfo(jobNote));
-      callback.onSuccess(notesJobInfo, context);
-    }
-    return notesJobInfo;
+
+    return notebook.processNote(noteId,
+      jobNote -> {
+        List<NoteJobInfo> notesJobInfo = new ArrayList<>();
+        if (jobNote == null) {
+          callback.onFailure(new IOException("Note " + noteId + " not found"), context);
+        } else {
+          notesJobInfo.add(new NoteJobInfo(jobNote));
+          callback.onSuccess(notesJobInfo, context);
+        }
+        return notesJobInfo;
+      });
   }
 
   /**
@@ -83,12 +88,15 @@ public class JobManagerService {
       return new ArrayList<>();
     }
 
-    List<NoteJobInfo> notesJobInfo = notebook.getNoteStream()
-            .filter(note -> authorizationService.isOwner(context.getUserAndRoles(), note.getId()))
-            .map(NoteJobInfo::new)
-            .filter(noteJobInfo -> noteJobInfo.unixTimeLastRun > lastUpdateServerUnixTime)
-            .collect(Collectors.toList());
-
+    List<NoteJobInfo> notesJobInfo = new LinkedList<>();
+    for (NoteInfo noteInfo : notebook.getNotesInfo()) {
+      if (authorizationService.isOwner(context.getUserAndRoles(), noteInfo.getId())) {
+        NoteJobInfo noteJobInfo = notebook.processNote(noteInfo.getId(), NoteJobInfo::new);
+        if (noteJobInfo.unixTimeLastRun > lastUpdateServerUnixTime) {
+          notesJobInfo.add(noteJobInfo);
+        }
+      }
+    }
     callback.onSuccess(notesJobInfo, context);
     return notesJobInfo;
   }
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/service/NotebookService.java b/zeppelin-server/src/main/java/org/apache/zeppelin/service/NotebookService.java
index a956af9..f1e9904 100644
--- a/zeppelin-server/src/main/java/org/apache/zeppelin/service/NotebookService.java
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/service/NotebookService.java
@@ -30,6 +30,7 @@ import java.time.Instant;
 import java.time.ZoneId;
 import java.time.format.DateTimeFormatter;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -49,6 +50,7 @@ import org.apache.zeppelin.notebook.NoteInfo;
 import org.apache.zeppelin.notebook.NoteManager;
 import org.apache.zeppelin.notebook.Notebook;
 import org.apache.zeppelin.notebook.Paragraph;
+import org.apache.zeppelin.notebook.Notebook.NoteProcessor;
 import org.apache.zeppelin.notebook.AuthorizationService;
 import org.apache.zeppelin.notebook.exception.CorruptedNoteException;
 import org.apache.zeppelin.notebook.exception.NotePathAlreadyExistsException;
@@ -98,51 +100,59 @@ public class NotebookService {
     this.schedulerService = schedulerService;
   }
 
-  public Note getHomeNote(ServiceContext context,
+  public String getHomeNote(ServiceContext context,
                           ServiceCallback<Note> callback) throws IOException {
     String noteId = notebook.getConf().getString(ZEPPELIN_NOTEBOOK_HOMESCREEN);
-    Note note = null;
     if (noteId != null) {
-      note = notebook.getNote(noteId);
-      if (note != null) {
-        if (!checkPermission(noteId, Permission.READER, Message.OP.GET_HOME_NOTE, context,
-                callback)) {
+      callback.onSuccess(null, context);
+    } else {
+      notebook.processNote(noteId,
+        note -> {
+          if (note != null && !checkPermission(noteId, Permission.READER, Message.OP.GET_HOME_NOTE, context,
+            callback)) {
+            return null;
+          }
+          callback.onSuccess(note, context);
           return null;
-        }
-      }
+        });
     }
-    callback.onSuccess(note, context);
-    return note;
+    return noteId;
   }
 
-  public Note getNote(String noteId,
+  public  <T> T getNote(String noteId,
                       ServiceContext context,
-                      ServiceCallback<Note> callback) throws IOException {
-    return getNote(noteId, false, context, callback);
+                      ServiceCallback<Note> callback,
+                      NoteProcessor<T> noteProcessor) throws IOException {
+    return getNote(noteId, false, context, callback, noteProcessor);
   }
 
-  public Note getNote(String noteId,
+  public <T> T getNote(String noteId,
                       boolean reload,
                       ServiceContext context,
-                      ServiceCallback<Note> callback) throws IOException {
-    Note note = notebook.getNote(noteId, reload);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return null;
-    }
-
-    if (!checkPermission(noteId, Permission.READER, Message.OP.GET_NOTE, context,
-        callback)) {
-      return null;
-    }
-    if (note.isPersonalizedMode()) {
-      note = note.getUserNote(context.getAutheInfo().getUser());
-    }
-    callback.onSuccess(note, context);
-    return note;
+                      ServiceCallback<Note> callback,
+                      NoteProcessor<T> noteProcessor) throws IOException {
+    return notebook.processNote(noteId, reload,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return null;
+        }
+        if (!checkPermission(noteId, Permission.READER, Message.OP.GET_NOTE, context,
+          callback)) {
+          return null;
+        }
+        Note newNote = note;
+        if (note.isPersonalizedMode()) {
+          newNote = note.getUserNote(context.getAutheInfo().getUser());
+        }
+        callback.onSuccess(newNote, context);
+        if (noteProcessor == null) {
+          return null;
+        }
+        return noteProcessor.process(newNote);
+      });
   }
 
-
   /**
    *
    * @param notePath
@@ -150,10 +160,10 @@ public class NotebookService {
    * @param addingEmptyParagraph
    * @param context
    * @param callback
-   * @return
+   * @return noteId
    * @throws IOException
    */
-  public Note createNote(String notePath,
+  public String createNote(String notePath,
                          String defaultInterpreterGroup,
                          boolean addingEmptyParagraph,
                          ServiceContext context,
@@ -165,15 +175,20 @@ public class NotebookService {
     }
 
     try {
-      Note note = notebook.createNote(normalizeNotePath(notePath), defaultInterpreterGroup,
+      String noteId = notebook.createNote(normalizeNotePath(notePath), defaultInterpreterGroup,
           context.getAutheInfo(), false);
       // it's an empty note. so add one paragraph
-      if (addingEmptyParagraph) {
-        note.addNewParagraph(context.getAutheInfo());
-      }
-      notebook.saveNote(note, context.getAutheInfo());
-      callback.onSuccess(note, context);
-      return note;
+      notebook.processNote(noteId,
+        note -> {
+          if (addingEmptyParagraph) {
+            note.addNewParagraph(context.getAutheInfo());
+          }
+          notebook.saveNote(note, context.getAutheInfo());
+          callback.onSuccess(note, context);
+          return null;
+        });
+
+      return noteId;
     } catch (IOException e) {
       callback.onFailure(e, context);
       return null;
@@ -213,19 +228,14 @@ public class NotebookService {
     if (!checkPermission(noteId, Permission.OWNER, Message.OP.DEL_NOTE, context, callback)) {
       return;
     }
-
-    Note note = null;
-    try {
-      note = notebook.getNote(noteId);
-    } catch (CorruptedNoteException e) {
-      notebook.removeCorruptedNote(noteId, context.getAutheInfo());
-      callback.onSuccess("Delete note successfully", context);
-      return;
-    }
-
-    if (note != null) {
-      notebook.removeNote(note, context.getAutheInfo());
-      callback.onSuccess("Delete note successfully", context);
+    if (notebook.containsNoteById(noteId)) {
+      try {
+        notebook.removeNote(noteId, context.getAutheInfo());
+        callback.onSuccess("Delete note successfully", context);
+      } catch (CorruptedNoteException e) {
+        notebook.removeCorruptedNote(noteId, context.getAutheInfo());
+        callback.onSuccess("Delete note successfully", context);
+      }
     } else {
       callback.onFailure(new NoteNotFoundException(noteId), context);
     }
@@ -256,29 +266,32 @@ public class NotebookService {
     if (!checkPermission(noteId, Permission.OWNER, Message.OP.NOTE_RENAME, context, callback)) {
       return;
     }
-    Note note = notebook.getNote(noteId);
-    if (note != null) {
-      note.setCronSupported(notebook.getConf());
-      if (isRelative && !note.getParentPath().equals("/")) {
-        newNotePath = note.getParentPath() + "/" + newNotePath;
-      } else {
-        if (!newNotePath.startsWith("/")) {
-          newNotePath = "/" + newNotePath;
+    notebook.processNote(noteId,
+      readNote -> {
+        if (readNote == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
         }
-      }
-      try {
-        notebook.moveNote(noteId, newNotePath, context.getAutheInfo());
-        callback.onSuccess(note, context);
-      } catch (NotePathAlreadyExistsException e) {
-        callback.onFailure(e, context);
-      }
-    } else {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-    }
+        readNote.setCronSupported(notebook.getConf());
+        String newNotePathReal = newNotePath;
+        if (isRelative && !readNote.getParentPath().equals("/")) {
+          newNotePathReal = readNote.getParentPath() + "/" + newNotePath;
+        } else {
+          if (!newNotePath.startsWith("/")) {
+            newNotePathReal = "/" + newNotePath;
+          }
+        }
+        try {
+          notebook.moveNote(noteId, newNotePathReal, context.getAutheInfo());
+          callback.onSuccess(readNote, context);
+        } catch (NotePathAlreadyExistsException e) {
+          callback.onFailure(e, context);
+        }
+        return null;
+      });
 
   }
 
-  public Note cloneNote(String noteId,
+  public String cloneNote(String noteId,
                         String newNotePath,
                         ServiceContext context,
                         ServiceCallback<Note> callback) throws IOException {
@@ -287,27 +300,43 @@ public class NotebookService {
       newNotePath = "/Cloned Note_" + noteId;
     }
     try {
-      Note newNote = notebook.cloneNote(noteId, normalizeNotePath(newNotePath),
+      String newNoteId = notebook.cloneNote(noteId, normalizeNotePath(newNotePath),
           context.getAutheInfo());
-      callback.onSuccess(newNote, context);
-      return newNote;
+      return notebook.processNote(newNoteId,
+        newNote -> {
+          callback.onSuccess(newNote, context);
+          return newNote.getId();
+        });
     } catch (IOException e) {
       callback.onFailure(new IOException("Fail to clone note", e), context);
       return null;
     }
   }
 
-  public Note importNote(String notePath,
+  /**
+   *
+   * @param notePath
+   * @param noteJson
+   * @param context
+   * @param callback
+   * @return NoteId of the imported Note
+   * @throws IOException
+   */
+  public String importNote(String notePath,
                          String noteJson,
                          ServiceContext context,
                          ServiceCallback<Note> callback) throws IOException {
     try {
       // pass notePath when it is null
-      Note note = notebook.importNote(noteJson, notePath == null ?
+      String noteId = notebook.importNote(noteJson, notePath == null ?
               notePath : normalizeNotePath(notePath),
           context.getAutheInfo());
-      callback.onSuccess(note, context);
-      return note;
+      return notebook.processNote(noteId,
+        note -> {
+          callback.onSuccess(note, context);
+          return note.getId();
+        });
+
     } catch (IOException e) {
       callback.onFailure(new IOException("Fail to import note: " + e.getMessage(), e), context);
       return null;
@@ -331,7 +360,7 @@ public class NotebookService {
    * return false when paragraph execution fails due to zeppelin internal issue.
    * @throws IOException
    */
-  public boolean runParagraph(String noteId,
+  public boolean runParagraph(Note note,
                               String paragraphId,
                               String title,
                               String text,
@@ -343,16 +372,16 @@ public class NotebookService {
                               ServiceContext context,
                               ServiceCallback<Paragraph> callback) throws IOException {
 
-    LOGGER.info("Start to run paragraph: {} of note: {}", paragraphId, noteId);
-    if (!checkPermission(noteId, Permission.RUNNER, Message.OP.RUN_PARAGRAPH, context, callback)) {
-      return false;
-    }
 
-    Note note = notebook.getNote(noteId);
     if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
       return false;
     }
+    LOGGER.info("Start to run paragraph: {} of note: {}", paragraphId, note.getId());
+    if (!checkPermission(note.getId(), Permission.RUNNER, Message.OP.RUN_PARAGRAPH, context, callback)) {
+      return false;
+    }
+
+
     Paragraph p = note.getParagraph(paragraphId);
     if (p == null) {
       callback.onFailure(new ParagraphNotFoundException(paragraphId), context);
@@ -421,61 +450,64 @@ public class NotebookService {
       return false;
     }
 
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return false;
-    }
+    return notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return false;
+        }
 
-    if (paragraphs != null) {
-      // run note via the data passed from frontend
-      try {
-        note.setRunning(true);
-        for (Map<String, Object> raw : paragraphs) {
-          String paragraphId = (String) raw.get("id");
-          if (paragraphId == null) {
-            LOGGER.warn("No id found in paragraph json: {}", raw);
-            continue;
-          }
+        if (paragraphs != null) {
+          // run note via the data passed from frontend
           try {
-            String text = (String) raw.get("paragraph");
-            String title = (String) raw.get("title");
-            Map<String, Object> params = (Map<String, Object>) raw.get("params");
-            Map<String, Object> config = (Map<String, Object>) raw.get("config");
-
-            if (!runParagraph(noteId, paragraphId, title, text, params, config, null, false, true,
-                    context, callback)) {
-              // stop execution when one paragraph fails.
-              return false;
-            }
-            // also stop execution when user code in a paragraph fails
-            Paragraph p = note.getParagraph(paragraphId);
-            InterpreterResult result = p.getReturn();
-            if (result != null && result.code() == ERROR) {
-              return false;
-            }
-            if (p.getStatus() == ABORT || p.isAborted()) {
-              return false;
+            note.setRunning(true);
+            for (Map<String, Object> raw : paragraphs) {
+              String paragraphId = (String) raw.get("id");
+              if (paragraphId == null) {
+                LOGGER.warn("No id found in paragraph json: {}", raw);
+                continue;
+              }
+              try {
+                String text = (String) raw.get("paragraph");
+                String title = (String) raw.get("title");
+                Map<String, Object> params = (Map<String, Object>) raw.get("params");
+                Map<String, Object> config = (Map<String, Object>) raw.get("config");
+
+                if (!runParagraph(note, paragraphId, title, text, params, config, null, false, true,
+                        context, callback)) {
+                  // stop execution when one paragraph fails.
+                  return false;
+                }
+                // also stop execution when user code in a paragraph fails
+                Paragraph p = note.getParagraph(paragraphId);
+                InterpreterResult result = p.getReturn();
+                if (result != null && result.code() == ERROR) {
+                  return false;
+                }
+                if (p.getStatus() == ABORT || p.isAborted()) {
+                  return false;
+                }
+              } catch (Exception e) {
+                throw new IOException("Fail to run paragraph json: " + raw, e);
+              }
             }
+          } finally {
+            note.setRunning(false);
+          }
+        } else {
+          try {
+            // run note directly when parameter `paragraphs` is null.
+            note.runAll(context.getAutheInfo(), true, false, new HashMap<>());
+            return true;
           } catch (Exception e) {
-            throw new IOException("Fail to run paragraph json: " + raw, e);
+            LOGGER.warn("Fail to run note: {}", note.getName(), e);
+            return false;
           }
         }
-      } finally {
-        note.setRunning(false);
-      }
-    } else {
-      try {
-        // run note directly when parameter `paragraphs` is null.
-        note.runAll(context.getAutheInfo(), true, false, new HashMap<>());
+
         return true;
-      } catch (Exception e) {
-        LOGGER.warn("Fail to run note: {}", note.getName(), e);
-        return false;
-      }
-    }
+      });
 
-    return true;
   }
 
   public void cancelParagraph(String noteId,
@@ -486,16 +518,21 @@ public class NotebookService {
         callback)) {
       return;
     }
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      throw new NoteNotFoundException(noteId);
-    }
-    Paragraph p = note.getParagraph(paragraphId);
-    if (p == null) {
-      throw new ParagraphNotFoundException(paragraphId);
-    }
-    p.abort();
-    callback.onSuccess(p, context);
+
+    notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          throw new NoteNotFoundException(noteId);
+        }
+        Paragraph p = note.getParagraph(paragraphId);
+        if (p == null) {
+          throw new ParagraphNotFoundException(paragraphId);
+        }
+        p.abort();
+        callback.onSuccess(p, context);
+        return null;
+      });
+
   }
 
   public void moveParagraph(String noteId,
@@ -507,21 +544,24 @@ public class NotebookService {
         callback)) {
       return;
     }
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      throw new NoteNotFoundException(noteId);
-    }
-    if (note.getParagraph(paragraphId) == null) {
-      throw new ParagraphNotFoundException(paragraphId);
-    }
-    if (newIndex >= note.getParagraphCount()) {
-      callback.onFailure(new BadRequestException("newIndex " + newIndex + " is out of bounds"),
-          context);
-      return;
-    }
-    note.moveParagraph(paragraphId, newIndex);
-    notebook.saveNote(note, context.getAutheInfo());
-    callback.onSuccess(note.getParagraph(newIndex), context);
+    notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          throw new NoteNotFoundException(noteId);
+        }
+        if (note.getParagraph(paragraphId) == null) {
+          throw new ParagraphNotFoundException(paragraphId);
+        }
+        if (newIndex >= note.getParagraphCount()) {
+          callback.onFailure(new BadRequestException("newIndex " + newIndex + " is out of bounds"),
+              context);
+          return null;
+        }
+        note.moveParagraph(paragraphId, newIndex);
+        notebook.saveNote(note, context.getAutheInfo());
+        callback.onSuccess(note.getParagraph(newIndex), context);
+        return null;
+      });
   }
 
   public void removeParagraph(String noteId,
@@ -532,16 +572,21 @@ public class NotebookService {
         callback)) {
       return;
     }
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      throw new NoteNotFoundException(noteId);
-    }
-    if (note.getParagraph(paragraphId) == null) {
-      throw new ParagraphNotFoundException(paragraphId);
-    }
-    Paragraph p = note.removeParagraph(context.getAutheInfo().getUser(), paragraphId);
-    notebook.saveNote(note, context.getAutheInfo());
-    callback.onSuccess(p, context);
+
+    notebook.processNote(noteId,
+        note -> {
+          if (note == null) {
+            throw new NoteNotFoundException(noteId);
+          }
+          if (note.getParagraph(paragraphId) == null) {
+            throw new ParagraphNotFoundException(paragraphId);
+          }
+          Paragraph p = note.removeParagraph(context.getAutheInfo().getUser(), paragraphId);
+          notebook.saveNote(note, context.getAutheInfo());
+          callback.onSuccess(p, context);
+          return null;
+        });
+
   }
 
   public Paragraph insertParagraph(String noteId,
@@ -553,15 +598,18 @@ public class NotebookService {
         callback)) {
       return null;
     }
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      throw new NoteNotFoundException(noteId);
-    }
-    Paragraph newPara = note.insertNewParagraph(index, context.getAutheInfo());
-    newPara.mergeConfig(config);
-    notebook.saveNote(note, context.getAutheInfo());
-    callback.onSuccess(newPara, context);
-    return newPara;
+
+    return notebook.processNote(noteId,
+        note -> {
+          if (note == null) {
+            throw new NoteNotFoundException(noteId);
+          }
+          Paragraph newPara = note.insertNewParagraph(index, context.getAutheInfo());
+          newPara.mergeConfig(config);
+          notebook.saveNote(note, context.getAutheInfo());
+          callback.onSuccess(newPara, context);
+          return newPara;
+        });
   }
 
   public void restoreNote(String noteId,
@@ -571,24 +619,28 @@ public class NotebookService {
         callback)) {
       return;
     }
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return;
-    }
+    notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return null;
+        }
+
+        if (!note.getPath().startsWith("/" + NoteManager.TRASH_FOLDER)) {
+          callback.onFailure(new IOException("Can not restore this note " + note.getPath() +
+              " as it is not in trash folder"), context);
+          return null;
+        }
+        try {
+          String destNotePath = note.getPath().replace("/" + NoteManager.TRASH_FOLDER, "");
+          notebook.moveNote(noteId, destNotePath, context.getAutheInfo());
+          callback.onSuccess(note, context);
+        } catch (IOException e) {
+          callback.onFailure(new IOException("Fail to restore note: " + noteId, e), context);
+        }
+        return null;
+      });
 
-    if (!note.getPath().startsWith("/" + NoteManager.TRASH_FOLDER)) {
-      callback.onFailure(new IOException("Can not restore this note " + note.getPath() +
-          " as it is not in trash folder"), context);
-      return;
-    }
-    try {
-      String destNotePath = note.getPath().replace("/" + NoteManager.TRASH_FOLDER, "");
-      notebook.moveNote(noteId, destNotePath, context.getAutheInfo());
-      callback.onSuccess(note, context);
-    } catch (IOException e) {
-      callback.onFailure(new IOException("Fail to restore note: " + noteId, e), context);
-    }
 
   }
 
@@ -613,7 +665,7 @@ public class NotebookService {
 
 
   public void restoreAll(ServiceContext context,
-                         ServiceCallback<?> callback) throws IOException {
+                         ServiceCallback<Void> callback) throws IOException {
 
     try {
       notebook.restoreAll(context.getAutheInfo());
@@ -636,33 +688,45 @@ public class NotebookService {
         callback)) {
       return;
     }
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return;
-    }
-    Paragraph p = note.getParagraph(paragraphId);
-    if (p == null) {
-      callback.onFailure(new ParagraphNotFoundException(paragraphId), context);
-      return;
-    }
 
-    p.settings.setParams(params);
-    p.mergeConfig(config);
-    p.setTitle(title);
-    p.setText(text);
-    if (note.isPersonalizedMode()) {
-      p = p.getUserParagraph(context.getAutheInfo().getUser());
-      p.settings.setParams(params);
-      p.mergeConfig(config);
-      p.setTitle(title);
-      p.setText(text);
-    }
-    notebook.saveNote(note, context.getAutheInfo());
-    callback.onSuccess(p, context);
+    notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return null;
+        }
+        Paragraph p = note.getParagraph(paragraphId);
+        if (p == null) {
+          callback.onFailure(new ParagraphNotFoundException(paragraphId), context);
+          return null;
+        }
+        p.settings.setParams(params);
+        p.mergeConfig(config);
+        p.setTitle(title);
+        p.setText(text);
+        if (note.isPersonalizedMode()) {
+          p = p.getUserParagraph(context.getAutheInfo().getUser());
+          p.settings.setParams(params);
+          p.mergeConfig(config);
+          p.setTitle(title);
+          p.setText(text);
+        }
+        notebook.saveNote(note, context.getAutheInfo());
+        callback.onSuccess(p, context);
+        return null;
+      });
   }
 
-  public Paragraph getNextSessionParagraph(String noteId,
+  /**
+   *
+   * @param noteId
+   * @param maxParagraph
+   * @param context
+   * @param callback
+   * @return paragraphId
+   * @throws IOException
+   */
+  public String getNextSessionParagraphId(String noteId,
                                         int maxParagraph,
                                         ServiceContext context,
                                         ServiceCallback<Paragraph> callback) throws IOException {
@@ -670,27 +734,29 @@ public class NotebookService {
             callback)) {
       throw new IOException("No privilege to access this note");
     }
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      throw new IOException("No such note");
-    }
-    synchronized (this) {
-      if (note.getParagraphCount() >= maxParagraph) {
-        boolean removed = false;
-        for (int i = 1; i < note.getParagraphCount(); ++i) {
-          if (note.getParagraph(i).getStatus().isCompleted()) {
-            note.removeParagraph(context.getAutheInfo().getUser(), note.getParagraph(i).getId());
-            removed = true;
-            break;
-          }
+    return notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          throw new IOException("No such note");
         }
-        if (!removed) {
-          throw new IOException("All the paragraphs are not completed, unable to find available paragraph");
+        synchronized (this) {
+          if (note.getParagraphCount() >= maxParagraph) {
+            boolean removed = false;
+            for (int i = 1; i < note.getParagraphCount(); ++i) {
+              if (note.getParagraph(i).getStatus().isCompleted()) {
+                note.removeParagraph(context.getAutheInfo().getUser(), note.getParagraph(i).getId());
+                removed = true;
+                break;
+              }
+            }
+            if (!removed) {
+              throw new IOException("All the paragraphs are not completed, unable to find available paragraph");
+            }
+          }
+          return note.addNewParagraph(context.getAutheInfo()).getId();
         }
-      }
-      return note.addNewParagraph(context.getAutheInfo());
-    }
+      });
   }
 
   public void clearParagraphOutput(String noteId,
@@ -701,25 +767,28 @@ public class NotebookService {
         callback)) {
       return;
     }
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return;
-    }
-    Paragraph p = note.getParagraph(paragraphId);
-    if (p == null) {
-      callback.onFailure(new ParagraphNotFoundException(paragraphId), context);
-      return;
-    }
-    Paragraph returnedParagraph;
-    if (note.isPersonalizedMode()) {
-      returnedParagraph = note.clearPersonalizedParagraphOutput(paragraphId,
-          context.getAutheInfo().getUser());
-    } else {
-      note.clearParagraphOutput(paragraphId);
-      returnedParagraph = note.getParagraph(paragraphId);
-    }
-    callback.onSuccess(returnedParagraph, context);
+    notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return null;
+        }
+        Paragraph p = note.getParagraph(paragraphId);
+        if (p == null) {
+          callback.onFailure(new ParagraphNotFoundException(paragraphId), context);
+          return null;
+        }
+        Paragraph returnedParagraph;
+        if (note.isPersonalizedMode()) {
+          returnedParagraph = note.clearPersonalizedParagraphOutput(paragraphId,
+              context.getAutheInfo().getUser());
+        } else {
+          note.clearParagraphOutput(paragraphId);
+          returnedParagraph = note.getParagraph(paragraphId);
+        }
+        callback.onSuccess(returnedParagraph, context);
+        return null;
+      });
   }
 
   public void clearAllParagraphOutput(String noteId,
@@ -729,15 +798,19 @@ public class NotebookService {
         callback)) {
       return;
     }
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return;
-    }
 
-    note.clearAllParagraphOutput();
-    notebook.saveNote(note, context.getAutheInfo());
-    callback.onSuccess(note, context);
+
+    notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return null;
+        }
+        note.clearAllParagraphOutput();
+        notebook.saveNote(note, context.getAutheInfo());
+        callback.onSuccess(note, context);
+        return null;
+      });
   }
 
 
@@ -750,27 +823,30 @@ public class NotebookService {
         callback)) {
       return;
     }
+    // use write lock because config and name are overwritten
+    notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return null;
+        }
 
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return;
-    }
-
-    if (!(Boolean) note.getConfig().get("isZeppelinNotebookCronEnable")) {
-      if (config.get("cron") != null) {
-        config.remove("cron");
-      }
-    }
-    boolean cronUpdated = isCronUpdated(config, note.getConfig());
-    note.setName(name);
-    note.setConfig(config);
-    if (cronUpdated) {
-      schedulerService.refreshCron(note.getId());
-    }
+        if (!(Boolean) note.getConfig().get("isZeppelinNotebookCronEnable")) {
+          if (config.get("cron") != null) {
+            config.remove("cron");
+          }
+        }
+        boolean cronUpdated = isCronUpdated(config, note.getConfig());
+        note.setName(name);
+        note.setConfig(config);
+        if (cronUpdated) {
+          schedulerService.refreshCron(note.getId());
+        }
 
-    notebook.updateNote(note, context.getAutheInfo());
-    callback.onSuccess(note, context);
+        notebook.updateNote(note, context.getAutheInfo());
+        callback.onSuccess(note, context);
+        return null;
+      });
   }
 
 
@@ -794,37 +870,43 @@ public class NotebookService {
         callback)) {
       return;
     }
+    // use write lock because noteParams are overwritten
+    notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return null;
+        }
 
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return;
-    }
-
-    note.setNoteParams(noteParams);
-    notebook.saveNote(note, context.getAutheInfo());
-    callback.onSuccess(note, context);
+        note.setNoteParams(noteParams);
+        notebook.saveNote(note, context.getAutheInfo());
+        callback.onSuccess(note, context);
+        return null;
+      });
   }
 
   public void removeNoteForms(String noteId,
                               String formName,
                               ServiceContext context,
                               ServiceCallback<Note> callback) throws IOException {
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return;
-    }
+    notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return null;
+        }
 
-    if (!checkPermission(noteId, Permission.WRITER, Message.OP.REMOVE_NOTE_FORMS, context,
-        callback)) {
-      return;
-    }
+        if (!checkPermission(noteId, Permission.WRITER, Message.OP.REMOVE_NOTE_FORMS, context,
+            callback)) {
+          return null;
+        }
 
-    note.getNoteForms().remove(formName);
-    note.getNoteParams().remove(formName);
-    notebook.saveNote(note, context.getAutheInfo());
-    callback.onSuccess(note, context);
+        note.getNoteForms().remove(formName);
+        note.getNoteParams().remove(formName);
+        notebook.saveNote(note, context.getAutheInfo());
+        callback.onSuccess(note, context);
+        return null;
+      });
   }
 
   public NotebookRepoWithVersionControl.Revision checkpointNote(
@@ -833,19 +915,19 @@ public class NotebookService {
       ServiceContext context,
       ServiceCallback<NotebookRepoWithVersionControl.Revision> callback) throws IOException {
 
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return null;
-    }
-
-    if (!checkPermission(noteId, Permission.WRITER, Message.OP.REMOVE_NOTE_FORMS, context,
-        callback)) {
-      return null;
-    }
+    NotebookRepoWithVersionControl.Revision revision = notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return null;
+        }
+        if (!checkPermission(noteId, Permission.WRITER, Message.OP.REMOVE_NOTE_FORMS, context,
+          callback)) {
+          return null;
+        }
+        return notebook.checkpointNote(noteId, note.getPath(), commitMessage, context.getAutheInfo());
+      });
 
-    NotebookRepoWithVersionControl.Revision revision =
-        notebook.checkpointNote(noteId, note.getPath(), commitMessage, context.getAutheInfo());
     callback.onSuccess(revision, context);
     return revision;
   }
@@ -855,49 +937,54 @@ public class NotebookService {
       ServiceContext context,
       ServiceCallback<List<NotebookRepoWithVersionControl.Revision>> callback) throws IOException {
 
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return null;
-    }
 
-    // TODO(zjffdu) Disable checking permission for now, otherwise zeppelin will send 2 AUTH_INFO
-    // message to frontend when frontend try to get note without proper privilege.
-    //    if (!checkPermission(noteId, Permission.READER, Message.OP.LIST_REVISION_HISTORY, context,
-    //        callback)) {
-    //      return null;
-    //    }
-    List<NotebookRepoWithVersionControl.Revision> revisions =
-        notebook.listRevisionHistory(noteId, note.getPath(), context.getAutheInfo());
+    List<NotebookRepoWithVersionControl.Revision> revisions = notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return null;
+        }
+        // TODO(zjffdu) Disable checking permission for now, otherwise zeppelin will send 2 AUTH_INFO
+        // message to frontend when frontend try to get note without proper privilege.
+        //    if (!checkPermission(noteId, Permission.READER, Message.OP.LIST_REVISION_HISTORY, context,
+        //        callback)) {
+        //      return null;
+        //    }
+        return notebook.listRevisionHistory(noteId, note.getPath(), context.getAutheInfo());
+      });
+
     callback.onSuccess(revisions, context);
     return revisions;
   }
 
 
-  public Note setNoteRevision(String noteId,
+  public void setNoteRevision(String noteId,
                               String revisionId,
                               ServiceContext context,
                               ServiceCallback<Note> callback) throws IOException {
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return null;
-    }
 
-    if (!checkPermission(noteId, Permission.WRITER, Message.OP.SET_NOTE_REVISION, context,
-        callback)) {
-      return null;
-    }
+    notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return null;
+        }
+
+        if (!checkPermission(noteId, Permission.WRITER, Message.OP.SET_NOTE_REVISION, context,
+            callback)) {
+          return null;
+        }
+
+        try {
+          Note resultNote = notebook.setNoteRevision(noteId, note.getPath(), revisionId,
+              context.getAutheInfo());
+          callback.onSuccess(resultNote, context);
+        } catch (Exception e) {
+          callback.onFailure(new IOException("Fail to set given note revision", e), context);
+        }
+        return null;
+      });
 
-    try {
-      Note resultNote = notebook.setNoteRevision(noteId, note.getPath(), revisionId,
-          context.getAutheInfo());
-      callback.onSuccess(resultNote, context);
-      return resultNote;
-    } catch (Exception e) {
-      callback.onFailure(new IOException("Fail to set given note revision", e), context);
-      return null;
-    }
   }
 
   public void getNotebyRevision(String noteId,
@@ -905,19 +992,22 @@ public class NotebookService {
                                 ServiceContext context,
                                 ServiceCallback<Note> callback) throws IOException {
 
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return;
-    }
+    notebook.processNote(noteId ,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return null;
+        }
 
-    if (!checkPermission(noteId, Permission.READER, Message.OP.NOTE_REVISION, context,
-        callback)) {
-      return;
-    }
-    Note revisionNote = notebook.getNoteByRevision(noteId, note.getPath(), revisionId,
-        context.getAutheInfo());
-    callback.onSuccess(revisionNote, context);
+        if (!checkPermission(noteId, Permission.READER, Message.OP.NOTE_REVISION, context,
+            callback)) {
+          return null;
+        }
+        Note revisionNote = notebook.getNoteByRevision(noteId, note.getPath(), revisionId,
+            context.getAutheInfo());
+        callback.onSuccess(revisionNote, context);
+        return null;
+      });
   }
 
   public void getNoteByRevisionForCompare(String noteId,
@@ -925,24 +1015,27 @@ public class NotebookService {
                                           ServiceContext context,
                                           ServiceCallback<Note> callback) throws IOException {
 
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return;
-    }
+    notebook.processNote(noteId ,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return null;
+        }
 
-    if (!checkPermission(noteId, Permission.READER, Message.OP.NOTE_REVISION_FOR_COMPARE, context,
-        callback)) {
-      return;
-    }
-    Note revisionNote;
-    if (revisionId.equals("Head")) {
-      revisionNote = note;
-    } else {
-      revisionNote = notebook.getNoteByRevision(noteId, note.getPath(), revisionId,
-          context.getAutheInfo());
-    }
-    callback.onSuccess(revisionNote, context);
+        if (!checkPermission(noteId, Permission.READER, Message.OP.NOTE_REVISION_FOR_COMPARE, context,
+            callback)) {
+          return null;
+        }
+        Note revisionNote;
+        if (revisionId.equals("Head")) {
+          revisionNote = note;
+        } else {
+          revisionNote = notebook.getNoteByRevision(noteId, note.getPath(), revisionId,
+              context.getAutheInfo());
+        }
+        callback.onSuccess(revisionNote, context);
+        return null;
+      });
   }
 
   public List<InterpreterCompletion> completion(
@@ -953,44 +1046,49 @@ public class NotebookService {
       ServiceContext context,
       ServiceCallback<List<InterpreterCompletion>> callback) throws IOException {
 
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return null;
-    }
+    return notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return null;
+        }
 
-    if (!checkPermission(noteId, Permission.WRITER, Message.OP.COMPLETION, context,
-        callback)) {
-      return null;
-    }
+        if (!checkPermission(noteId, Permission.WRITER, Message.OP.COMPLETION, context,
+            callback)) {
+          return null;
+        }
+
+        try {
+          List<InterpreterCompletion> completions = note.completion(paragraphId, buffer, cursor,
+                  context.getAutheInfo());
+          callback.onSuccess(completions, context);
+          return completions;
+        } catch (RuntimeException e) {
+          callback.onFailure(new IOException("Fail to get completion", e), context);
+          return null;
+        }
+    });
 
-    try {
-      List<InterpreterCompletion> completions = note.completion(paragraphId, buffer, cursor,
-              context.getAutheInfo());
-      callback.onSuccess(completions, context);
-      return completions;
-    } catch (RuntimeException e) {
-      callback.onFailure(new IOException("Fail to get completion", e), context);
-      return null;
-    }
   }
 
   public void getEditorSetting(String noteId,
                                String paragraphText,
                                ServiceContext context,
                                ServiceCallback<Map<String, Object>> callback) throws IOException {
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return;
-    }
-    try {
-      Map<String, Object> settings = notebook.getInterpreterSettingManager().
-          getEditorSetting(paragraphText, noteId);
-      callback.onSuccess(settings, context);
-    } catch (Exception e) {
-      callback.onFailure(new IOException("Fail to getEditorSetting", e), context);
-    }
+    notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+        }
+        try {
+          Map<String, Object> settings = notebook.getInterpreterSettingManager().
+              getEditorSetting(paragraphText, noteId);
+          callback.onSuccess(settings, context);
+        } catch (Exception e) {
+          callback.onFailure(new IOException("Fail to getEditorSetting", e), context);
+        }
+        return null;
+      });
   }
 
   public void updatePersonalizedMode(String noteId,
@@ -998,20 +1096,23 @@ public class NotebookService {
                                      ServiceContext context,
                                      ServiceCallback<Note> callback) throws IOException {
 
-    Note note = notebook.getNote(noteId);
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return;
-    }
+    notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          callback.onFailure(new NoteNotFoundException(noteId), context);
+          return null;
+        }
 
-    if (!checkPermission(noteId, Permission.WRITER, Message.OP.UPDATE_PERSONALIZED_MODE, context,
-        callback)) {
-      return;
-    }
+        if (!checkPermission(noteId, Permission.WRITER, Message.OP.UPDATE_PERSONALIZED_MODE, context,
+            callback)) {
+          return null;
+        }
 
-    note.setPersonalizedMode(isPersonalized);
-    notebook.saveNote(note, context.getAutheInfo());
-    callback.onSuccess(note, context);
+        note.setPersonalizedMode(isPersonalized);
+        notebook.saveNote(note, context.getAutheInfo());
+        callback.onSuccess(note, context);
+        return null;
+      });
   }
 
   public void moveNoteToTrash(String noteId,
@@ -1026,22 +1127,23 @@ public class NotebookService {
       destNotePath = destNotePath + " " + TRASH_CONFLICT_TIMESTAMP_FORMATTER.format(Instant.now());
     }
 
-    Note note = null;
+    final String finalDestNotePath = destNotePath;
+
     try {
-       note = notebook.getNote(noteId);
+      notebook.processNote(noteId,
+        note -> {
+          if (note == null) {
+            callback.onFailure(new NoteNotFoundException(noteId), context);
+            return null;
+          }
+          notebook.moveNote(noteId, finalDestNotePath, context.getAutheInfo());
+          callback.onSuccess(note, context);
+          return null;
+        });
     } catch (CorruptedNoteException e) {
         LOGGER.info("Move corrupted note to trash");
         notebook.moveNote(noteId, destNotePath, context.getAutheInfo());
-        return;
-    }
-
-    if (note == null) {
-      callback.onFailure(new NoteNotFoundException(noteId), context);
-      return;
     }
-
-    notebook.moveNote(noteId, destNotePath, context.getAutheInfo());
-    callback.onSuccess(note, context);
   }
 
   public void moveFolderToTrash(String folderPath,
@@ -1129,34 +1231,36 @@ public class NotebookService {
       Job.Status status = Job.Status.valueOf((String) message.get("status"));
       Map<String, Object> params = (Map<String, Object>) message.get("params");
       Map<String, Object> config = (Map<String, Object>) message.get("config");
+      notebook.processNote(noteId,
+        note -> {
+          Paragraph p = setParagraphUsingMessage(note, message, paragraphId,
+            text, title, params, config);
+          p.setResult((InterpreterResult) message.get("results"));
+          p.setErrorMessage((String) message.get("errorMessage"));
+          p.setStatusWithoutNotification(status);
+
+          // Spell uses ISO 8601 formatted string generated from moment
+          String dateStarted = (String) message.get("dateStarted");
+          String dateFinished = (String) message.get("dateFinished");
+          SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSX");
 
-      Note note = notebook.getNote(noteId);
-      Paragraph p = setParagraphUsingMessage(note, message, paragraphId,
-          text, title, params, config);
-      p.setResult((InterpreterResult) message.get("results"));
-      p.setErrorMessage((String) message.get("errorMessage"));
-      p.setStatusWithoutNotification(status);
-
-      // Spell uses ISO 8601 formatted string generated from moment
-      String dateStarted = (String) message.get("dateStarted");
-      String dateFinished = (String) message.get("dateFinished");
-      SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSX");
-
-      try {
-        p.setDateStarted(df.parse(dateStarted));
-      } catch (ParseException e) {
-        LOGGER.error("Failed parse dateStarted", e);
-      }
+          try {
+            p.setDateStarted(df.parse(dateStarted));
+          } catch (ParseException e) {
+            LOGGER.error("Failed parse dateStarted", e);
+          }
 
-      try {
-        p.setDateFinished(df.parse(dateFinished));
-      } catch (ParseException e) {
-        LOGGER.error("Failed parse dateFinished", e);
-      }
+          try {
+            p.setDateFinished(df.parse(dateFinished));
+          } catch (ParseException e) {
+            LOGGER.error("Failed parse dateFinished", e);
+          }
 
-      addNewParagraphIfLastParagraphIsExecuted(note, p);
-      notebook.saveNote(note, context.getAutheInfo());
-      callback.onSuccess(p, context);
+          addNewParagraphIfLastParagraphIsExecuted(note, p);
+          notebook.saveNote(note, context.getAutheInfo());
+          callback.onSuccess(p, context);
+          return null;
+        });
     } catch (IOException e) {
       callback.onFailure(new IOException("Fail to run spell", e), context);
     }
@@ -1207,45 +1311,48 @@ public class NotebookService {
     AngularObject ao = null;
     boolean global = false;
     // propagate change to (Remote) AngularObjectRegistry
-    Note note = notebook.getNote(noteId);
-    if (note != null) {
-      List<InterpreterSetting> settings =
-              note.getBindedInterpreterSettings(new ArrayList(context.getUserAndRoles()));
-      for (InterpreterSetting setting : settings) {
-        if (setting.getInterpreterGroup(user, note.getId()) == null) {
-          continue;
+    List<InterpreterSetting> settings = notebook.processNote(noteId,
+      note -> {
+        if (note == null) {
+          return Collections.emptyList();
+        } else {
+          return note.getBindedInterpreterSettings(new ArrayList<>(context.getUserAndRoles()));
         }
-        if (interpreterGroupId.equals(setting.getInterpreterGroup(user, note.getId())
-            .getId())) {
-          AngularObjectRegistry angularObjectRegistry =
-              setting.getInterpreterGroup(user, note.getId()).getAngularObjectRegistry();
-
-          // first trying to get local registry
-          ao = angularObjectRegistry.get(varName, noteId, paragraphId);
+      });
+    for (InterpreterSetting setting : settings) {
+      if (setting.getInterpreterGroup(user, noteId) == null) {
+        continue;
+      }
+      if (interpreterGroupId.equals(setting.getInterpreterGroup(user, noteId)
+          .getId())) {
+        AngularObjectRegistry angularObjectRegistry =
+            setting.getInterpreterGroup(user, noteId).getAngularObjectRegistry();
+
+        // first trying to get local registry
+        ao = angularObjectRegistry.get(varName, noteId, paragraphId);
+        if (ao == null) {
+          // then try notebook scope registry
+          ao = angularObjectRegistry.get(varName, noteId, null);
           if (ao == null) {
-            // then try notebook scope registry
-            ao = angularObjectRegistry.get(varName, noteId, null);
+            // then try global scope registry
+            ao = angularObjectRegistry.get(varName, null, null);
             if (ao == null) {
-              // then try global scope registry
-              ao = angularObjectRegistry.get(varName, null, null);
-              if (ao == null) {
-                LOGGER.warn("Object {} is not binded", varName);
-              } else {
-                // path from client -> server
-                ao.set(varValue, false);
-                global = true;
-              }
+              LOGGER.warn("Object {} is not binded", varName);
             } else {
               // path from client -> server
               ao.set(varValue, false);
-              global = false;
+              global = true;
             }
           } else {
+            // path from client -> server
             ao.set(varValue, false);
             global = false;
           }
-          break;
+        } else {
+          ao.set(varValue, false);
+          global = false;
         }
+        break;
       }
     }
 
@@ -1263,30 +1370,33 @@ public class NotebookService {
       }
 
 
-      Note note = notebook.getNote(noteId);
-      if (note == null) {
-        return;
-      }
-      Paragraph p = note.getParagraph(paragraphId);
-      if (p == null) {
-        return;
-      }
+      notebook.processNote(noteId,
+        note -> {
+          if (note == null) {
+            return null;
+          }
+          Paragraph p = note.getParagraph(paragraphId);
+          if (p == null) {
+            return null;
+          }
 
-      DiffMatchPatch dmp = new DiffMatchPatch();
-      LinkedList<DiffMatchPatch.Patch> patches = null;
-      try {
-        patches = (LinkedList<DiffMatchPatch.Patch>) dmp.patchFromText(patchText);
-      } catch (ClassCastException e) {
-        LOGGER.error("Failed to parse patches", e);
-      }
-      if (patches == null) {
-        return;
-      }
+          DiffMatchPatch dmp = new DiffMatchPatch();
+          LinkedList<DiffMatchPatch.Patch> patches = null;
+          try {
+            patches = (LinkedList<DiffMatchPatch.Patch>) dmp.patchFromText(patchText);
+          } catch (ClassCastException e) {
+            LOGGER.error("Failed to parse patches", e);
+          }
+          if (patches == null) {
+            return null;
+          }
 
-      String paragraphText = p.getText() == null ? "" : p.getText();
-      paragraphText = (String) dmp.patchApply(patches, paragraphText)[0];
-      p.setText(paragraphText);
-      callback.onSuccess(patchText, context);
+          String paragraphText = p.getText() == null ? "" : p.getText();
+          paragraphText = (String) dmp.patchApply(patches, paragraphText)[0];
+          p.setText(paragraphText);
+          callback.onSuccess(patchText, context);
+          return null;
+      });
     } catch (IOException e) {
       callback.onFailure(new IOException("Fail to patch", e), context);
     }
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/service/SessionManagerService.java b/zeppelin-server/src/main/java/org/apache/zeppelin/service/SessionManagerService.java
index 71bc37c..ed5dbc3 100644
--- a/zeppelin-server/src/main/java/org/apache/zeppelin/service/SessionManagerService.java
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/service/SessionManagerService.java
@@ -111,8 +111,8 @@ public class SessionManagerService {
       throw new Exception("Unable to generate session id");
     }
 
-    Note sessionNote = notebook.createNote(buildNotePath(interpreter, sessionId), AuthenticationInfo.ANONYMOUS);
-    SessionInfo sessionInfo = new SessionInfo(sessionId, sessionNote.getId(), interpreter);
+    String sessionNoteId = notebook.createNote(buildNotePath(interpreter, sessionId), AuthenticationInfo.ANONYMOUS);
+    SessionInfo sessionInfo = new SessionInfo(sessionId, sessionNoteId, interpreter);
     sessions.put(sessionId, sessionInfo);
     return sessionInfo;
   }
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 0cb1f15..238545c 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
@@ -308,8 +308,9 @@ public class NotebookServer implements AngularObjectRegistryListener,
       }
 
       if (Message.isDisabledForRunningNotes(receivedMessage.op)) {
-        Note note = getNotebook().getNote((String) receivedMessage.get("noteId"));
-        if (note != null && note.isRunning()) {
+        boolean noteRunning = getNotebook().processNote((String) receivedMessage.get("noteId"),
+          note -> note != null && note.isRunning());
+        if (noteRunning) {
           throw new Exception("Note is now running sequentially. Can not be performed: " + receivedMessage.op);
         }
       }
@@ -578,40 +579,47 @@ public class NotebookServer implements AngularObjectRegistryListener,
                                      Message fromMessage) throws IOException {
     List<InterpreterSettingsList> settingList = new ArrayList<>();
     String noteId = (String) fromMessage.data.get("noteId");
-    Note note = getNotebook().getNote(noteId);
-    if (note != null) {
-      List<InterpreterSetting> bindedSettings =
-          note.getBindedInterpreterSettings(new ArrayList<>(context.getUserAndRoles()));
-      for (InterpreterSetting setting : bindedSettings) {
-        settingList.add(new InterpreterSettingsList(setting.getId(), setting.getName(),
-            setting.getInterpreterInfos(), true));
-      }
-    }
-    conn.send(serializeMessage(new Message(OP.INTERPRETER_BINDINGS).put("interpreterBindings", settingList)));
+
+    getNotebook().processNote(noteId,
+      note -> {
+        if (note != null) {
+          List<InterpreterSetting> bindedSettings =
+              note.getBindedInterpreterSettings(new ArrayList<>(context.getUserAndRoles()));
+          for (InterpreterSetting setting : bindedSettings) {
+            settingList.add(new InterpreterSettingsList(setting.getId(), setting.getName(),
+                setting.getInterpreterInfos(), true));
+          }
+        }
+        conn.send(serializeMessage(new Message(OP.INTERPRETER_BINDINGS).put("interpreterBindings", settingList)));
+        return null;
+      });
   }
 
   public void saveInterpreterBindings(NotebookSocket conn, ServiceContext context, Message fromMessage)
       throws IOException {
     List<InterpreterSettingsList> settingList = new ArrayList<>();
     String noteId = (String) fromMessage.data.get("noteId");
-    Note note = getNotebook().getNote(noteId);
-    if (note != null) {
-      List<String> settingIdList =
-          gson.fromJson(String.valueOf(fromMessage.data.get("selectedSettingIds")),
-              new TypeToken<ArrayList<String>>() {
-              }.getType());
-      if (!settingIdList.isEmpty()) {
-        note.setDefaultInterpreterGroup(settingIdList.get(0));
-        getNotebook().saveNote(note, context.getAutheInfo());
-      }
-      List<InterpreterSetting> bindedSettings =
-          note.getBindedInterpreterSettings(new ArrayList<>(context.getUserAndRoles()));
-      for (InterpreterSetting setting : bindedSettings) {
-        settingList.add(new InterpreterSettingsList(setting.getId(), setting.getName(),
-            setting.getInterpreterInfos(), true));
-      }
-    }
-
+    // use write lock, because defaultInterpreterGroup is overwritten
+    getNotebook().processNote(noteId,
+      note -> {
+        if (note != null) {
+          List<String> settingIdList =
+              gson.fromJson(String.valueOf(fromMessage.data.get("selectedSettingIds")),
+                  new TypeToken<ArrayList<String>>() {
+                  }.getType());
+          if (!settingIdList.isEmpty()) {
+            note.setDefaultInterpreterGroup(settingIdList.get(0));
+            getNotebook().saveNote(note, context.getAutheInfo());
+          }
+          List<InterpreterSetting> bindedSettings =
+            note.getBindedInterpreterSettings(new ArrayList<>(context.getUserAndRoles()));
+          for (InterpreterSetting setting : bindedSettings) {
+            settingList.add(new InterpreterSettingsList(setting.getId(), setting.getName(),
+              setting.getInterpreterInfos(), true));
+          }
+        }
+        return null;
+      });
     conn.send(serializeMessage(
         new Message(OP.INTERPRETER_BINDINGS).put("interpreterBindings", settingList)));
   }
@@ -854,7 +862,7 @@ public class NotebookServer implements AngularObjectRegistryListener,
             sendAllAngularObjects(note, context.getAutheInfo().getUser(),
                 conn);
           }
-        });
+        }, null);
   }
 
   private void reloadNote(NotebookSocket conn, ServiceContext context, Message fromMessage)
@@ -872,7 +880,7 @@ public class NotebookServer implements AngularObjectRegistryListener,
             updateAngularObjectRegistry(conn, note);
             sendAllAngularObjects(note, context.getAutheInfo().getUser(), conn);
           }
-        });
+        }, null);
   }
 
   /**
@@ -984,9 +992,12 @@ public class NotebookServer implements AngularObjectRegistryListener,
           @Override
           public void onFailure(Exception ex, ServiceContext context) throws IOException {
             super.onFailure(ex, context);
-
-            // If there was a failure, then resend the latest notebook information to update stale UI
-            broadcastNote(getNotebook().getNote(noteId));
+         // If there was a failure, then resend the latest notebook information to update stale UI
+            getNotebook().processNote(noteId,
+              note -> {
+                broadcastNote(note);
+                return null;
+              });
           }
         });
   }
@@ -1106,9 +1117,9 @@ public class NotebookServer implements AngularObjectRegistryListener,
     String folderPath = (String) fromMessage.get("id");
     folderPath = "/" + folderPath;
     getNotebookService().restoreFolder(folderPath, context,
-        new WebSocketServiceCallback(conn) {
+        new WebSocketServiceCallback<Void>(conn) {
           @Override
-          public void onSuccess(Object result, ServiceContext context) throws IOException {
+          public void onSuccess(Void result, ServiceContext context) throws IOException {
             super.onSuccess(result, context);
             broadcastNoteList(context.getAutheInfo(), context.getUserAndRoles());
           }
@@ -1119,9 +1130,9 @@ public class NotebookServer implements AngularObjectRegistryListener,
                           ServiceContext context,
                           Message fromMessage) throws IOException {
     getNotebookService().restoreAll(context,
-        new WebSocketServiceCallback(conn) {
+        new WebSocketServiceCallback<Void>(conn) {
           @Override
-          public void onSuccess(Object result, ServiceContext context) throws IOException {
+          public void onSuccess(Void result, ServiceContext context) throws IOException {
             super.onSuccess(result, context);
             broadcastNoteList(context.getAutheInfo(), context.getUserAndRoles());
           }
@@ -1130,9 +1141,9 @@ public class NotebookServer implements AngularObjectRegistryListener,
 
   private void emptyTrash(NotebookSocket conn, ServiceContext context) throws IOException {
     getNotebookService().emptyTrash(context,
-        new WebSocketServiceCallback(conn) {
+        new WebSocketServiceCallback<Void>(conn) {
           @Override
-          public void onSuccess(Object result, ServiceContext context) throws IOException {
+          public void onSuccess(Void result, ServiceContext context) throws IOException {
             super.onSuccess(result, context);
             broadcastNoteList(context.getAutheInfo(), context.getUserAndRoles());
           }
@@ -1236,18 +1247,21 @@ public class NotebookServer implements AngularObjectRegistryListener,
 
   protected void convertNote(NotebookSocket conn, Message fromMessage) throws IOException {
     String noteId = fromMessage.get("noteId").toString();
-    Note note = getNotebook().getNote(noteId);
-    if (note == null) {
-      throw new IOException("No such note: " + noteId);
-    } else {
-      Message resp = new Message(OP.CONVERTED_NOTE_NBFORMAT)
-          .put("nbformat", new JupyterUtil().getNbformat(note.toJson()))
-          .put("noteName", fromMessage.get("noteName"));
-      conn.send(serializeMessage(resp));
-    }
+    getNotebook().processNote(noteId,
+      note -> {
+        if (note == null) {
+          throw new IOException("No such note: " + noteId);
+        } else {
+          Message resp = new Message(OP.CONVERTED_NOTE_NBFORMAT)
+              .put("nbformat", new JupyterUtil().getNbformat(note.toJson()))
+              .put("noteName", fromMessage.get("noteName"));
+          conn.send(serializeMessage(resp));
+          return null;
+        }
+      });
   }
 
-  protected Note importNote(NotebookSocket conn, ServiceContext context, Message fromMessage) throws IOException {
+  protected String importNote(NotebookSocket conn, ServiceContext context, Message fromMessage) throws IOException {
     String noteJson = null;
     String noteName = (String) ((Map) fromMessage.get("note")).get("name");
     // Checking whether the notebook data is from a Jupyter or a Zeppelin Notebook.
@@ -1258,7 +1272,7 @@ public class NotebookServer implements AngularObjectRegistryListener,
       noteJson = new JupyterUtil().getJson(
           gson.toJson(fromMessage.get("note")), IdHashes.generateId(), "%python", "%md");
     }
-    Note note = getNotebookService().importNote(noteName, noteJson, context,
+    return getNotebookService().importNote(noteName, noteJson, context,
         new WebSocketServiceCallback<Note>(conn) {
           @Override
           public void onSuccess(Note note, ServiceContext context) throws IOException {
@@ -1272,8 +1286,6 @@ public class NotebookServer implements AngularObjectRegistryListener,
             }
           }
         });
-
-    return note;
   }
 
   private void removeParagraph(NotebookSocket conn,
@@ -1367,8 +1379,11 @@ public class NotebookServer implements AngularObjectRegistryListener,
                     .put("noteId", noteId)
                     .put("paragraphId", ao.getParagraphId()),
                 conn);
-            Note note = getNotebook().getNote(noteId);
-            note.addOrUpdateAngularObject(interpreterGroupId, ao);
+            getNotebook().processNote(noteId,
+              note -> {
+                note.addOrUpdateAngularObject(interpreterGroupId, ao);
+                return null;
+              });
           }
         });
   }
@@ -1383,21 +1398,29 @@ public class NotebookServer implements AngularObjectRegistryListener,
     String varName = fromMessage.getType("name");
     Object varValue = fromMessage.get("value");
     String paragraphId = fromMessage.getType("paragraphId");
-    Note note = getNotebook().getNote(noteId);
-
     if (paragraphId == null) {
       throw new IllegalArgumentException(
           "target paragraph not specified for " + "angular value bind");
     }
 
-    if (note != null) {
-      final InterpreterGroup interpreterGroup = findInterpreterGroupForParagraph(note, paragraphId);
-      final RemoteAngularObjectRegistry registry = (RemoteAngularObjectRegistry)
-          interpreterGroup.getAngularObjectRegistry();
-      AngularObject ao = pushAngularObjectToRemoteRegistry(noteId, paragraphId, varName, varValue,
-          registry, interpreterGroup.getId(), conn);
-      note.addOrUpdateAngularObject(interpreterGroup.getId(), ao);
-    }
+    getNotebook().processNote(noteId,
+      note -> {
+        if (note != null) {
+          InterpreterGroup interpreterGroup;
+          try {
+            interpreterGroup = findInterpreterGroupForParagraph(note, paragraphId);
+          } catch (Exception e) {
+            LOG.error("No interpreter group found for noteId {} and paragraphId {}", noteId, paragraphId, e);
+            return null;
+          }
+          final RemoteAngularObjectRegistry registry = (RemoteAngularObjectRegistry)
+              interpreterGroup.getAngularObjectRegistry();
+          AngularObject ao = pushAngularObjectToRemoteRegistry(noteId, paragraphId, varName, varValue,
+              registry, interpreterGroup.getId(), conn);
+          note.addOrUpdateAngularObject(interpreterGroup.getId(), ao);
+        }
+        return null;
+      });
   }
 
   /**
@@ -1409,21 +1432,29 @@ public class NotebookServer implements AngularObjectRegistryListener,
     String noteId = fromMessage.getType("noteId");
     String varName = fromMessage.getType("name");
     String paragraphId = fromMessage.getType("paragraphId");
-    Note note = getNotebook().getNote(noteId);
-
     if (paragraphId == null) {
       throw new IllegalArgumentException(
           "target paragraph not specified for " + "angular value unBind");
     }
 
-    if (note != null) {
-      final InterpreterGroup interpreterGroup = findInterpreterGroupForParagraph(note, paragraphId);
-      final RemoteAngularObjectRegistry registry =
-          (RemoteAngularObjectRegistry) interpreterGroup.getAngularObjectRegistry();
-      AngularObject ao =
-          removeAngularFromRemoteRegistry(noteId, paragraphId, varName, registry, interpreterGroup.getId(), conn);
-      note.deleteAngularObject(interpreterGroup.getId(), noteId, paragraphId, varName);
-    }
+    getNotebook().processNote(noteId,
+      note -> {
+        if (note != null) {
+          InterpreterGroup interpreterGroup;
+          try {
+            interpreterGroup = findInterpreterGroupForParagraph(note, paragraphId);
+          } catch (Exception e) {
+            LOG.error("No interpreter group found for noteId {} and paragraphId {}", noteId, paragraphId, e);
+            return null;
+          }
+          final RemoteAngularObjectRegistry registry =
+              (RemoteAngularObjectRegistry) interpreterGroup.getAngularObjectRegistry();
+          AngularObject ao =
+              removeAngularFromRemoteRegistry(noteId, paragraphId, varName, registry, interpreterGroup.getId(), conn);
+          note.deleteAngularObject(interpreterGroup.getId(), noteId, paragraphId, varName);
+        }
+        return null;
+      });
   }
 
   private InterpreterGroup findInterpreterGroupForParagraph(Note note, String paragraphId) throws Exception {
@@ -1539,12 +1570,15 @@ public class NotebookServer implements AngularObjectRegistryListener,
             new WebSocketServiceCallback<Paragraph>(conn))) {
           // If one paragraph fails, we need to broadcast paragraph states to the client,
           // or paragraphs not run will stay in PENDING state.
-          Note note = getNotebookService().getNote(noteId, context, new SimpleServiceCallback());
-          if (note != null) {
-            for (Paragraph p : note.getParagraphs()) {
-              broadcastParagraph(note, p, null);
-            }
-          }
+          getNotebookService().getNote(noteId, context, new SimpleServiceCallback<>(),
+            note -> {
+              if (note != null) {
+                for (Paragraph p : note.getParagraphs()) {
+                  broadcastParagraph(note, p, null);
+                }
+              }
+              return null;
+            });
         }
       } catch (Throwable t) {
         NotebookServer.LOG.error("Error in running all paragraphs", t);
@@ -1578,29 +1612,34 @@ public class NotebookServer implements AngularObjectRegistryListener,
     String title = (String) fromMessage.get("title");
     Map<String, Object> params = (Map<String, Object>) fromMessage.get("params");
     Map<String, Object> config = (Map<String, Object>) fromMessage.get("config");
-    getNotebookService().runParagraph(noteId, paragraphId, title, text, params, config, null,
-        false, false, context,
-        new WebSocketServiceCallback<Paragraph>(conn) {
-          @Override
-          public void onSuccess(Paragraph p, ServiceContext context)
-              throws IOException {
-            super.onSuccess(p, context);
-            if (p.getNote().isPersonalizedMode()) {
-              Paragraph p2 = p.getNote().clearPersonalizedParagraphOutput(paragraphId,
-                  context.getAutheInfo().getUser());
-              connectionManager.unicastParagraph(p.getNote(), p2, context.getAutheInfo().getUser(), fromMessage.msgId);
+    getNotebook().processNote(noteId,
+      note -> {
+        getNotebookService().runParagraph(note, paragraphId, title, text, params, config, null,
+          false, false, context,
+          new WebSocketServiceCallback<Paragraph>(conn) {
+            @Override
+            public void onSuccess(Paragraph p, ServiceContext context)
+                throws IOException {
+              super.onSuccess(p, context);
+              if (p.getNote().isPersonalizedMode()) {
+                Paragraph p2 = p.getNote().clearPersonalizedParagraphOutput(paragraphId,
+                    context.getAutheInfo().getUser());
+                connectionManager.unicastParagraph(p.getNote(), p2, context.getAutheInfo().getUser(), fromMessage.msgId);
+              }
+
+              // if it's the last paragraph and not empty, let's add a new one
+              boolean isTheLastParagraph = p.getNote().isLastParagraph(paragraphId);
+              if (!(StringUtils.isEmpty(p.getText()) ||
+                StringUtils.isEmpty(p.getScriptText())) &&
+                  isTheLastParagraph) {
+                Paragraph newPara = p.getNote().addNewParagraph(p.getAuthenticationInfo());
+                broadcastNewParagraph(p.getNote(), newPara);
+              }
             }
+          });
+        return null;
+      });
 
-            // if it's the last paragraph and not empty, let's add a new one
-            boolean isTheLastParagraph = p.getNote().isLastParagraph(paragraphId);
-            if (!(StringUtils.isEmpty(p.getText()) ||
-              StringUtils.isEmpty(p.getScriptText())) &&
-                isTheLastParagraph) {
-              Paragraph newPara = p.getNote().addNewParagraph(p.getAuthenticationInfo());
-              broadcastNewParagraph(p.getNote(), newPara);
-            }
-          }
-        });
   }
 
   private void sendAllConfigurations(NotebookSocket conn,
@@ -1630,8 +1669,9 @@ public class NotebookServer implements AngularObjectRegistryListener,
           public void onSuccess(Revision revision, ServiceContext context) throws IOException {
             super.onSuccess(revision, context);
             if (!Revision.isEmpty(revision)) {
-              List<Revision> revisions = getNotebook().listRevisionHistory(noteId,
-                  getNotebook().getNote(noteId).getPath(), context.getAutheInfo());
+
+              List<Revision> revisions = getNotebook().processNote(noteId,
+                note -> getNotebook().listRevisionHistory(noteId, note.getPath(), context.getAutheInfo()));
               conn.send(serializeMessage(new Message(OP.LIST_REVISION_HISTORY).put("revisionList", revisions)));
             } else {
               conn.send(serializeMessage(
@@ -1751,21 +1791,24 @@ public class NotebookServer implements AngularObjectRegistryListener,
         .put("type", type)
         .put("data", output);
     try {
-      Note note = getNotebook().getNote(noteId);
-      if (note == null) {
-        LOG.warn("Note {} not found", noteId);
-        return;
-      }
-      Paragraph paragraph = note.getParagraph(paragraphId);
-      paragraph.updateOutputBuffer(index, type, output);
-      if (note.isPersonalizedMode()) {
-        String user = note.getParagraph(paragraphId).getUser();
-        if (null != user) {
-          connectionManager.multicastToUser(user, msg);
-        }
-      } else {
-        connectionManager.broadcast(noteId, msg);
-      }
+      getNotebook().processNote(noteId,
+        note -> {
+          if (note == null) {
+            LOG.warn("Note {} not found", noteId);
+            return null;
+          }
+          Paragraph paragraph = note.getParagraph(paragraphId);
+          paragraph.updateOutputBuffer(index, type, output);
+          if (note.isPersonalizedMode()) {
+            String user = note.getParagraph(paragraphId).getUser();
+            if (null != user) {
+              connectionManager.multicastToUser(user, msg);
+            }
+          } else {
+            connectionManager.broadcast(noteId, msg);
+          }
+          return null;
+        });
     } catch (IOException e) {
       LOG.warn("Fail to call onOutputUpdated", e);
     }
@@ -1781,15 +1824,19 @@ public class NotebookServer implements AngularObjectRegistryListener,
     }
 
     try {
-      final Note note = getNotebook().getNote(noteId);
-      if (note == null) {
-        // It is possible the note is removed, but the job is still running
-        LOG.warn("Note {} doesn't existed, it maybe deleted.", noteId);
-      } else {
-        note.clearParagraphOutput(paragraphId);
-        Paragraph paragraph = note.getParagraph(paragraphId);
-        broadcastParagraph(note, paragraph, MSG_ID_NOT_DEFINED);
-      }
+      getNotebook().processNote(noteId,
+        note -> {
+          if (note == null) {
+            // It is possible the note is removed, but the job is still running
+            LOG.warn("Note {} doesn't existed, it maybe deleted.", noteId);
+          } else {
+            note.clearParagraphOutput(paragraphId);
+            Paragraph paragraph = note.getParagraph(paragraphId);
+            broadcastParagraph(note, paragraph, MSG_ID_NOT_DEFINED);
+          }
+          return null;
+        });
+
     } catch (IOException e) {
       LOG.warn("Fail to call onOutputClear", e);
     }
@@ -1844,51 +1891,54 @@ public class NotebookServer implements AngularObjectRegistryListener,
                             List<Integer> paragraphIndices,
                             List<String> paragraphIds,
                             String curParagraphId) throws IOException {
-    final Note note = getNotebook().getNote(noteId);
-    final List<String> toBeRunParagraphIds = new ArrayList<>();
-    if (note == null) {
-      throw new IOException("Not existed noteId: " + noteId);
-    }
-    if (!paragraphIds.isEmpty() && !paragraphIndices.isEmpty()) {
-      throw new IOException("Can not specify paragraphIds and paragraphIndices together");
-    }
-    if (paragraphIds != null && !paragraphIds.isEmpty()) {
-      for (String paragraphId : paragraphIds) {
-        if (note.getParagraph(paragraphId) == null) {
-          throw new IOException("Not existed paragraphId: " + paragraphId);
+    getNotebook().processNote(noteId,
+      note -> {
+        final List<String> toBeRunParagraphIds = new ArrayList<>();
+        if (note == null) {
+          throw new IOException("Not existed noteId: " + noteId);
         }
-        if (!paragraphId.equals(curParagraphId)) {
-          toBeRunParagraphIds.add(paragraphId);
+        if (!paragraphIds.isEmpty() && !paragraphIndices.isEmpty()) {
+          throw new IOException("Can not specify paragraphIds and paragraphIndices together");
         }
-      }
-    }
-    if (paragraphIndices != null && !paragraphIndices.isEmpty()) {
-      for (int paragraphIndex : paragraphIndices) {
-        if (note.getParagraph(paragraphIndex) == null) {
-          throw new IOException("Not existed paragraphIndex: " + paragraphIndex);
-        }
-        if (!note.getParagraph(paragraphIndex).getId().equals(curParagraphId)) {
-          toBeRunParagraphIds.add(note.getParagraph(paragraphIndex).getId());
+        if (paragraphIds != null && !paragraphIds.isEmpty()) {
+          for (String paragraphId : paragraphIds) {
+            if (note.getParagraph(paragraphId) == null) {
+              throw new IOException("Not existed paragraphId: " + paragraphId);
+            }
+            if (!paragraphId.equals(curParagraphId)) {
+              toBeRunParagraphIds.add(paragraphId);
+            }
+          }
         }
-      }
-    }
-    // run the whole note except the current paragraph
-    if (paragraphIds.isEmpty() && paragraphIndices.isEmpty()) {
-      for (Paragraph paragraph : note.getParagraphs()) {
-        if (!paragraph.getId().equals(curParagraphId)) {
-          toBeRunParagraphIds.add(paragraph.getId());
+        if (paragraphIndices != null && !paragraphIndices.isEmpty()) {
+          for (int paragraphIndex : paragraphIndices) {
+            if (note.getParagraph(paragraphIndex) == null) {
+              throw new IOException("Not existed paragraphIndex: " + paragraphIndex);
+            }
+            if (!note.getParagraph(paragraphIndex).getId().equals(curParagraphId)) {
+              toBeRunParagraphIds.add(note.getParagraph(paragraphIndex).getId());
+            }
+          }
         }
-      }
-    }
-    Runnable runThread = new Runnable() {
-      @Override
-      public void run() {
-        for (String paragraphId : toBeRunParagraphIds) {
-          note.run(paragraphId, true);
+        // run the whole note except the current paragraph
+        if (paragraphIds.isEmpty() && paragraphIndices.isEmpty()) {
+          for (Paragraph paragraph : note.getParagraphs()) {
+            if (!paragraph.getId().equals(curParagraphId)) {
+              toBeRunParagraphIds.add(paragraph.getId());
+            }
+          }
         }
-      }
-    };
-    executorService.submit(runThread);
+        Runnable runThread = new Runnable() {
+          @Override
+          public void run() {
+            for (String paragraphId : toBeRunParagraphIds) {
+              note.run(paragraphId, true);
+            }
+          }
+        };
+        executorService.submit(runThread);
+        return null;
+      });
   }
 
   @Override
@@ -1899,7 +1949,7 @@ public class NotebookServer implements AngularObjectRegistryListener,
       getJobManagerService().getNoteJobInfoByUnixTime(System.currentTimeMillis() - 5000, context,
           new JobManagerServiceCallback());
     } catch (IOException e) {
-      LOG.warn("can not broadcast for job manager: " + e.getMessage(), e);
+      LOG.warn("can not broadcast for job manager: {}", e.getMessage(), e);
     }
   }
 
@@ -1908,14 +1958,14 @@ public class NotebookServer implements AngularObjectRegistryListener,
     try {
       broadcastUpdateNoteJobInfo(note, System.currentTimeMillis() - 5000);
     } catch (IOException e) {
-      LOG.warn("can not broadcast for job manager: " + e.getMessage(), e);
+      LOG.warn("can not broadcast for job manager: {}", e.getMessage(), e);
     }
 
     try {
       getJobManagerService().removeNoteJobInfo(note.getId(), null,
           new JobManagerServiceCallback());
     } catch (IOException e) {
-      LOG.warn("can not broadcast for job manager: " + e.getMessage(), e);
+      LOG.warn("can not broadcast for job manager: {}", e.getMessage(), e);
     }
 
   }
@@ -1926,13 +1976,13 @@ public class NotebookServer implements AngularObjectRegistryListener,
       getJobManagerService().getNoteJobInfo(p.getNote().getId(), null,
           new JobManagerServiceCallback());
     } catch (IOException e) {
-      LOG.warn("can not broadcast for job manager: " + e.getMessage(), e);
+      LOG.warn("can not broadcast for job manager: {}", e.getMessage(), e);
     }
   }
 
   @Override
   public void onParagraphUpdate(Paragraph p) {
-
+    // do nothing
   }
 
   @Override
@@ -1941,13 +1991,13 @@ public class NotebookServer implements AngularObjectRegistryListener,
       getJobManagerService().getNoteJobInfo(note.getId(), null,
           new JobManagerServiceCallback());
     } catch (IOException e) {
-      LOG.warn("can not broadcast for job manager: " + e.getMessage(), e);
+      LOG.warn("can not broadcast for job manager: {}", e.getMessage(), e);
     }
   }
 
   @Override
   public void onNoteUpdate(Note note, AuthenticationInfo subject) {
-
+    // do nothing
   }
 
   @Override
@@ -1956,7 +2006,7 @@ public class NotebookServer implements AngularObjectRegistryListener,
       getJobManagerService().getNoteJobInfo(p.getNote().getId(), null,
           new JobManagerServiceCallback());
     } catch (IOException e) {
-      LOG.warn("can not broadcast for job manager: " + e.getMessage(), e);
+      LOG.warn("can not broadcast for job manager: {}", e.getMessage(), e);
     }
   }
 
@@ -2002,12 +2052,17 @@ public class NotebookServer implements AngularObjectRegistryListener,
       }
 
       try {
-        if (getNotebook().getNote(p.getNote().getId()) == null) {
-          // It is possible the note is removed, but the job is still running
-          LOG.warn("Note {} doesn't existed.", p.getNote().getId());
-        } else {
-          getNotebook().saveNote(p.getNote(), p.getAuthenticationInfo());
-        }
+        String noteId = p.getNote().getId();
+        getNotebook().processNote(noteId,
+          note -> {
+            if (note == null) {
+              LOG.warn("Note {} doesn't existed.", noteId);
+              return null;
+            } else {
+              getNotebook().saveNote(p.getNote(), p.getAuthenticationInfo());
+            }
+            return null;
+          });
       } catch (IOException e) {
         LOG.error(e.toString(), e);
       }
@@ -2025,11 +2080,14 @@ public class NotebookServer implements AngularObjectRegistryListener,
   @Override
   public void checkpointOutput(String noteId, String paragraphId) {
     try {
-      Note note = getNotebook().getNote(noteId);
-      note.getParagraph(paragraphId).checkpointOutput();
-      getNotebook().saveNote(note, AuthenticationInfo.ANONYMOUS);
+      getNotebook().processNote(noteId,
+        note -> {
+          note.getParagraph(paragraphId).checkpointOutput();
+          getNotebook().saveNote(note, AuthenticationInfo.ANONYMOUS);
+          return null;
+        });
     } catch (IOException e) {
-      LOG.warn("Fail to save note: " + noteId, e);
+      LOG.warn("Fail to save note: {}", noteId, e);
     }
   }
 
@@ -2042,7 +2100,7 @@ public class NotebookServer implements AngularObjectRegistryListener,
       throws IOException {
     List<InterpreterSetting> settings =
         getNotebook().getBindedInterpreterSettings(note.getId());
-    if (settings == null || settings.size() == 0) {
+    if (settings == null || settings.isEmpty()) {
       return;
     }
 
@@ -2080,31 +2138,34 @@ public class NotebookServer implements AngularObjectRegistryListener,
     // not global scope, so we just need to load the corresponded note.
     if (angularObject.getNoteId() != null) {
       try {
-        Note note = getNotebook().getNote(angularObject.getNoteId());
-        updateNoteAngularObject(note, angularObject, interpreterGroupId);
+        updateNoteAngularObject(angularObject.getNoteId(), angularObject, interpreterGroupId);
       } catch (IOException e) {
-        LOG.error("AngularObject's note: {} is not found", angularObject.getNoteId());
+        LOG.error("AngularObject's note: {} is not found", angularObject.getNoteId(), e);
       }
     } else {
       // global scope angular object needs to load and iterate all notes, this is inefficient.
-      getNotebook().getNoteStream().forEach(note -> {
-        if (angularObject.getNoteId() != null && !note.getId().equals(angularObject.getNoteId())) {
+      getNotebook().getNotesInfo().stream().forEach(noteInfo -> {
+        if (angularObject.getNoteId() != null && !noteInfo.getId().equals(angularObject.getNoteId())) {
           return;
         }
-        updateNoteAngularObject(note, angularObject, interpreterGroupId);
+        try {
+          updateNoteAngularObject(noteInfo.getId(), angularObject, interpreterGroupId);
+        } catch (IOException e) {
+          LOG.error("AngularObject's note: {} is not found", angularObject.getNoteId(), e);
+        }
       });
     }
   }
 
-  private void updateNoteAngularObject(Note note, AngularObject angularObject, String interpreterGroupId) {
-    List<InterpreterSetting> intpSettings =
-        note.getBindedInterpreterSettings(new ArrayList<>(authorizationService.getOwners(note.getId())));
+  private void updateNoteAngularObject(String noteId, AngularObject angularObject, String interpreterGroupId) throws IOException {
+    List<InterpreterSetting> intpSettings = getNotebook().
+      processNote(noteId, note -> note.getBindedInterpreterSettings(new ArrayList<>(authorizationService.getOwners(note.getId()))));
     if (intpSettings.isEmpty()) {
       return;
     }
-    connectionManager.broadcast(note.getId(), new Message(OP.ANGULAR_OBJECT_UPDATE)
+    connectionManager.broadcast(noteId, new Message(OP.ANGULAR_OBJECT_UPDATE)
         .put("angularObject", angularObject)
-        .put("interpreterGroupId", interpreterGroupId).put("noteId", note.getId())
+        .put("interpreterGroupId", interpreterGroupId).put("noteId", noteId)
         .put("paragraphId", angularObject.getParagraphId()));
   }
 
@@ -2112,19 +2173,15 @@ public class NotebookServer implements AngularObjectRegistryListener,
   public void onRemoveAngularObject(String interpreterGroupId, AngularObject angularObject) {
     // not global scope, so we just need to load the corresponded note.
     if (angularObject.getNoteId() != null) {
-      try {
-        Note note = getNotebook().getNote(angularObject.getNoteId());
-        removeNoteAngularObject(angularObject.getNoteId(), angularObject, interpreterGroupId);
-      } catch (IOException e) {
-        LOG.error("AngularObject's note: {} is not found", angularObject.getNoteId());
-      }
+      String noteId = angularObject.getNoteId();
+      removeNoteAngularObject(noteId, angularObject, interpreterGroupId);
     } else {
       // global scope angular object needs to load and iterate all notes, this is inefficient.
-      getNotebook().getNoteStream().forEach(note -> {
-        if (angularObject.getNoteId() != null && !note.getId().equals(angularObject.getNoteId())) {
+      getNotebook().getNotesInfo().forEach(noteInfo -> {
+        if (angularObject.getNoteId() != null && !noteInfo.getId().equals(angularObject.getNoteId())) {
           return;
         }
-        removeNoteAngularObject(note.getId(), angularObject, interpreterGroupId);
+        removeNoteAngularObject(noteInfo.getId(), angularObject, interpreterGroupId);
       });
     }
   }
@@ -2187,28 +2244,30 @@ public class NotebookServer implements AngularObjectRegistryListener,
   public void onParaInfosReceived(String noteId, String paragraphId,
                                   String interpreterSettingId, Map<String, String> metaInfos) {
     try {
-      Note note = getNotebook().getNote(noteId);
-      if (note != null) {
-        Paragraph paragraph = note.getParagraph(paragraphId);
-        if (paragraph != null) {
-          InterpreterSetting setting = getNotebook().getInterpreterSettingManager()
-              .get(interpreterSettingId);
-          String label = metaInfos.get("label");
-          String tooltip = metaInfos.get("tooltip");
-          List<String> keysToRemove = Arrays.asList("noteId", "paraId", "label", "tooltip");
-          for (String removeKey : keysToRemove) {
-            metaInfos.remove(removeKey);
+      getNotebook().processNote(noteId,
+        note -> {
+          if (note != null) {
+            Paragraph paragraph = note.getParagraph(paragraphId);
+            if (paragraph != null) {
+              InterpreterSetting setting = getNotebook().getInterpreterSettingManager()
+                  .get(interpreterSettingId);
+              String label = metaInfos.get("label");
+              String tooltip = metaInfos.get("tooltip");
+              List<String> keysToRemove = Arrays.asList("noteId", "paraId", "label", "tooltip");
+              for (String removeKey : keysToRemove) {
+                metaInfos.remove(removeKey);
+              }
+              paragraph
+                .updateRuntimeInfos(label, tooltip, metaInfos, setting.getGroup(), setting.getId());
+              getNotebook().saveNote(note, AuthenticationInfo.ANONYMOUS);
+              connectionManager.broadcast(
+                  note.getId(),
+                  new Message(OP.PARAS_INFO).put("id", paragraphId).put("infos",
+                      paragraph.getRuntimeInfos()));
+            }
           }
-
-          paragraph
-              .updateRuntimeInfos(label, tooltip, metaInfos, setting.getGroup(), setting.getId());
-          getNotebook().saveNote(note, AuthenticationInfo.ANONYMOUS);
-          connectionManager.broadcast(
-              note.getId(),
-              new Message(OP.PARAS_INFO).put("id", paragraphId).put("infos",
-                  paragraph.getRuntimeInfos()));
-        }
-      }
+          return null;
+        });
     } catch (IOException e) {
       LOG.warn("Fail to call onParaInfosReceived", e);
     }
@@ -2216,37 +2275,35 @@ public class NotebookServer implements AngularObjectRegistryListener,
 
   @Override
   public List<ParagraphInfo> getParagraphList(String user, String noteId)
-      throws TException, IOException {
-    Notebook notebook = getNotebook();
-    Note note = notebook.getNote(noteId);
-    if (null == note) {
-      throw new ServiceException("Not found this note : " + noteId);
-    }
-
-    // Check READER permission
-    Set<String> userAndRoles = new HashSet<>();
-    userAndRoles.add(user);
-    AuthorizationService notebookAuthorization = authorizationService;
-    boolean isAllowed = notebookAuthorization.isReader(noteId, userAndRoles);
-    Set<String> allowed = notebookAuthorization.getReaders(noteId);
-    if (!isAllowed) {
-      String errorMsg = "Insufficient privileges to READER note. " +
-          "Allowed users or roles: " + allowed;
-      throw new ServiceException(errorMsg);
-    }
-
-    // Convert Paragraph to ParagraphInfo
-    List<ParagraphInfo> paragraphInfos = new ArrayList<>();
-    List<Paragraph> paragraphs = note.getParagraphs();
-    for (Paragraph paragraph : paragraphs) {
-      ParagraphInfo paraInfo = new ParagraphInfo();
-      paraInfo.setNoteId(noteId);
-      paraInfo.setParagraphId(paragraph.getId());
-      paraInfo.setParagraphTitle(paragraph.getTitle());
-      paraInfo.setParagraphText(paragraph.getText());
-      paragraphInfos.add(paraInfo);
-    }
-    return paragraphInfos;
+      throws IOException, TException, ServiceException{
+      // Check READER permission
+      Set<String> userAndRoles = new HashSet<>();
+      userAndRoles.add(user);
+      boolean isAllowed = authorizationService.isReader(noteId, userAndRoles);
+      Set<String> allowed = authorizationService.getReaders(noteId);
+      if (!isAllowed) {
+        String errorMsg = "Insufficient privileges to READER note. " +
+            "Allowed users or roles: " + allowed;
+        throw new ServiceException(errorMsg);
+      }
+      return getNotebook().processNote(noteId,
+        note -> {
+          if (null == note) {
+            throw new IOException("Not found this note : " + noteId);
+          }
+          // Convert Paragraph to ParagraphInfo
+          List<ParagraphInfo> paragraphInfos = new ArrayList<>();
+          List<Paragraph> paragraphs = note.getParagraphs();
+          for (Paragraph paragraph : paragraphs) {
+            ParagraphInfo paraInfo = new ParagraphInfo();
+            paraInfo.setNoteId(noteId);
+            paraInfo.setParagraphId(paragraph.getId());
+            paraInfo.setParagraphTitle(paragraph.getTitle());
+            paraInfo.setParagraphText(paragraph.getText());
+            paragraphInfos.add(paraInfo);
+          }
+          return paragraphInfos;
+        });
   }
 
   private void broadcastNoteForms(Note note) {
diff --git a/zeppelin-server/src/test/java/org/apache/zeppelin/cluster/ClusterEventTest.java b/zeppelin-server/src/test/java/org/apache/zeppelin/cluster/ClusterEventTest.java
index cd588bd..cd16bf8 100644
--- a/zeppelin-server/src/test/java/org/apache/zeppelin/cluster/ClusterEventTest.java
+++ b/zeppelin-server/src/test/java/org/apache/zeppelin/cluster/ClusterEventTest.java
@@ -289,12 +289,15 @@ public class ClusterEventTest extends ZeppelinServerMock {
 
   @Test
   public void testRenameNoteEvent() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
       String oldName = "old_name";
-      note = TestUtils.getInstance(Notebook.class).createNote(oldName, anonymous);
-      assertEquals(note.getName(), oldName);
-      String noteId = note.getId();
+      noteId = TestUtils.getInstance(Notebook.class).createNote(oldName, anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          assertEquals(note.getName(), oldName);
+          return null;
+        });
 
       final String newName = "testName";
       String jsonRequest = "{\"name\": " + newName + "}";
@@ -303,7 +306,11 @@ public class ClusterEventTest extends ZeppelinServerMock {
       assertThat("test testRenameNote:", put, AbstractTestRestApi.isAllowed());
       put.close();
 
-      assertEquals(note.getName(), newName);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          assertEquals(note.getName(), newName);
+          return null;
+        });
 
       // wait cluster sync event
       Thread.sleep(1000);
@@ -312,21 +319,21 @@ public class ClusterEventTest extends ZeppelinServerMock {
       LOGGER.error(e.getMessage(), e);
     } finally {
       // cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testCloneNoteEvent() throws IOException {
-    Note note1 = null;
+    String note1Id = null;
     String clonedNoteId = null;
     try {
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
       Thread.sleep(1000);
 
-      CloseableHttpResponse post = AbstractTestRestApi.httpPost("/notebook/" + note1.getId(), "");
+      CloseableHttpResponse post = AbstractTestRestApi.httpPost("/notebook/" + note1Id, "");
       LOG.info("testCloneNote response\n" + post.getStatusLine().getReasonPhrase());
       assertThat(post, AbstractTestRestApi.isAllowed());
 
@@ -351,31 +358,34 @@ public class ClusterEventTest extends ZeppelinServerMock {
       LOGGER.error(e.getMessage(), e);
     } finally {
       // cleanup
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
-      Note clonedNote = TestUtils.getInstance(Notebook.class).getNote(clonedNoteId);
-      if (null != clonedNote) {
-        TestUtils.getInstance(Notebook.class).removeNote(clonedNote, anonymous);
+      if (null != clonedNoteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(clonedNoteId, anonymous);
       }
     }
   }
 
   @Test
   public void insertParagraphEvent() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
       // Create note and set result explicitly
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p1 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      InterpreterResult result = new InterpreterResult(InterpreterResult.Code.SUCCESS,
-          InterpreterResult.Type.TEXT, "result");
-      p1.setResult(result);
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p1 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          InterpreterResult result = new InterpreterResult(InterpreterResult.Code.SUCCESS,
+              InterpreterResult.Type.TEXT, "result");
+          p1.setResult(result);
+          return null;
+        });
 
       // insert new paragraph
       NewParagraphRequest newParagraphRequest = new NewParagraphRequest();
 
-      CloseableHttpResponse post = AbstractTestRestApi.httpPost("/notebook/" + note.getId() + "/paragraph", newParagraphRequest.toJson());
+      CloseableHttpResponse post = AbstractTestRestApi.httpPost("/notebook/" + noteId + "/paragraph", newParagraphRequest.toJson());
       LOG.info("test clear paragraph output response\n" + EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8));
       assertThat(post, AbstractTestRestApi.isAllowed());
       post.close();
@@ -387,24 +397,28 @@ public class ClusterEventTest extends ZeppelinServerMock {
       LOGGER.error(e.getMessage(), e);
     } finally {
       // cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testClusterAuthEvent() throws IOException {
-    Note note = null;
+    String noteId = null;
 
     try {
-      note = notebook.createNote("note1", anonymous);
-      Paragraph p1 = note.addNewParagraph(anonymous);
-      p1.setText("%md start remote interpreter process");
-      p1.setAuthenticationInfo(anonymous);
-      notebookServer.getNotebook().saveNote(note, anonymous);
+      noteId = notebook.createNote("note1", anonymous);
+      notebook.processNote(noteId,
+        note -> {
+          Paragraph p1 = note.addNewParagraph(anonymous);
+          p1.setText("%md start remote interpreter process");
+          p1.setAuthenticationInfo(anonymous);
+          notebookServer.getNotebook().saveNote(note, anonymous);
+          return null;
+        });
+
 
-      String noteId = note.getId();
       String user1Id = "user1", user2Id = "user2";
 
       // test user1 can get anonymous's note
@@ -453,8 +467,8 @@ public class ClusterEventTest extends ZeppelinServerMock {
     } catch (InterruptedException e) {
       LOGGER.error(e.getMessage(), e);
     } finally {
-      if (null != note) {
-        notebook.removeNote(note, anonymous);
+      if (null != noteId) {
+        notebook.removeNote(noteId, anonymous);
       }
     }
   }
diff --git a/zeppelin-server/src/test/java/org/apache/zeppelin/recovery/RecoveryTest.java b/zeppelin-server/src/test/java/org/apache/zeppelin/recovery/RecoveryTest.java
index 5a60f3d..fb51806 100644
--- a/zeppelin-server/src/test/java/org/apache/zeppelin/recovery/RecoveryTest.java
+++ b/zeppelin-server/src/test/java/org/apache/zeppelin/recovery/RecoveryTest.java
@@ -22,12 +22,12 @@ import org.apache.commons.io.FileUtils;
 import org.apache.http.client.methods.CloseableHttpResponse;
 import org.apache.http.util.EntityUtils;
 import org.apache.zeppelin.conf.ZeppelinConfiguration;
+import org.apache.zeppelin.interpreter.InterpreterException;
 import org.apache.zeppelin.interpreter.InterpreterSetting;
 import org.apache.zeppelin.interpreter.InterpreterSettingManager;
 import org.apache.zeppelin.interpreter.ManagedInterpreterGroup;
 import org.apache.zeppelin.interpreter.recovery.FileSystemRecoveryStorage;
 import org.apache.zeppelin.interpreter.recovery.StopInterpreter;
-import org.apache.zeppelin.notebook.Note;
 import org.apache.zeppelin.notebook.Notebook;
 import org.apache.zeppelin.notebook.Paragraph;
 import org.apache.zeppelin.rest.AbstractTestRestApi;
@@ -80,43 +80,60 @@ public class RecoveryTest extends AbstractTestRestApi {
   @Test
   public void testRecovery() throws Exception {
     LOG.info("Test testRecovery");
-    Note note1 = null;
+    String note1Id = null;
     try {
-      note1 = notebook.createNote("note1", anonymous);
-
-      // run python interpreter and create new variable `user`
-      Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p1.setText("%python user='abc'");
-      CloseableHttpResponse post = httpPost("/notebook/job/" + note1.getId() +"?blocking=true", "");
+      note1Id = notebook.createNote("note1", anonymous);
+      notebook.processNote(note1Id,
+        note1 -> {
+          // run python interpreter and create new variable `user`
+          Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p1.setText("%python user='abc'");
+          return null;
+        });
+
+      CloseableHttpResponse post = httpPost("/notebook/job/" + note1Id +"?blocking=true", "");
       assertThat(post, isAllowed());
       Map<String, Object> resp = gson.fromJson(EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
       assertEquals("OK", resp.get("status"));
       post.close();
-      assertEquals(Job.Status.FINISHED, p1.getStatus());
-      TestUtils.getInstance(Notebook.class).saveNote(note1, anonymous);
+      notebook.processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          assertEquals(Job.Status.FINISHED, p1.getStatus());
+          TestUtils.getInstance(Notebook.class).saveNote(note1, anonymous);
+          return null;
+        });
 
       // shutdown zeppelin and restart it
       shutDown();
       startUp(RecoveryTest.class.getSimpleName(), false);
 
       // run the paragraph again, but change the text to print variable `user`
-      note1 = TestUtils.getInstance(Notebook.class).getNote(note1.getId());
       Thread.sleep(10 * 1000);
-      note1 = TestUtils.getInstance(Notebook.class).getNote(note1.getId());
-      p1 = note1.getParagraph(p1.getId());
-      p1.setText("%python print(user)");
-      post = httpPost("/notebook/job/" + note1.getId() + "?blocking=true", "");
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          p1 = note1.getParagraph(p1.getId());
+          p1.setText("%python print(user)");
+          return null;
+        });
+      post = httpPost("/notebook/job/" + note1Id + "?blocking=true", "");
       assertEquals("OK", resp.get("status"));
       post.close();
-      assertEquals(Job.Status.FINISHED, p1.getStatus());
-      assertEquals("abc\n", p1.getReturn().message().get(0).getData());
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          assertEquals(Job.Status.FINISHED, p1.getStatus());
+          assertEquals("abc\n", p1.getReturn().message().get(0).getData());
+          return null;
+        });
     } catch (Exception e) {
       LOG.error(e.toString(), e);
       throw e;
     } finally {
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
@@ -124,26 +141,39 @@ public class RecoveryTest extends AbstractTestRestApi {
   @Test
   public void testRecovery_2() throws Exception {
     LOG.info("Test testRecovery_2");
-    Note note1 = null;
+    String note1Id = null;
     try {
-      note1 = notebook.createNote("note2", AuthenticationInfo.ANONYMOUS);
+      note1Id = notebook.createNote("note2", AuthenticationInfo.ANONYMOUS);
 
       // run python interpreter and create new variable `user`
-      Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p1.setText("%python user='abc'");
-      CloseableHttpResponse post = httpPost("/notebook/job/" + note1.getId() + "?blocking=true", "");
+      notebook.processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p1.setText("%python user='abc'");
+          return null;
+        });
+      CloseableHttpResponse post = httpPost("/notebook/job/" + note1Id + "?blocking=true", "");
       assertThat(post, isAllowed());
       Map<String, Object> resp = gson.fromJson(EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
       assertEquals("OK", resp.get("status"));
       post.close();
-      assertEquals(Job.Status.FINISHED, p1.getStatus());
-      TestUtils.getInstance(Notebook.class).saveNote(note1, AuthenticationInfo.ANONYMOUS);
-      // restart the python interpreter
-      TestUtils.getInstance(Notebook.class).getInterpreterSettingManager().restart(
-          ((ManagedInterpreterGroup) p1.getBindedInterpreter().getInterpreterGroup())
-              .getInterpreterSetting().getId()
-      );
+      notebook.processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          assertEquals(Job.Status.FINISHED, p1.getStatus());
+          TestUtils.getInstance(Notebook.class).saveNote(note1, AuthenticationInfo.ANONYMOUS);
+          // restart the python interpreter
+          try {
+            TestUtils.getInstance(Notebook.class).getInterpreterSettingManager().restart(
+                ((ManagedInterpreterGroup) p1.getBindedInterpreter().getInterpreterGroup())
+                    .getInterpreterSetting().getId()
+            );
+          } catch (InterpreterException e) {
+            fail();
+          }
+          return null;
+        });
 
       // shutdown zeppelin and restart it
       shutDown();
@@ -152,19 +182,27 @@ public class RecoveryTest extends AbstractTestRestApi {
       Thread.sleep(5 * 1000);
       // run the paragraph again, but change the text to print variable `user`.
       // can not recover the python interpreter, because it has been shutdown.
-      note1 = TestUtils.getInstance(Notebook.class).getNote(note1.getId());
-      p1 = note1.getParagraph(p1.getId());
-      p1.setText("%python print(user)");
-      post = httpPost("/notebook/job/" + note1.getId() + "?blocking=true", "");
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          p1.setText("%python print(user)");
+          return null;
+        });
+      post = httpPost("/notebook/job/" + note1Id + "?blocking=true", "");
       assertEquals("OK", resp.get("status"));
       post.close();
-      assertEquals(Job.Status.ERROR, p1.getStatus());
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          assertEquals(Job.Status.ERROR, p1.getStatus());
+          return null;
+        });
     } catch (Exception e) {
       LOG.error(e.toString(), e);
       throw e;
     } finally {
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
@@ -172,21 +210,30 @@ public class RecoveryTest extends AbstractTestRestApi {
   @Test
   public void testRecovery_3() throws Exception {
     LOG.info("Test testRecovery_3");
-    Note note1 = null;
+    String note1Id = null;
     try {
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note3", AuthenticationInfo.ANONYMOUS);
-
-      // run python interpreter and create new variable `user`
-      Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p1.setText("%python user='abc'");
-      CloseableHttpResponse post = httpPost("/notebook/job/" + note1.getId() + "?blocking=true", "");
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note3", AuthenticationInfo.ANONYMOUS);
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          // run python interpreter and create new variable `user`
+          Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p1.setText("%python user='abc'");
+          return null;
+        });
+
+      CloseableHttpResponse post = httpPost("/notebook/job/" + note1Id + "?blocking=true", "");
       assertThat(post, isAllowed());
       Map<String, Object> resp = gson.fromJson(EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
       assertEquals("OK", resp.get("status"));
       post.close();
-      assertEquals(Job.Status.FINISHED, p1.getStatus());
-      TestUtils.getInstance(Notebook.class).saveNote(note1, AuthenticationInfo.ANONYMOUS);
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          assertEquals(Job.Status.FINISHED, p1.getStatus());
+          TestUtils.getInstance(Notebook.class).saveNote(note1, AuthenticationInfo.ANONYMOUS);
+          return null;
+        });
 
       // shutdown zeppelin and restart it
       shutDown();
@@ -197,19 +244,28 @@ public class RecoveryTest extends AbstractTestRestApi {
       Thread.sleep(5 * 1000);
       // run the paragraph again, but change the text to print variable `user`.
       // can not recover the python interpreter, because it has been shutdown.
-      note1 = TestUtils.getInstance(Notebook.class).getNote(note1.getId());
-      p1 = note1.getParagraph(p1.getId());
-      p1.setText("%python print(user)");
-      post = httpPost("/notebook/job/" + note1.getId() + "?blocking=true", "");
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          p1.setText("%python print(user)");
+          return null;
+        });
+
+      post = httpPost("/notebook/job/" + note1Id + "?blocking=true", "");
       assertEquals("OK", resp.get("status"));
       post.close();
-      assertEquals(Job.Status.ERROR, p1.getStatus());
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          assertEquals(Job.Status.ERROR, p1.getStatus());
+          return null;
+        });
     } catch (Exception e ) {
       LOG.error(e.toString(), e);
       throw e;
     } finally {
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
@@ -217,14 +273,16 @@ public class RecoveryTest extends AbstractTestRestApi {
   @Test
   public void testRecovery_Running_Paragraph_sh() throws Exception {
     LOG.info("Test testRecovery_Running_Paragraph_sh");
-    Note note1 = null;
+    String note1Id = null;
     try {
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note4", AuthenticationInfo.ANONYMOUS);
-
-      // run sh paragraph async, print 'hello' after 10 seconds
-      Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note4", AuthenticationInfo.ANONYMOUS);
+      Paragraph p1 = TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          return note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+        });
       p1.setText("%sh sleep 10\necho 'hello'");
-      CloseableHttpResponse post = httpPost("/notebook/job/" + note1.getId() + "/" + p1.getId(), "");
+      // run sh paragraph async, print 'hello' after 10 seconds
+      CloseableHttpResponse post = httpPost("/notebook/job/" + note1Id + "/" + p1.getId(), "");
       assertThat(post, isAllowed());
       post.close();
       long start = System.currentTimeMillis();
@@ -259,8 +317,8 @@ public class RecoveryTest extends AbstractTestRestApi {
       LOG.error(e.toString(), e);
       throw e;
     } finally {
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
@@ -268,22 +326,25 @@ public class RecoveryTest extends AbstractTestRestApi {
   @Test
   public void testRecovery_Finished_Paragraph_python() throws Exception {
     LOG.info("Test testRecovery_Finished_Paragraph_python");
-    Note note1 = null;
+    String note1Id = null;
     try {
       InterpreterSettingManager interpreterSettingManager = TestUtils.getInstance(InterpreterSettingManager.class);
       InterpreterSetting interpreterSetting = interpreterSettingManager.getInterpreterSettingByName("python");
       interpreterSetting.setProperty("zeppelin.python.useIPython", "false");
       interpreterSetting.setProperty("zeppelin.interpreter.result.cache", "100");
 
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note4", AuthenticationInfo.ANONYMOUS);
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note4", AuthenticationInfo.ANONYMOUS);
 
       // run  paragraph async, print 'hello' after 10 seconds
-      Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+      Paragraph p1 = TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          return note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+        });
       p1.setText("%python import time\n" +
               "for i in range(1, 10):\n" +
               "    time.sleep(1)\n" +
               "    print(i)");
-      CloseableHttpResponse post = httpPost("/notebook/job/" + note1.getId() + "/" + p1.getId(), "");
+      CloseableHttpResponse post = httpPost("/notebook/job/" + note1Id + "/" + p1.getId(), "");
       assertThat(post, isAllowed());
       post.close();
 
@@ -315,8 +376,8 @@ public class RecoveryTest extends AbstractTestRestApi {
       LOG.error(e.toString(), e);
       throw e;
     } finally {
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
diff --git a/zeppelin-server/src/test/java/org/apache/zeppelin/rest/InterpreterRestApiTest.java b/zeppelin-server/src/test/java/org/apache/zeppelin/rest/InterpreterRestApiTest.java
index abfb754..7b3e693 100644
--- a/zeppelin-server/src/test/java/org/apache/zeppelin/rest/InterpreterRestApiTest.java
+++ b/zeppelin-server/src/test/java/org/apache/zeppelin/rest/InterpreterRestApiTest.java
@@ -26,11 +26,9 @@ import org.apache.http.client.methods.CloseableHttpResponse;
 import org.apache.http.util.EntityUtils;
 import org.apache.zeppelin.interpreter.InterpreterOption;
 import org.apache.zeppelin.interpreter.InterpreterSetting;
-import org.apache.zeppelin.notebook.Note;
 import org.apache.zeppelin.notebook.Notebook;
 import org.apache.zeppelin.notebook.Paragraph;
 import org.apache.zeppelin.scheduler.Job.Status;
-import org.apache.zeppelin.server.ZeppelinServer;
 import org.apache.zeppelin.user.AuthenticationInfo;
 import org.apache.zeppelin.utils.TestUtils;
 import org.junit.AfterClass;
@@ -292,28 +290,50 @@ public class InterpreterRestApiTest extends AbstractTestRestApi {
 
   }
 
+  @Test
   public void testInterpreterRestart() throws IOException, InterruptedException {
-    Note note = null;
+    String noteId = null;
     try {
       // when: create new note
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      Paragraph p = note.getLastParagraph();
-      Map config = p.getConfig();
-      config.put("enabled", true);
-
-      // when: run markdown paragraph
-      p.setConfig(config);
-      p.setText("%md markdown");
-      p.setAuthenticationInfo(anonymous);
-      note.run(p.getId());
-      while (p.getStatus() != Status.FINISHED) {
-        Thread.sleep(100);
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+
+      String pId = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          Map<String, Object> config = p.getConfig();
+          config.put("enabled", true);
+
+          // when: run markdown paragraph
+          p.setConfig(config);
+          p.setText("%md markdown");
+          p.setAuthenticationInfo(anonymous);
+          note.run(p.getId());
+          return p.getId();
+        });
+
+      Status status = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.getParagraph(pId);
+          return p.getStatus();
+        });
+      while (status != Status.FINISHED) {
+         Thread.sleep(100);
+         status = TestUtils.getInstance(Notebook.class).processNote(noteId,
+           note -> {
+             Paragraph p = note.getParagraph(pId);
+              return p.getStatus();
+           });
       }
-      assertEquals(p.getReturn().message().get(0).getData(), getSimulatedMarkdownResult("markdown"));
+
+      List<InterpreterSetting> settings = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.getParagraph(pId);
+          assertEquals(p.getReturn().message().get(0).getData(), getSimulatedMarkdownResult("markdown"));
+          return note.getBindedInterpreterSettings(new ArrayList<>());
+        });
 
       // when: restart interpreter
-      for (InterpreterSetting setting : note.getBindedInterpreterSettings(new ArrayList<>())) {
+      for (InterpreterSetting setting : settings) {
         if (setting.getName().equals("md")) {
           // call restart interpreter API
           CloseableHttpResponse put = httpPut("/interpreter/setting/restart/" + setting.getId(), "");
@@ -324,56 +344,98 @@ public class InterpreterRestApiTest extends AbstractTestRestApi {
       }
 
       // when: run markdown paragraph, again
-      p = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p.setConfig(config);
-      p.setText("%md markdown restarted");
-      p.setAuthenticationInfo(anonymous);
-      note.run(p.getId());
-      while (p.getStatus() != Status.FINISHED) {
+      String p2Id = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          Map<String, Object> config = p.getConfig();
+          config.put("enabled", true);
+
+          // when: run markdown paragraph
+          p.setConfig(config);
+          p.setText("%md markdown restarted");
+          p.setAuthenticationInfo(anonymous);
+          note.run(p.getId());
+          return p.getId();
+        });
+
+      status = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.getParagraph(p2Id);
+          return p.getStatus();
+        });
+      while (status != Status.FINISHED) {
         Thread.sleep(100);
+        status = TestUtils.getInstance(Notebook.class).processNote(noteId,
+          note -> {
+            Paragraph p = note.getParagraph(p2Id);
+            return p.getStatus();
+          });
       }
 
       // then
-      assertEquals(p.getReturn().message().get(0).getData(),
-              getSimulatedMarkdownResult("markdown restarted"));
+      status = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.getParagraph(p2Id);
+          assertEquals(p.getReturn().message().get(0).getData(),
+            getSimulatedMarkdownResult("markdown restarted"));
+          return null;
+        });
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testRestartInterpreterPerNote() throws IOException, InterruptedException {
-    Note note = null;
+    String noteId = null;
     try {
       // when: create new note
-      note = TestUtils.getInstance(Notebook.class).createNote("note2", anonymous);
-      note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      Paragraph p = note.getLastParagraph();
-      Map config = p.getConfig();
-      config.put("enabled", true);
-
-      // when: run markdown paragraph.
-      p.setConfig(config);
-      p.setText("%md markdown");
-      p.setAuthenticationInfo(anonymous);
-      note.run(p.getId());
-      while (p.getStatus() != Status.FINISHED) {
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note2", anonymous);
+      String pId = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          Map<String, Object> config = p.getConfig();
+          config.put("enabled", true);
+          // when: run markdown paragraph.
+          p.setConfig(config);
+          p.setText("%md markdown");
+          p.setAuthenticationInfo(anonymous);
+          note.run(p.getId());
+          return p.getId();
+        });
+
+      Status status = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.getParagraph(pId);
+          return p.getStatus();
+        });
+      while (status != Status.FINISHED) {
         Thread.sleep(100);
+        status = TestUtils.getInstance(Notebook.class).processNote(noteId,
+          note -> {
+            Paragraph p = note.getParagraph(pId);
+            return p.getStatus();
+          });
       }
-      assertEquals(p.getReturn().message().get(0).getData(), getSimulatedMarkdownResult("markdown"));
+      List<InterpreterSetting> settings = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.getParagraph(pId);
+          assertEquals(p.getReturn().message().get(0).getData(), getSimulatedMarkdownResult("markdown"));
+          return note.getBindedInterpreterSettings(new ArrayList<>());
+        });
 
       // when: get md interpreter
       InterpreterSetting mdIntpSetting = null;
-      for (InterpreterSetting setting : note.getBindedInterpreterSettings(new ArrayList<>())) {
+      for (InterpreterSetting setting : settings) {
         if (setting.getName().equals("md")) {
           mdIntpSetting = setting;
           break;
         }
       }
 
-      String jsonRequest = "{\"noteId\":\"" + note.getId() + "\"}";
+      String jsonRequest = "{\"noteId\":\"" + noteId + "\"}";
 
       // Restart isolated mode of Interpreter for note.
       mdIntpSetting.getOption().setPerNote(InterpreterOption.ISOLATED);
@@ -394,8 +456,8 @@ public class InterpreterRestApiTest extends AbstractTestRestApi {
       put.close();
 
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
diff --git a/zeppelin-server/src/test/java/org/apache/zeppelin/rest/NotebookRestApiTest.java b/zeppelin-server/src/test/java/org/apache/zeppelin/rest/NotebookRestApiTest.java
index 9fc51a9..9d2401c 100644
--- a/zeppelin-server/src/test/java/org/apache/zeppelin/rest/NotebookRestApiTest.java
+++ b/zeppelin-server/src/test/java/org/apache/zeppelin/rest/NotebookRestApiTest.java
@@ -17,6 +17,7 @@
 package org.apache.zeppelin.rest;
 
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
@@ -81,12 +82,17 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
   @Test
   public void testGetReloadNote() throws IOException {
     LOG.info("Running testGetNote");
-    Note note1 = null;
+    String note1Id = null;
     try {
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      TestUtils.getInstance(Notebook.class).saveNote(note1, anonymous);
-      CloseableHttpResponse get = httpGet("/notebook/" + note1.getId());
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          TestUtils.getInstance(Notebook.class).saveNote(note1, anonymous);
+          return null;
+        });
+
+      CloseableHttpResponse get = httpGet("/notebook/" + note1Id);
       assertThat(get, isAllowed());
       Map<String, Object> resp = gson.fromJson(EntityUtils.toString(get.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
@@ -94,9 +100,13 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       assertEquals(1, ((List)noteObject.get("paragraphs")).size());
 
       // add one new paragraph, but don't save it and reload it again
-      note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          return null;
+        });
 
-      get = httpGet("/notebook/" + note1.getId() + "?reload=true");
+      get = httpGet("/notebook/" + note1Id + "?reload=true");
       assertThat(get, isAllowed());
       resp = gson.fromJson(EntityUtils.toString(get.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
@@ -105,8 +115,8 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       get.close();
     } finally {
       // cleanup
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
@@ -114,14 +124,15 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
   @Test
   public void testGetNoteParagraphJobStatus() throws IOException {
     LOG.info("Running testGetNoteParagraphJobStatus");
-    Note note1 = null;
+    String note1Id = null;
     try {
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      String paragraphId = TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          return note1.addNewParagraph(AuthenticationInfo.ANONYMOUS).getId();
+        });
 
-      String paragraphId = note1.getLastParagraph().getId();
-
-      CloseableHttpResponse get = httpGet("/notebook/job/" + note1.getId() + "/" + paragraphId);
+      CloseableHttpResponse get = httpGet("/notebook/job/" + note1Id + "/" + paragraphId);
       assertThat(get, isAllowed());
       Map<String, Object> resp = gson.fromJson(EntityUtils.toString(get.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
@@ -133,8 +144,8 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       get.close();
     } finally {
       // cleanup
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
@@ -142,15 +153,17 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
   @Test
   public void testRunParagraphJob() throws Exception {
     LOG.info("Running testRunParagraphJob");
-    Note note1 = null;
+    String note1Id = null;
     try {
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      Paragraph p = TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          return note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+        });
 
-      Paragraph p = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
 
       // run blank paragraph
-      CloseableHttpResponse post = httpPost("/notebook/job/" + note1.getId() + "/" + p.getId(), "");
+      CloseableHttpResponse post = httpPost("/notebook/job/" + note1Id + "/" + p.getId(), "");
       assertThat(post, isAllowed());
       Map<String, Object> resp = gson.fromJson(EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
@@ -161,7 +174,7 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
 
       // run non-blank paragraph
       p.setText("test");
-      post = httpPost("/notebook/job/" + note1.getId() + "/" + p.getId(), "");
+      post = httpPost("/notebook/job/" + note1Id + "/" + p.getId(), "");
       assertThat(post, isAllowed());
       resp = gson.fromJson(EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
@@ -171,8 +184,8 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       assertNotEquals(Job.Status.FINISHED, p.getStatus());
     } finally {
       // cleanup
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
@@ -180,12 +193,13 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
   @Test
   public void testRunParagraphSynchronously() throws IOException {
     LOG.info("Running testRunParagraphSynchronously");
-    Note note1 = null;
+    String note1Id = null;
     try {
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-
-      Paragraph p = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      Paragraph p = TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          return note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+        });
 
       // run non-blank paragraph
       String title = "title";
@@ -193,7 +207,7 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       p.setTitle(title);
       p.setText(text);
 
-      CloseableHttpResponse post = httpPost("/notebook/run/" + note1.getId() + "/" + p.getId(), "");
+      CloseableHttpResponse post = httpPost("/notebook/run/" + note1Id + "/" + p.getId(), "");
       assertThat(post, isAllowed());
       Map<String, Object> resp = gson.fromJson(EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8),
           new TypeToken<Map<String, Object>>() {}.getType());
@@ -210,7 +224,7 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       p.setTitle(title);
       p.setText(text);
 
-      post = httpPost("/notebook/run/" + note1.getId() + "/" + p.getId(), "");
+      post = httpPost("/notebook/run/" + note1Id + "/" + p.getId(), "");
       assertEquals(200, post.getStatusLine().getStatusCode());
       resp = gson.fromJson(EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
@@ -228,8 +242,8 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       assertEquals(text, p.getText());
     } finally {
       // cleanup
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
@@ -245,12 +259,16 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
             new TypeToken<Map<String, Object>>() {}.getType());
     assertEquals("OK", resp1.get("status"));
 
-    String noteId1 = (String) resp1.get("body");
-    Note note1 = TestUtils.getInstance(Notebook.class).getNote(noteId1);
-    assertEquals("test1", note1.getName());
-    assertEquals(1, note1.getParagraphCount());
-    assertNull(note1.getParagraph(0).getText());
-    assertNull(note1.getParagraph(0).getTitle());
+    String note1Id = (String) resp1.get("body");
+    TestUtils.getInstance(Notebook.class).processNote(note1Id,
+      note1 -> {
+        assertEquals("test1", note1.getName());
+        assertEquals(1, note1.getParagraphCount());
+        assertNull(note1.getParagraph(0).getText());
+        assertNull(note1.getParagraph(0).getTitle());
+        return null;
+      });
+
 
     String message2 = "{\n\t\"name\" : \"test2\"\n}";
     CloseableHttpResponse post2 = httpPost("/notebook/", message2);
@@ -261,7 +279,10 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
     assertEquals("OK", resp2.get("status"));
 
     String noteId2 = (String) resp2.get("body");
-    Note note2 = TestUtils.getInstance(Notebook.class).getNote(noteId2);
+    Note note2 = TestUtils.getInstance(Notebook.class).processNote(noteId2,
+      note -> {
+        return note;
+      });
     assertEquals("test2", note2.getName());
     assertEquals(0, note2.getParagraphCount());
   }
@@ -269,9 +290,9 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
   @Test
   public void testRunNoteBlocking() throws IOException {
     LOG.info("Running testRunNoteBlocking");
-    Note note1 = null;
+    String note1Id = null;
     try {
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
       // 2 paragraphs
       // P1:
       //    %python
@@ -283,25 +304,35 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       //    %python
       //    print(user)
       //
-      Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      Paragraph p2 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p1.setText("%python from __future__ import print_function\nimport time\ntime.sleep(1)\nuser='abc'");
-      p2.setText("%python print(user)");
-
-      CloseableHttpResponse post = httpPost("/notebook/job/" + note1.getId() + "?blocking=true", "");
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          Paragraph p2 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p1.setText("%python from __future__ import print_function\nimport time\ntime.sleep(1)\nuser='abc'");
+          p2.setText("%python print(user)");
+          return null;
+        });
+
+      CloseableHttpResponse post = httpPost("/notebook/job/" + note1Id + "?blocking=true", "");
       assertThat(post, isAllowed());
       Map<String, Object> resp = gson.fromJson(EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
       assertEquals("OK", resp.get("status"));
       post.close();
 
-      assertEquals(Job.Status.FINISHED, p1.getStatus());
-      assertEquals(Job.Status.FINISHED, p2.getStatus());
-      assertEquals("abc\n", p2.getReturn().message().get(0).getData());
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          Paragraph p2 = note1.getParagraph(1);
+          assertEquals(Job.Status.FINISHED, p1.getStatus());
+          assertEquals(Job.Status.FINISHED, p2.getStatus());
+          assertEquals("abc\n", p2.getReturn().message().get(0).getData());
+          return null;
+        });
     } finally {
       // cleanup
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
@@ -309,9 +340,9 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
   @Test
   public void testRunNoteNonBlocking() throws Exception {
     LOG.info("Running testRunNoteNonBlocking");
-    Note note1 = null;
+    String note1Id = null;
     try {
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
       // 2 paragraphs
       // P1:
       //    %python
@@ -323,28 +354,41 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       //    %%sh(interpolate=true)
       //    echo '{name}'
       //
-      Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      Paragraph p2 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p1.setText("%python import time\ntime.sleep(5)\nname='hello'\nz.put('name', name)");
-      p2.setText("%sh(interpolate=true) echo '{name}'");
-
-      CloseableHttpResponse post = httpPost("/notebook/job/" + note1.getId() + "?blocking=true", "");
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          Paragraph p2 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p1.setText("%python import time\ntime.sleep(5)\nname='hello'\nz.put('name', name)");
+          p2.setText("%sh(interpolate=true) echo '{name}'");
+          return null;
+        });
+
+      CloseableHttpResponse post = httpPost("/notebook/job/" + note1Id + "?blocking=true", "");
       assertThat(post, isAllowed());
       Map<String, Object> resp = gson.fromJson(EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
       assertEquals("OK", resp.get("status"));
       post.close();
 
-      p1.waitUntilFinished();
-      p2.waitUntilFinished();
-
-      assertEquals(Job.Status.FINISHED, p1.getStatus());
-      assertEquals(Job.Status.FINISHED, p2.getStatus());
-      assertEquals("hello\n", p2.getReturn().message().get(0).getData());
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          Paragraph p2 = note1.getParagraph(1);
+          try {
+            p1.waitUntilFinished();
+            p2.waitUntilFinished();
+          } catch (InterruptedException e) {
+            fail();
+          }
+          assertEquals(Job.Status.FINISHED, p1.getStatus());
+          assertEquals(Job.Status.FINISHED, p2.getStatus());
+          assertEquals("hello\n", p2.getReturn().message().get(0).getData());
+          return null;
+        });
     } finally {
       // cleanup
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
@@ -352,14 +396,14 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
   @Test
   public void testRunNoteBlocking_Isolated() throws IOException {
     LOG.info("Running testRunNoteBlocking_Isolated");
-    Note note1 = null;
+    String note1Id = null;
     try {
       InterpreterSettingManager interpreterSettingManager =
               TestUtils.getInstance(InterpreterSettingManager.class);
       InterpreterSetting interpreterSetting = interpreterSettingManager.getInterpreterSettingByName("python");
       int pythonProcessNum = interpreterSetting.getAllInterpreterGroups().size();
 
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
       // 2 paragraphs
       // P1:
       //    %python
@@ -371,28 +415,37 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       //    %python
       //    print(user)
       //
-      Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      Paragraph p2 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p1.setText("%python from __future__ import print_function\nimport time\ntime.sleep(1)\nuser='abc'");
-      p2.setText("%python print(user)");
-
-      CloseableHttpResponse post = httpPost("/notebook/job/" + note1.getId() + "?blocking=true&isolated=true", "");
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          Paragraph p2 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p1.setText("%python from __future__ import print_function\nimport time\ntime.sleep(1)\nuser='abc'");
+          p2.setText("%python print(user)");
+          return null;
+        });
+
+      CloseableHttpResponse post = httpPost("/notebook/job/" + note1Id + "?blocking=true&isolated=true", "");
       assertThat(post, isAllowed());
       Map<String, Object> resp = gson.fromJson(EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
       assertEquals("OK", resp.get("status"));
       post.close();
 
-      assertEquals(Job.Status.FINISHED, p1.getStatus());
-      assertEquals(Job.Status.FINISHED, p2.getStatus());
-      assertEquals("abc\n", p2.getReturn().message().get(0).getData());
-
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          Paragraph p2 = note1.getParagraph(1);
+          assertEquals(Job.Status.FINISHED, p1.getStatus());
+          assertEquals(Job.Status.FINISHED, p2.getStatus());
+          assertEquals("abc\n", p2.getReturn().message().get(0).getData());
+          return null;
+        });
       // no new python process is created because it is isolated mode.
       assertEquals(pythonProcessNum, interpreterSetting.getAllInterpreterGroups().size());
     } finally {
       // cleanup
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
@@ -400,14 +453,14 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
   @Test
   public void testRunNoteNonBlocking_Isolated() throws IOException, InterruptedException {
     LOG.info("Running testRunNoteNonBlocking_Isolated");
-    Note note1 = null;
+    String note1Id = null;
     try {
       InterpreterSettingManager interpreterSettingManager =
               TestUtils.getInstance(InterpreterSettingManager.class);
       InterpreterSetting interpreterSetting = interpreterSettingManager.getInterpreterSettingByName("python");
       int pythonProcessNum = interpreterSetting.getAllInterpreterGroups().size();
 
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
       // 2 paragraphs
       // P1:
       //    %python
@@ -419,12 +472,17 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       //    %python
       //    print(user)
       //
-      Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      Paragraph p2 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p1.setText("%python from __future__ import print_function\nimport time\ntime.sleep(1)\nuser='abc'");
-      p2.setText("%python print(user)");
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          Paragraph p2 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p1.setText("%python from __future__ import print_function\nimport time\ntime.sleep(1)\nuser='abc'");
+          p2.setText("%python print(user)");
+          return null;
+        });
 
-      CloseableHttpResponse post = httpPost("/notebook/job/" + note1.getId() + "?blocking=false&isolated=true", "");
+
+      CloseableHttpResponse post = httpPost("/notebook/job/" + note1Id + "?blocking=false&isolated=true", "");
       assertThat(post, isAllowed());
       Map<String, Object> resp = gson.fromJson(EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
@@ -432,28 +490,36 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       post.close();
 
       // wait for all the paragraphs are done
-      while(note1.isRunning()) {
+      boolean isRunning = TestUtils.getInstance(Notebook.class).processNote(note1Id, Note::isRunning);
+      while(isRunning) {
         Thread.sleep(1000);
+        isRunning = TestUtils.getInstance(Notebook.class).processNote(note1Id, Note::isRunning);
       }
-      assertEquals(Job.Status.FINISHED, p1.getStatus());
-      assertEquals(Job.Status.FINISHED, p2.getStatus());
-      assertEquals("abc\n", p2.getReturn().message().get(0).getData());
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          Paragraph p2 = note1.getParagraph(1);
+          assertEquals(Job.Status.FINISHED, p1.getStatus());
+          assertEquals(Job.Status.FINISHED, p2.getStatus());
+          assertEquals("abc\n", p2.getReturn().message().get(0).getData());
+          return null;
+        });
 
       // no new python process is created because it is isolated mode.
       assertEquals(pythonProcessNum, interpreterSetting.getAllInterpreterGroups().size());
     } finally {
       // cleanup
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
 
   @Test
   public void testRunNoteWithParams() throws IOException, InterruptedException {
-    Note note1 = null;
+    String note1Id = null;
     try {
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
       // 2 paragraphs
       // P1:
       //    %python
@@ -463,15 +529,19 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       //    %sh
       //    echo ${name|world}
       //
-      Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      Paragraph p2 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p1.setText("%python name = z.input('name', 'world')\nprint(name)");
-      p2.setText("%sh(form=simple) echo '${name=world}'");
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          Paragraph p2 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p1.setText("%python name = z.input('name', 'world')\nprint(name)");
+          p2.setText("%sh(form=simple) echo '${name=world}'");
+          return null;
+        });
 
       Map<String, Object> paramsMap = new HashMap<>();
       paramsMap.put("name", "zeppelin");
       ParametersRequest parametersRequest = new ParametersRequest(paramsMap);
-      CloseableHttpResponse post = httpPost("/notebook/job/" + note1.getId() + "?blocking=false&isolated=true&",
+      CloseableHttpResponse post = httpPost("/notebook/job/" + note1Id + "?blocking=false&isolated=true&",
               parametersRequest.toJson());
       assertThat(post, isAllowed());
       Map<String, Object> resp = gson.fromJson(EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8),
@@ -480,16 +550,24 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       post.close();
 
       // wait for all the paragraphs are done
-      while(note1.isRunning()) {
+      boolean isRunning = TestUtils.getInstance(Notebook.class).processNote(note1Id, Note::isRunning);
+      while(isRunning) {
         Thread.sleep(1000);
+        isRunning = TestUtils.getInstance(Notebook.class).processNote(note1Id, Note::isRunning);
       }
-      assertEquals(Job.Status.FINISHED, p1.getStatus());
-      assertEquals(Job.Status.FINISHED, p2.getStatus());
-      assertEquals("zeppelin\n", p1.getReturn().message().get(0).getData());
-      assertEquals("zeppelin\n", p2.getReturn().message().get(0).getData());
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          Paragraph p2 = note1.getParagraph(1);
+          assertEquals(Job.Status.FINISHED, p1.getStatus());
+          assertEquals(Job.Status.FINISHED, p2.getStatus());
+          assertEquals("zeppelin\n", p1.getReturn().message().get(0).getData());
+          assertEquals("zeppelin\n", p2.getReturn().message().get(0).getData());
+          return null;
+        });
 
       // another attempt rest api call without params
-      post = httpPost("/notebook/job/" + note1.getId() + "?blocking=false&isolated=true", "");
+      post = httpPost("/notebook/job/" + note1Id + "?blocking=false&isolated=true", "");
       assertThat(post, isAllowed());
       resp = gson.fromJson(EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
@@ -497,17 +575,25 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       post.close();
 
       // wait for all the paragraphs are done
-      while(note1.isRunning()) {
+      isRunning = TestUtils.getInstance(Notebook.class).processNote(note1Id, Note::isRunning);
+      while(isRunning) {
         Thread.sleep(1000);
+        isRunning = TestUtils.getInstance(Notebook.class).processNote(note1Id, Note::isRunning);
       }
-      assertEquals(Job.Status.FINISHED, p1.getStatus());
-      assertEquals(Job.Status.FINISHED, p2.getStatus());
-      assertEquals("world\n", p1.getReturn().message().get(0).getData());
-      assertEquals("world\n", p2.getReturn().message().get(0).getData());
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          Paragraph p2 = note1.getParagraph(1);
+          assertEquals(Job.Status.FINISHED, p1.getStatus());
+          assertEquals(Job.Status.FINISHED, p2.getStatus());
+          assertEquals("world\n", p1.getReturn().message().get(0).getData());
+          assertEquals("world\n", p2.getReturn().message().get(0).getData());
+          return null;
+        });
     } finally {
       // cleanup
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
@@ -515,9 +601,9 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
   @Test
   public void testRunAllParagraph_FirstFailed() throws IOException {
     LOG.info("Running testRunAllParagraph_FirstFailed");
-    Note note1 = null;
+    String note1Id = null;
     try {
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
       // 2 paragraphs
       // P1:
       //    %python
@@ -531,22 +617,33 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       //    user2='abc'
       //    print(user2)
       //
-      Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      Paragraph p2 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p1.setText("%python from __future__ import print_function\nimport time\ntime.sleep(1)\nprint(user2)");
-      p2.setText("%python user2='abc'\nprint(user2)");
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          Paragraph p2 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p1.setText("%python from __future__ import print_function\nimport time\ntime.sleep(1)\nprint(user2)");
+          p2.setText("%python user2='abc'\nprint(user2)");
+          return null;
+        });
 
-      CloseableHttpResponse post = httpPost("/notebook/job/" + note1.getId() + "?blocking=true", "");
-      assertThat(post, isAllowed());
 
-      assertEquals(Job.Status.ERROR, p1.getStatus());
-      // p2 will be skipped because p1 is failed.
-      assertEquals(Job.Status.READY, p2.getStatus());
+      CloseableHttpResponse post = httpPost("/notebook/job/" + note1Id + "?blocking=true", "");
+      assertThat(post, isAllowed());
       post.close();
+
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          Paragraph p2 = note1.getParagraph(1);
+          assertEquals(Job.Status.ERROR, p1.getStatus());
+          // p2 will be skipped because p1 is failed.
+          assertEquals(Job.Status.READY, p2.getStatus());
+          return null;
+        });
     } finally {
       // cleanup
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
@@ -554,11 +651,11 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
   @Test
   public void testCloneNote() throws IOException {
     LOG.info("Running testCloneNote");
-    Note note1 = null;
+    String note1Id = null;
     String clonedNoteId = null;
     try {
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      CloseableHttpResponse post = httpPost("/notebook/" + note1.getId(), "");
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      CloseableHttpResponse post = httpPost("/notebook/" + note1Id, "");
       String postResponse = EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8);
       LOG.info("testCloneNote response\n" + postResponse);
       assertThat(post, isAllowed());
@@ -577,14 +674,11 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       get.close();
     } finally {
       // cleanup
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
       if (null != clonedNoteId) {
-        Note clonedNote = TestUtils.getInstance(Notebook.class).getNote(clonedNoteId);
-        if (clonedNote != null) {
-          TestUtils.getInstance(Notebook.class).removeNote(clonedNote, anonymous);
-        }
+        TestUtils.getInstance(Notebook.class).removeNote(clonedNoteId, anonymous);
       }
     }
   }
@@ -592,12 +686,11 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
   @Test
   public void testRenameNote() throws IOException {
     LOG.info("Running testRenameNote");
-    Note note = null;
+    String noteId = null;
     try {
       String oldName = "old_name";
-      note = TestUtils.getInstance(Notebook.class).createNote(oldName, anonymous);
-      assertEquals(note.getName(), oldName);
-      String noteId = note.getId();
+      noteId = TestUtils.getInstance(Notebook.class).createNote(oldName, anonymous);
+      assertEquals(oldName, TestUtils.getInstance(Notebook.class).processNote(noteId, Note::getName));
 
       final String newName = "testName";
       String jsonRequest = "{\"name\": " + newName + "}";
@@ -606,11 +699,11 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       assertThat("test testRenameNote:", put, isAllowed());
       put.close();
 
-      assertEquals(note.getName(), newName);
+      assertEquals(newName, TestUtils.getInstance(Notebook.class).processNote(noteId, Note::getName));
     } finally {
       // cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
@@ -618,13 +711,16 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
   @Test
   public void testUpdateParagraphConfig() throws IOException {
     LOG.info("Running testUpdateParagraphConfig");
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      String noteId = note.getId();
-      Paragraph p = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      assertNull(p.getConfig().get("colWidth"));
-      String paragraphId = p.getId();
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      String paragraphId = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          assertNull(p.getConfig().get("colWidth"));
+          return p.getId();
+        });
+
       String jsonRequest = "{\"colWidth\": 6.0}";
 
       CloseableHttpResponse put = httpPut("/notebook/" + noteId + "/paragraph/" + paragraphId + "/config",
@@ -638,12 +734,15 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       put.close();
 
       assertEquals(config.get("colWidth"), 6.0);
-      note = TestUtils.getInstance(Notebook.class).getNote(noteId);
-      assertEquals(note.getParagraph(paragraphId).getConfig().get("colWidth"), 6.0);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          assertEquals(note.getParagraph(paragraphId).getConfig().get("colWidth"), 6.0);
+          return null;
+        });
     } finally {
       // cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
@@ -651,26 +750,36 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
   @Test
   public void testClearAllParagraphOutput() throws IOException {
     LOG.info("Running testClearAllParagraphOutput");
-    Note note = null;
+    String noteId = null;
     try {
       // Create note and set result explicitly
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      Paragraph p1 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      InterpreterResult result = new InterpreterResult(InterpreterResult.Code.SUCCESS,
-              InterpreterResult.Type.TEXT, "result");
-      p1.setResult(result);
-
-      Paragraph p2 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p2.setReturn(result, new Throwable());
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      String p1Id = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p1 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          InterpreterResult result = new InterpreterResult(InterpreterResult.Code.SUCCESS,
+                  InterpreterResult.Type.TEXT, "result");
+          p1.setResult(result);
+          return p1.getId();
+        });
+
+      String p2Id = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p2 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          InterpreterResult result = new InterpreterResult(InterpreterResult.Code.SUCCESS,
+                  InterpreterResult.Type.TEXT, "result");
+          p2.setReturn(result, new Throwable());
+          return p2.getId();
+        });
 
       // clear paragraph result
-      CloseableHttpResponse put = httpPut("/notebook/" + note.getId() + "/clear", "");
+      CloseableHttpResponse put = httpPut("/notebook/" + noteId + "/clear", "");
       LOG.info("test clear paragraph output response\n" + EntityUtils.toString(put.getEntity(), StandardCharsets.UTF_8));
       assertThat(put, isAllowed());
       put.close();
 
       // check if paragraph results are cleared
-      CloseableHttpResponse get = httpGet("/notebook/" + note.getId() + "/paragraph/" + p1.getId());
+      CloseableHttpResponse get = httpGet("/notebook/" + noteId + "/paragraph/" + p1Id);
       assertThat(get, isAllowed());
       Map<String, Object> resp1 = gson.fromJson(EntityUtils.toString(get.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
@@ -678,7 +787,7 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       assertNull(resp1Body.get("result"));
       get.close();
 
-      get = httpGet("/notebook/" + note.getId() + "/paragraph/" + p2.getId());
+      get = httpGet("/notebook/" + noteId + "/paragraph/" + p2Id);
       assertThat(get, isAllowed());
       Map<String, Object> resp2 = gson.fromJson(EntityUtils.toString(get.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
@@ -687,8 +796,8 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       get.close();
     } finally {
       // cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
@@ -696,9 +805,9 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
   @Test
   public void testRunWithServerRestart() throws Exception {
     LOG.info("Running testRunWithServerRestart");
-    Note note1 = null;
+    String note1Id = null;
     try {
-      note1 = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      note1Id = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
       // 2 paragraphs
       // P1:
       //    %python
@@ -710,15 +819,20 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       //    %python
       //    print(user)
       //
-      Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      Paragraph p2 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p1.setText("%python from __future__ import print_function\nimport time\ntime.sleep(1)\nuser='abc'");
-      p2.setText("%python print(user)");
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          Paragraph p2 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p1.setText("%python from __future__ import print_function\nimport time\ntime.sleep(1)\nuser='abc'");
+          p2.setText("%python print(user)");
+          return null;
+        });
 
-      CloseableHttpResponse post1 = httpPost("/notebook/job/" + note1.getId() + "?blocking=true", "");
+
+      CloseableHttpResponse post1 = httpPost("/notebook/job/" + note1Id + "?blocking=true", "");
       assertThat(post1, isAllowed());
       post1.close();
-      CloseableHttpResponse put = httpPut("/notebook/" + note1.getId() + "/clear", "");
+      CloseableHttpResponse put = httpPut("/notebook/" + note1Id + "/clear", "");
       LOG.info("test clear paragraph output response\n" + EntityUtils.toString(put.getEntity(), StandardCharsets.UTF_8));
       assertThat(put, isAllowed());
       put.close();
@@ -727,25 +841,27 @@ public class NotebookRestApiTest extends AbstractTestRestApi {
       AbstractTestRestApi.shutDown(false);
       startUp(NotebookRestApiTest.class.getSimpleName(), false);
 
-      note1 = TestUtils.getInstance(Notebook.class).getNote(note1.getId());
-      p1 = note1.getParagraph(p1.getId());
-      p2 = note1.getParagraph(p2.getId());
-
-      CloseableHttpResponse post2 = httpPost("/notebook/job/" + note1.getId() + "?blocking=true", "");
+      CloseableHttpResponse post2 = httpPost("/notebook/job/" + note1Id + "?blocking=true", "");
       assertThat(post2, isAllowed());
       Map<String, Object> resp = gson.fromJson(EntityUtils.toString(post2.getEntity(), StandardCharsets.UTF_8),
           new TypeToken<Map<String, Object>>() {}.getType());
       assertEquals("OK", resp.get("status"));
       post2.close();
 
-      assertEquals(Job.Status.FINISHED, p1.getStatus());
-      assertEquals(p2.getReturn().toString(), Job.Status.FINISHED, p2.getStatus());
-      assertNotNull(p2.getReturn());
-      assertEquals("abc\n", p2.getReturn().message().get(0).getData());
+      TestUtils.getInstance(Notebook.class).processNote(note1Id,
+        note1 -> {
+          Paragraph p1 = note1.getParagraph(0);
+          Paragraph p2 = note1.getParagraph(1);
+          assertEquals(Job.Status.FINISHED, p1.getStatus());
+          assertEquals(p2.getReturn().toString(), Job.Status.FINISHED, p2.getStatus());
+          assertNotNull(p2.getReturn());
+          assertEquals("abc\n", p2.getReturn().message().get(0).getData());
+          return null;
+        });
     } finally {
       // cleanup
-      if (null != note1) {
-        TestUtils.getInstance(Notebook.class).removeNote(note1, anonymous);
+      if (null != note1Id) {
+        TestUtils.getInstance(Notebook.class).removeNote(note1Id, anonymous);
       }
     }
   }
diff --git a/zeppelin-server/src/test/java/org/apache/zeppelin/rest/NotebookSecurityRestApiTest.java b/zeppelin-server/src/test/java/org/apache/zeppelin/rest/NotebookSecurityRestApiTest.java
index 84b7c35..6e4b0c2 100644
--- a/zeppelin-server/src/test/java/org/apache/zeppelin/rest/NotebookSecurityRestApiTest.java
+++ b/zeppelin-server/src/test/java/org/apache/zeppelin/rest/NotebookSecurityRestApiTest.java
@@ -105,8 +105,11 @@ public class NotebookSecurityRestApiTest extends AbstractTestRestApi {
     userTryRemoveNote(noteId, "user2", "password3", isForbidden());
     userTryRemoveNote(noteId, "user1", "password2", isAllowed());
 
-    Note deletedNote = TestUtils.getInstance(Notebook.class).getNote(noteId);
-    assertNull("Deleted note should be null", deletedNote);
+    TestUtils.getInstance(Notebook.class).processNote(noteId,
+      deletedNote -> {
+        assertNull("Deleted note should be null", deletedNote);
+        return null;
+      });
   }
 
   private void userTryRemoveNote(String noteId, String user, String pwd,
@@ -142,8 +145,11 @@ public class NotebookSecurityRestApiTest extends AbstractTestRestApi {
     post.close();
     String newNoteId = (String) resp.get("body");
     Notebook notebook = TestUtils.getInstance(Notebook.class);
-    Note newNote = notebook.getNote(newNoteId);
-    assertNotNull("Can not find new note by id", newNote);
+    notebook.processNote(newNoteId,
+      newNote -> {
+        assertNotNull("Can not find new note by id", newNote);
+        return null;
+      });
     return newNoteId;
   }
 
@@ -153,8 +159,11 @@ public class NotebookSecurityRestApiTest extends AbstractTestRestApi {
     delete.close();
     // make sure note is deleted
     if (!noteId.isEmpty()) {
-      Note deletedNote = TestUtils.getInstance(Notebook.class).getNote(noteId);
-      assertNull("Deleted note should be null", deletedNote);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        deletedNote -> {
+          assertNull("Deleted note should be null", deletedNote);
+          return null;
+        });
     }
   }
 
diff --git a/zeppelin-server/src/test/java/org/apache/zeppelin/rest/ZeppelinRestApiTest.java b/zeppelin-server/src/test/java/org/apache/zeppelin/rest/ZeppelinRestApiTest.java
index aa396e0..34f2bea 100644
--- a/zeppelin-server/src/test/java/org/apache/zeppelin/rest/ZeppelinRestApiTest.java
+++ b/zeppelin-server/src/test/java/org/apache/zeppelin/rest/ZeppelinRestApiTest.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import com.google.gson.Gson;
 import com.google.gson.reflect.TypeToken;
@@ -91,22 +92,27 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
   @Test
   public void testGetNoteInfo() throws IOException {
     LOG.info("testGetNoteInfo");
-    Note note = null;
+    String noteId = null;
     try {
       // Create note to get info
-      note = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
-      assertNotNull("can't create new note", note);
-      note.setName("note");
-      Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      Map<String, Object> config = paragraph.getConfig();
-      config.put("enabled", true);
-      paragraph.setConfig(config);
-      String paragraphText = "%md This is my new paragraph in my new note";
-      paragraph.setText(paragraphText);
-      TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
-
-      String sourceNoteId = note.getId();
-      CloseableHttpResponse get = httpGet("/notebook/" + sourceNoteId);
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1", anonymous);
+      assertNotNull("can't create new note", noteId);
+      // use write lock because name is overwritten
+      String paragraphText = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          note.setName("note");
+          Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          Map<String, Object> config = paragraph.getConfig();
+          config.put("enabled", true);
+          paragraph.setConfig(config);
+          String paragraphTextTmp = "%md This is my new paragraph in my new note";
+          paragraph.setText(paragraphTextTmp);
+          TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
+          return paragraphTextTmp;
+        });
+
+
+      CloseableHttpResponse get = httpGet("/notebook/" + noteId);
       String getResponse = EntityUtils.toString(get.getEntity(), StandardCharsets.UTF_8);
       LOG.info("testGetNoteInfo \n" + getResponse);
       assertThat("test note get method:", get, isAllowed());
@@ -124,8 +130,8 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
       assertEquals(paragraphText, paragraphs.get(0).get("text"));
       get.close();
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
@@ -162,33 +168,37 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
 
     String newNoteId =  (String) resp.get("body");
     LOG.info("newNoteId:=" + newNoteId);
-    Note newNote = TestUtils.getInstance(Notebook.class).getNote(newNoteId);
-    assertNotNull("Can not find new note by id", newNote);
-    // This is partial test as newNote is in memory but is not persistent
-    String newNoteName = newNote.getName();
-    LOG.info("new note name is: " + newNoteName);
-    String expectedNoteName = noteName;
-    if (noteName.isEmpty()) {
-      expectedNoteName = "Note " + newNoteId;
-    }
-    assertEquals("compare note name", expectedNoteName, newNoteName);
-    assertEquals("initial paragraph check failed", 3, newNote.getParagraphs().size());
-    for (Paragraph p : newNote.getParagraphs()) {
-      if (StringUtils.isEmpty(p.getText())) {
-        continue;
-      }
-      assertTrue("paragraph title check failed", p.getTitle().startsWith("title"));
-      assertTrue("paragraph text check failed", p.getText().startsWith("text"));
-      if (p.getTitle().equals("titleConfig")) {
-        assertEquals("paragraph col width check failed", 9.0, p.getConfig().get("colWidth"));
-        assertTrue("paragraph show title check failed", ((boolean) p.getConfig().get("title")));
-        Map graph = ((List<Map>) p.getConfig().get("results")).get(0);
-        String mode = ((Map) graph.get("graph")).get("mode").toString();
-        assertEquals("paragraph graph mode check failed", "pieChart", mode);
-      }
-    }
+    TestUtils.getInstance(Notebook.class).processNote(newNoteId,
+      newNote -> {
+        assertNotNull("Can not find new note by id", newNote);
+        // This is partial test as newNote is in memory but is not persistent
+        String newNoteName = newNote.getName();
+        LOG.info("new note name is: " + newNoteName);
+        String expectedNoteName = noteName;
+        if (noteName.isEmpty()) {
+          expectedNoteName = "Note " + newNoteId;
+        }
+        assertEquals("compare note name", expectedNoteName, newNoteName);
+        assertEquals("initial paragraph check failed", 3, newNote.getParagraphs().size());
+        for (Paragraph p : newNote.getParagraphs()) {
+          if (StringUtils.isEmpty(p.getText())) {
+            continue;
+          }
+          assertTrue("paragraph title check failed", p.getTitle().startsWith("title"));
+          assertTrue("paragraph text check failed", p.getText().startsWith("text"));
+          if (p.getTitle().equals("titleConfig")) {
+            assertEquals("paragraph col width check failed", 9.0, p.getConfig().get("colWidth"));
+            assertTrue("paragraph show title check failed", ((boolean) p.getConfig().get("title")));
+            Map graph = ((List<Map>) p.getConfig().get("results")).get(0);
+            String mode = ((Map) graph.get("graph")).get("mode").toString();
+            assertEquals("paragraph graph mode check failed", "pieChart", mode);
+          }
+        }
+        return null;
+      });
+
     // cleanup
-    TestUtils.getInstance(Notebook.class).removeNote(newNote, anonymous);
+    TestUtils.getInstance(Notebook.class).removeNote(newNoteId, anonymous);
     post.close();
   }
 
@@ -205,34 +215,35 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
 
     String newNoteId =  (String) resp.get("body");
     LOG.info("newNoteId:=" + newNoteId);
-    Note newNote = TestUtils.getInstance(Notebook.class).getNote(newNoteId);
-    assertNotNull("Can not find new note by id", newNote);
-    // This is partial test as newNote is in memory but is not persistent
-    String newNoteName = newNote.getName();
-    LOG.info("new note name is: " + newNoteName);
-    if (StringUtils.isBlank(noteName)) {
-      noteName = "Untitled Note";
-    }
-    assertEquals("compare note name", noteName, newNoteName);
+    TestUtils.getInstance(Notebook.class).processNote(newNoteId,
+      newNote -> {
+        assertNotNull("Can not find new note by id", newNote);
+        // This is partial test as newNote is in memory but is not persistent
+        String newNoteName = newNote.getName();
+        LOG.info("new note name is: " + newNoteName);
+        String noteNameTmp = noteName;
+        if (StringUtils.isBlank(noteNameTmp)) {
+          noteNameTmp = "Untitled Note";
+        }
+        assertEquals("compare note name", noteNameTmp, newNoteName);
+        return null;
+      });
     // cleanup
-    TestUtils.getInstance(Notebook.class).removeNote(newNote, anonymous);
+    TestUtils.getInstance(Notebook.class).removeNote(newNoteId, anonymous);
     post.close();
   }
 
   @Test
   public void testDeleteNote() throws IOException {
     LOG.info("testDeleteNote");
-    Note note = null;
+    String noteId = null;
     try {
       //Create note and get ID
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testDeletedNote", anonymous);
-      String noteId = note.getId();
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testDeletedNote", anonymous);
       testDeleteNote(noteId);
     } finally {
-      if (null != note) {
-        if (TestUtils.getInstance(Notebook.class).getNote(note.getId()) != null) {
-          TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
-        }
+      if (noteId != null) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
@@ -247,20 +258,25 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
   public void testExportNote() throws IOException {
     LOG.info("testExportNote");
 
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testExportNote", anonymous);
-      assertNotNull("can't create new note", note);
-      note.setName("source note for export");
-      Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      Map<String, Object> config = paragraph.getConfig();
-      config.put("enabled", true);
-      paragraph.setConfig(config);
-      paragraph.setText("%md This is my new paragraph in my new note");
-      TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
-      String sourceNoteId = note.getId();
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testExportNote", anonymous);
+      // use write lock because name is overwritten
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          assertNotNull("can't create new note", note);
+          note.setName("source note for export");
+          Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          Map<String, Object> config = paragraph.getConfig();
+          config.put("enabled", true);
+          paragraph.setConfig(config);
+          paragraph.setText("%md This is my new paragraph in my new note");
+          TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
+          return null;
+        });
+
       // Call export Note REST API
-      CloseableHttpResponse get = httpGet("/notebook/export/" + sourceNoteId);
+      CloseableHttpResponse get = httpGet("/notebook/export/" + noteId);
       String getResponse = EntityUtils.toString(get.getEntity(), StandardCharsets.UTF_8);
       LOG.info("testNoteExport \n" + getResponse);
       assertThat("test note export method:", get, isAllowed());
@@ -274,37 +290,42 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
       LOG.info("export JSON:=" + exportJSON);
       get.close();
     } finally {
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testImportNotebook() throws IOException {
-    Note note = null;
-    Note newNote = null;
+    String noteId = null;
     Map<String, Object> resp;
     String oldJson;
     String noteName;
+    String importId = null;
     try {
       noteName = "source note for import";
       LOG.info("testImportNote");
       // create test note
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testImportNotebook", anonymous);
-      assertNotNull("can't create new note", note);
-      note.setName(noteName);
-      Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      Map<String, Object> config = paragraph.getConfig();
-      config.put("enabled", true);
-      paragraph.setConfig(config);
-      paragraph.setText("%md This is my new paragraph in my new note");
-      TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
-      String sourceNoteId = note.getId();
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testImportNotebook", anonymous);
+      // use write lock because name is overwritten
+      int paragraphSize = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          assertNotNull("can't create new note", note);
+          note.setName(noteName);
+          Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          Map<String, Object> config = paragraph.getConfig();
+          config.put("enabled", true);
+          paragraph.setConfig(config);
+          paragraph.setText("%md This is my new paragraph in my new note");
+          TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
+          return note.getParagraphs().size();
+        });
+
       // get note content as JSON
-      oldJson = getNoteContent(sourceNoteId);
+      oldJson = getNoteContent(noteId);
       // delete it first then import it
-      TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
 
       // call note post
       CloseableHttpResponse importPost = httpPost("/notebook/import/", oldJson);
@@ -312,25 +333,21 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
       resp =
           gson.fromJson(EntityUtils.toString(importPost.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
-      String importId = (String) resp.get("body");
+      importId = (String) resp.get("body");
 
       assertNotNull("Did not get back a note id in body", importId);
-      newNote = TestUtils.getInstance(Notebook.class).getNote(importId);
-      assertEquals("Compare note names", noteName, newNote.getName());
-      assertEquals("Compare paragraphs count", note.getParagraphs().size(), newNote.getParagraphs()
-          .size());
+      TestUtils.getInstance(Notebook.class).processNote(importId,
+        newNote -> {
+          assertEquals("Compare note names", noteName, newNote.getName());
+          assertEquals("Compare paragraphs count", paragraphSize, newNote.getParagraphs().size());
+          return null;
+      });
       importPost.close();
     } finally {
-      if (null != note) {
-        if (TestUtils.getInstance(Notebook.class).getNote(note.getId()) != null) {
-          TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
-        }
-      }
-      if (null != newNote) {
-        if (TestUtils.getInstance(Notebook.class).getNote(newNote.getId()) != null) {
-          TestUtils.getInstance(Notebook.class).removeNote(newNote, anonymous);
-        }
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
+      TestUtils.getInstance(Notebook.class).removeNote(importId, anonymous);
     }
   }
 
@@ -354,8 +371,11 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
     delete.close();
     // make sure note is deleted
     if (!noteId.isEmpty()) {
-      Note deletedNote = TestUtils.getInstance(Notebook.class).getNote(noteId);
-      assertNull("Deleted note should be null", deletedNote);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        deletedNote -> {
+          assertNull("Deleted note should be null", deletedNote);
+          return null;
+        });
     }
   }
 
@@ -369,24 +389,28 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
   @Test
   public void testCloneNote() throws IOException, IllegalArgumentException {
     LOG.info("testCloneNote");
-    Note note = null, newNote = null;
+    String noteId = null;
+    String newNoteId = null;
     try {
       // Create note to clone
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testCloneNote", anonymous);
-      assertNotNull("can't create new note", note);
-      note.setName("source note for clone");
-      Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      Map<String, Object> config = paragraph.getConfig();
-      config.put("enabled", true);
-      paragraph.setConfig(config);
-      paragraph.setText("%md This is my new paragraph in my new note");
-      TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
-      String sourceNoteId = note.getId();
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testCloneNote", anonymous);
+      int paragraphSize = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          assertNotNull("can't create new note", note);
+          note.setName("source note for clone");
+          Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          Map<String, Object> config = paragraph.getConfig();
+          config.put("enabled", true);
+          paragraph.setConfig(config);
+          paragraph.setText("%md This is my new paragraph in my new note");
+          TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
+          return note.getParagraphs().size();
+        });
 
       String noteName = "clone Note Name";
       // Call Clone Note REST API
       String jsonRequest = "{\"name\":\"" + noteName + "\"}";
-      CloseableHttpResponse post = httpPost("/notebook/" + sourceNoteId, jsonRequest);
+      CloseableHttpResponse post = httpPost("/notebook/" + noteId, jsonRequest);
       String postResponse = EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8);
       LOG.info("testNoteClone \n" + postResponse);
       assertThat("test note clone method:", post, isAllowed());
@@ -394,22 +418,23 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
       Map<String, Object> resp = gson.fromJson(postResponse,
               new TypeToken<Map<String, Object>>() {}.getType());
 
-      String newNoteId =  (String) resp.get("body");
+      newNoteId =  (String) resp.get("body");
       LOG.info("newNoteId:=" + newNoteId);
-      newNote = TestUtils.getInstance(Notebook.class).getNote(newNoteId);
-      assertNotNull("Can not find new note by id", newNote);
-      assertEquals("Compare note names", noteName, newNote.getName());
-      assertEquals("Compare paragraphs count", note.getParagraphs().size(),
-              newNote.getParagraphs().size());
+      TestUtils.getInstance(Notebook.class).processNote(newNoteId,
+        newNote -> {
+          assertNotNull("Can not find new note by id", newNote);
+          assertEquals("Compare note names", noteName, newNote.getName());
+          assertEquals("Compare paragraphs count", paragraphSize, newNote.getParagraphs().size());
+          return null;
+        });
+
       post.close();
     } finally {
       //cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
-      }
-      if (null != newNote) {
-        TestUtils.getInstance(Notebook.class).removeNote(newNote, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
+      TestUtils.getInstance(Notebook.class).removeNote(newNoteId, anonymous);
     }
   }
 
@@ -425,7 +450,7 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
     HashSet<String> anonymous = new HashSet<>(Arrays.asList("anonymous"));
     AuthorizationService authorizationService = TestUtils.getInstance(AuthorizationService.class);
     assertEquals("List notes are equal", TestUtils.getInstance(Notebook.class)
-            .getAllNotes(note -> authorizationService.isReader(note.getId(), anonymous))
+            .getNotesInfo(noteId -> authorizationService.isReader(noteId, anonymous))
             .size(), body.size());
     get.close();
   }
@@ -434,27 +459,42 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
   public void testNoteJobs() throws Exception {
     LOG.info("testNoteJobs");
 
-    Note note = null;
+    String noteId = null;
     try {
       // Create note to run test.
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testNoteJobs", anonymous);
-      assertNotNull("can't create new note", note);
-      note.setName("note for run test");
-      Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testNoteJobs", anonymous);
+      // use write lock because name is overwritten
+      String paragraphId = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          assertNotNull("can't create new note", note);
+          note.setName("note for run test");
+          Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+
+          Map<String, Object> config = paragraph.getConfig();
+          config.put("enabled", true);
+          paragraph.setConfig(config);
+
+          paragraph.setText("%md This is test paragraph.");
+          TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
+          return paragraph.getId();
+        });
+
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          try {
+            note.runAll(anonymous, true, false, new HashMap<>());
+          } catch (Exception e) {
+            fail();
+          }
+          return null;
+        });
 
-      Map<String, Object> config = paragraph.getConfig();
-      config.put("enabled", true);
-      paragraph.setConfig(config);
-
-      paragraph.setText("%md This is test paragraph.");
-      TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
-      String noteId = note.getId();
-
-      note.runAll(anonymous, true, false, new HashMap<>());
       // wait until job is finished or timeout.
       int timeout = 1;
-      while (!paragraph.isTerminated()) {
+      boolean terminated = TestUtils.getInstance(Notebook.class).processNote(noteId, note -> note.getParagraph(0).isTerminated());
+      while (!terminated) {
         Thread.sleep(1000);
+        terminated = TestUtils.getInstance(Notebook.class).processNote(noteId, note -> note.getParagraph(0).isTerminated());
         if (timeout++ > 10) {
           LOG.info("testNoteJobs timeout job.");
           break;
@@ -473,20 +513,20 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
       Thread.sleep(1000);
 
       // Call Run paragraph REST API
-      CloseableHttpResponse postParagraph = httpPost("/notebook/job/" + noteId + "/" + paragraph.getId(), "");
+      CloseableHttpResponse postParagraph = httpPost("/notebook/job/" + noteId + "/" + paragraphId, "");
       assertThat("test paragraph run:", postParagraph, isAllowed());
       postParagraph.close();
       Thread.sleep(1000);
 
       // Call Stop paragraph REST API
-      CloseableHttpResponse deleteParagraph = httpDelete("/notebook/job/" + noteId + "/" + paragraph.getId());
+      CloseableHttpResponse deleteParagraph = httpDelete("/notebook/job/" + noteId + "/" + paragraphId);
       assertThat("test paragraph stop:", deleteParagraph, isAllowed());
       deleteParagraph.close();
       Thread.sleep(1000);
     } finally {
       //cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
@@ -495,24 +535,36 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
   public void testGetNoteJob() throws Exception {
     LOG.info("testGetNoteJob");
 
-    Note note = null;
+    String noteId = null;
     try {
       // Create note to run test.
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testGetNoteJob", anonymous);
-      assertNotNull("can't create new note", note);
-      note.setName("note for run test");
-      Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-
-      Map<String, Object> config = paragraph.getConfig();
-      config.put("enabled", true);
-      paragraph.setConfig(config);
-
-      paragraph.setText("%sh sleep 1");
-      paragraph.setAuthenticationInfo(anonymous);
-      TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
-      String noteId = note.getId();
-
-      note.runAll(anonymous, true, false, new HashMap<>());
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testGetNoteJob", anonymous);
+      // use write lock because name is overwritten
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          assertNotNull("can't create new note", note);
+          note.setName("note for run test");
+          Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+
+          Map<String, Object> config = paragraph.getConfig();
+          config.put("enabled", true);
+          paragraph.setConfig(config);
+
+          paragraph.setText("%sh sleep 1");
+          paragraph.setAuthenticationInfo(anonymous);
+          TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
+          return null;
+        });
+
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          try {
+            note.runAll(anonymous, true, false, new HashMap<>());
+          } catch (Exception e) {
+            fail();
+          }
+          return null;
+        });
       // assume that status of the paragraph is running
       CloseableHttpResponse get = httpGet("/notebook/job/" + noteId);
       assertThat("test get note job: ", get, isAllowed());
@@ -530,8 +582,10 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
 
       // wait until job is finished or timeout.
       int timeout = 1;
-      while (!paragraph.isTerminated()) {
+      boolean terminated = TestUtils.getInstance(Notebook.class).processNote(noteId, note -> note.getParagraph(0).isTerminated());
+      while (!terminated) {
         Thread.sleep(100);
+        terminated = TestUtils.getInstance(Notebook.class).processNote(noteId, note -> note.getParagraph(0).isTerminated());
         if (timeout++ > 10) {
           LOG.info("testGetNoteJob timeout job.");
           break;
@@ -539,8 +593,8 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
       }
     } finally {
       //cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
@@ -549,40 +603,56 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
   public void testRunParagraphWithParams() throws Exception {
     LOG.info("testRunParagraphWithParams");
 
-    Note note = null;
+    String noteId = null;
     try {
       // Create note to run test.
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testRunParagraphWithParams", anonymous);
-      assertNotNull("can't create new note", note);
-      note.setName("note for run test");
-      Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-
-      Map<String, Object> config = paragraph.getConfig();
-      config.put("enabled", true);
-      paragraph.setConfig(config);
-
-      paragraph.setText("%spark\nval param = z.input(\"param\").toString\nprintln(param)");
-      TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
-      String noteId = note.getId();
-
-      note.runAll(anonymous, true, false, new HashMap<>());
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testRunParagraphWithParams", anonymous);
+      // use write lock because name is overwritten
+      String paragraphId = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          assertNotNull("can't create new note", note);
+          note.setName("note for run test");
+          Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+
+          Map<String, Object> config = paragraph.getConfig();
+          config.put("enabled", true);
+          paragraph.setConfig(config);
+
+          paragraph.setText("%spark\nval param = z.input(\"param\").toString\nprintln(param)");
+          TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
+          return paragraph.getId();
+        });
+
+
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          try {
+            note.runAll(anonymous, true, false, new HashMap<>());
+          } catch (Exception e) {
+            fail();
+          }
+          return null;
+        });
 
       // Call Run paragraph REST API
-      CloseableHttpResponse postParagraph = httpPost("/notebook/job/" + noteId + "/" + paragraph.getId(),
+      CloseableHttpResponse postParagraph = httpPost("/notebook/job/" + noteId + "/" + paragraphId,
           "{\"params\": {\"param\": \"hello\", \"param2\": \"world\"}}");
       assertThat("test paragraph run:", postParagraph, isAllowed());
       postParagraph.close();
       Thread.sleep(1000);
 
-      Note retrNote = TestUtils.getInstance(Notebook.class).getNote(noteId);
-      Paragraph retrParagraph = retrNote.getParagraph(paragraph.getId());
-      Map<String, Object> params = retrParagraph.settings.getParams();
-      assertEquals("hello", params.get("param"));
-      assertEquals("world", params.get("param2"));
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        retrNote -> {
+          Paragraph retrParagraph = retrNote.getParagraph(paragraphId);
+          Map<String, Object> params = retrParagraph.settings.getParams();
+          assertEquals("hello", params.get("param"));
+          assertEquals("world", params.get("param2"));
+          return null;
+        });
     } finally {
       //cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
@@ -590,20 +660,32 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
   @Test
   public void testJobs() throws Exception {
     // create a note and a paragraph
-    Note note = null;
+    String noteId = null;
     try {
       System.setProperty(ConfVars.ZEPPELIN_NOTEBOOK_CRON_ENABLE.getVarName(), "true");
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testJobs", anonymous);
-
-      note.setName("note for run test");
-      Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      paragraph.setText("%md This is test paragraph.");
-
-      Map<String, Object> config = paragraph.getConfig();
-      config.put("enabled", true);
-      paragraph.setConfig(config);
-
-      note.runAll(AuthenticationInfo.ANONYMOUS, false, false, new HashMap<>());
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testJobs", anonymous);
+      // Use write lock, because name is overwritten
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          note.setName("note for run test");
+          Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          paragraph.setText("%md This is test paragraph.");
+
+          Map<String, Object> config = paragraph.getConfig();
+          config.put("enabled", true);
+          paragraph.setConfig(config);
+          return null;
+        });
+
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          try {
+            note.runAll(AuthenticationInfo.ANONYMOUS, false, false, new HashMap<>());
+          } catch (Exception e) {
+            fail();
+          }
+          return null;
+        });
 
       String jsonRequest = "{\"cron\":\"* * * * * ?\" }";
       // right cron expression but not exist note.
@@ -612,26 +694,26 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
       postCron.close();
 
       // right cron expression.
-      postCron = httpPost("/notebook/cron/" + note.getId(), jsonRequest);
+      postCron = httpPost("/notebook/cron/" + noteId, jsonRequest);
       assertThat("", postCron, isAllowed());
       postCron.close();
       Thread.sleep(1000);
 
       // wrong cron expression.
       jsonRequest = "{\"cron\":\"a * * * * ?\" }";
-      postCron = httpPost("/notebook/cron/" + note.getId(), jsonRequest);
+      postCron = httpPost("/notebook/cron/" + noteId, jsonRequest);
       assertThat("", postCron, isBadRequest());
       postCron.close();
       Thread.sleep(1000);
 
       // remove cron job.
-      CloseableHttpResponse deleteCron = httpDelete("/notebook/cron/" + note.getId());
+      CloseableHttpResponse deleteCron = httpDelete("/notebook/cron/" + noteId);
       assertThat("", deleteCron, isAllowed());
       deleteCron.close();
     } finally {
       //cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
       System.clearProperty(ConfVars.ZEPPELIN_NOTEBOOK_CRON_ENABLE.getVarName());
     }
@@ -639,40 +721,66 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
 
   @Test
   public void testCronDisable() throws Exception {
-    Note note = null;
+    String noteId = null;
     try {
       // create a note and a paragraph
       System.setProperty(ConfVars.ZEPPELIN_NOTEBOOK_CRON_ENABLE.getVarName(), "false");
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testCronDisable", anonymous);
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testCronDisable", anonymous);
+      // use write lock because Name is overwritten
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          note.setName("note for run test");
+          Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          paragraph.setText("%md This is test paragraph.");
+
+          Map<String, Object> config = paragraph.getConfig();
+          config.put("enabled", true);
+          paragraph.setConfig(config);
+          return null;
+        });
+
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          try {
+            note.runAll(AuthenticationInfo.ANONYMOUS, true, true, new HashMap<>());
+          } catch (Exception e) {
+            fail();
+          }
+          return null;
+        });
 
-      note.setName("note for run test");
-      Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      paragraph.setText("%md This is test paragraph.");
-
-      Map<String, Object> config = paragraph.getConfig();
-      config.put("enabled", true);
-      paragraph.setConfig(config);
-
-      note.runAll(AuthenticationInfo.ANONYMOUS, true, true, new HashMap<>());
 
       String jsonRequest = "{\"cron\":\"* * * * * ?\" }";
       // right cron expression.
-      CloseableHttpResponse postCron = httpPost("/notebook/cron/" + note.getId(), jsonRequest);
+      CloseableHttpResponse postCron = httpPost("/notebook/cron/" + noteId, jsonRequest);
       assertThat("", postCron, isForbidden());
       postCron.close();
 
       System.setProperty(ConfVars.ZEPPELIN_NOTEBOOK_CRON_ENABLE.getVarName(), "true");
       System.setProperty(ConfVars.ZEPPELIN_NOTEBOOK_CRON_FOLDERS.getVarName(), "/System");
 
-      note.setName("System/test2");
-      note.runAll(AuthenticationInfo.ANONYMOUS, true, true, new HashMap<>());
-      postCron = httpPost("/notebook/cron/" + note.getId(), jsonRequest);
+      // use write lock, because Name is overwritten
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          note.setName("System/test2");
+          return null;
+        });
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          try {
+            note.runAll(AuthenticationInfo.ANONYMOUS, true, true, new HashMap<>());
+          } catch (Exception e) {
+            fail();
+          }
+          return null;
+        });
+      postCron = httpPost("/notebook/cron/" + noteId, jsonRequest);
       assertThat("", postCron, isAllowed());
       postCron.close();
       Thread.sleep(1000);
 
       // remove cron job.
-      CloseableHttpResponse deleteCron = httpDelete("/notebook/cron/" + note.getId());
+      CloseableHttpResponse deleteCron = httpDelete("/notebook/cron/" + noteId);
       assertThat("", deleteCron, isAllowed());
       deleteCron.close();
       Thread.sleep(1000);
@@ -680,8 +788,8 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
       System.clearProperty(ConfVars.ZEPPELIN_NOTEBOOK_CRON_FOLDERS.getVarName());
     } finally {
       //cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
       System.clearProperty(ConfVars.ZEPPELIN_NOTEBOOK_CRON_ENABLE.getVarName());
     }
@@ -689,17 +797,32 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
 
   @Test
   public void testRegressionZEPPELIN_527() throws Exception {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testRegressionZEPPELIN_527", anonymous);
-
-      note.setName("note for run test");
-      Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      paragraph.setText("%spark\nval param = z.input(\"param\").toString\nprintln(param)");
-      note.runAll(AuthenticationInfo.ANONYMOUS, true, false, new HashMap<>());
-      TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
-
-      CloseableHttpResponse getNoteJobs = httpGet("/notebook/job/" + note.getId());
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testRegressionZEPPELIN_527", anonymous);
+      // use write lock because name is overwritten
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          note.setName("note for run test");
+          Paragraph paragraph = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          paragraph.setText("%spark\nval param = z.input(\"param\").toString\nprintln(param)");
+          return null;
+        });
+
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          try {
+            note.runAll(AuthenticationInfo.ANONYMOUS, true, false, new HashMap<>());
+            TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
+          } catch (Exception e) {
+            fail();
+          }
+          return null;
+        });
+
+
+
+      CloseableHttpResponse getNoteJobs = httpGet("/notebook/job/" + noteId);
       assertThat("test note jobs run:", getNoteJobs, isAllowed());
       Map<String, Object> resp = gson.fromJson(EntityUtils.toString(getNoteJobs.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
@@ -709,20 +832,20 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
       getNoteJobs.close();
     } finally {
       //cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testInsertParagraph() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testInsertParagraph", anonymous);
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testInsertParagraph", anonymous);
 
       String jsonRequest = "{\"title\": \"title1\", \"text\": \"text1\"}";
-      CloseableHttpResponse post = httpPost("/notebook/" + note.getId() + "/paragraph", jsonRequest);
+      CloseableHttpResponse post = httpPost("/notebook/" + noteId + "/paragraph", jsonRequest);
       String postResponse = EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8);
       LOG.info("testInsertParagraph response\n" + postResponse);
       assertThat("Test insert method:", post, isAllowed());
@@ -734,24 +857,25 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
       String newParagraphId = (String) resp.get("body");
       LOG.info("newParagraphId:=" + newParagraphId);
 
-      Note retrNote = TestUtils.getInstance(Notebook.class).getNote(note.getId());
-      Paragraph newParagraph = retrNote.getParagraph(newParagraphId);
-      assertNotNull("Can not find new paragraph by id", newParagraph);
-
-      assertEquals("title1", newParagraph.getTitle());
-      assertEquals("text1", newParagraph.getText());
-
-      Paragraph lastParagraph = note.getLastParagraph();
-      assertEquals(newParagraph.getId(), lastParagraph.getId());
+      Paragraph lastParagraph = TestUtils.getInstance(Notebook.class).processNote(noteId, Note::getLastParagraph);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        retrNote -> {
+          Paragraph newParagraph = retrNote.getParagraph(newParagraphId);
+          assertNotNull("Can not find new paragraph by id", newParagraph);
+          assertEquals("title1", newParagraph.getTitle());
+          assertEquals("text1", newParagraph.getText());
+          assertEquals(newParagraph.getId(), lastParagraph.getId());
+          return null;
+        });
 
       // insert to index 0
       String jsonRequest2 = "{\"index\": 0, \"title\": \"title2\", \"text\": \"text2\"}";
-      CloseableHttpResponse post2 = httpPost("/notebook/" + note.getId() + "/paragraph", jsonRequest2);
+      CloseableHttpResponse post2 = httpPost("/notebook/" + noteId + "/paragraph", jsonRequest2);
       LOG.info("testInsertParagraph response2\n" + EntityUtils.toString(post2.getEntity(), StandardCharsets.UTF_8));
       assertThat("Test insert method:", post2, isAllowed());
       post2.close();
 
-      Paragraph paragraphAtIdx0 = note.getParagraphs().get(0);
+      Paragraph paragraphAtIdx0 =TestUtils.getInstance(Notebook.class).processNote(noteId, note -> note.getParagraphs().get(0));
       assertEquals("title2", paragraphAtIdx0.getTitle());
       assertEquals("text2", paragraphAtIdx0.getText());
 
@@ -759,12 +883,12 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
       String jsonRequest3 = "{\"title\": \"title3\", \"text\": \"text3\", " +
                             "\"config\": {\"colWidth\": 9.0, \"title\": true, " +
                             "\"results\": [{\"graph\": {\"mode\": \"pieChart\"}}]}}";
-      CloseableHttpResponse post3 = httpPost("/notebook/" + note.getId() + "/paragraph", jsonRequest3);
+      CloseableHttpResponse post3 = httpPost("/notebook/" + noteId + "/paragraph", jsonRequest3);
       LOG.info("testInsertParagraph response4\n" + EntityUtils.toString(post3.getEntity(), StandardCharsets.UTF_8));
       assertThat("Test insert method:", post3, isAllowed());
       post3.close();
 
-      Paragraph p = note.getLastParagraph();
+      Paragraph p = TestUtils.getInstance(Notebook.class).processNote(noteId, Note::getLastParagraph);
       assertEquals("title3", p.getTitle());
       assertEquals("text3", p.getText());
       Map result = ((List<Map>) p.getConfig().get("results")).get(0);
@@ -774,73 +898,82 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
       assertTrue(((boolean) p.getConfig().get("title")));
     } finally {
       //cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testUpdateParagraph() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testUpdateParagraph", anonymous);
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testUpdateParagraph", anonymous);
 
       String jsonRequest = "{\"title\": \"title1\", \"text\": \"text1\"}";
-      CloseableHttpResponse post = httpPost("/notebook/" + note.getId() + "/paragraph", jsonRequest);
+      CloseableHttpResponse post = httpPost("/notebook/" + noteId + "/paragraph", jsonRequest);
       Map<String, Object> resp = gson.fromJson(EntityUtils.toString(post.getEntity(), StandardCharsets.UTF_8),
               new TypeToken<Map<String, Object>>() {}.getType());
       post.close();
 
       String newParagraphId = (String) resp.get("body");
-      Paragraph newParagraph = TestUtils.getInstance(Notebook.class).getNote(note.getId())
-              .getParagraph(newParagraphId);
-
-      assertEquals("title1", newParagraph.getTitle());
-      assertEquals("text1", newParagraph.getText());
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        noteP -> {
+          Paragraph newParagraph = noteP.getParagraph(newParagraphId);
+          assertEquals("title1", newParagraph.getTitle());
+          assertEquals("text1", newParagraph.getText());
+          return null;
+        });
 
       String updateRequest = "{\"text\": \"updated text\"}";
-      CloseableHttpResponse put = httpPut("/notebook/" + note.getId() + "/paragraph/" + newParagraphId,
+      CloseableHttpResponse put = httpPut("/notebook/" + noteId + "/paragraph/" + newParagraphId,
               updateRequest);
       assertThat("Test update method:", put, isAllowed());
       put.close();
 
-      Paragraph updatedParagraph = TestUtils.getInstance(Notebook.class).getNote(note.getId())
-              .getParagraph(newParagraphId);
-
-      assertEquals("title1", updatedParagraph.getTitle());
-      assertEquals("updated text", updatedParagraph.getText());
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        noteP -> {
+          Paragraph updatedParagraph = noteP.getParagraph(newParagraphId);
+          assertEquals("title1", updatedParagraph.getTitle());
+          assertEquals("updated text", updatedParagraph.getText());
+          return null;
+        });
 
       String updateBothRequest = "{\"title\": \"updated title\", \"text\" : \"updated text 2\" }";
-      CloseableHttpResponse updatePut = httpPut("/notebook/" + note.getId() + "/paragraph/" + newParagraphId,
+      CloseableHttpResponse updatePut = httpPut("/notebook/" + noteId + "/paragraph/" + newParagraphId,
               updateBothRequest);
       updatePut.close();
 
-      Paragraph updatedBothParagraph = TestUtils.getInstance(Notebook.class).getNote(note.getId())
-              .getParagraph(newParagraphId);
-
-      assertEquals("updated title", updatedBothParagraph.getTitle());
-      assertEquals("updated text 2", updatedBothParagraph.getText());
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        noteP -> {
+          Paragraph updatedBothParagraph = noteP.getParagraph(newParagraphId);
+          assertEquals("updated title", updatedBothParagraph.getTitle());
+          assertEquals("updated text 2", updatedBothParagraph.getText());
+          return null;
+        });
     } finally {
       //cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testGetParagraph() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testGetParagraph", anonymous);
-
-      Paragraph p = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p.setTitle("hello");
-      p.setText("world");
-      TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
-
-      CloseableHttpResponse get = httpGet("/notebook/" + note.getId() + "/paragraph/" + p.getId());
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testGetParagraph", anonymous);
+      String pId = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p.setTitle("hello");
+          p.setText("world");
+          TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
+          return p.getId();
+        });
+
+      CloseableHttpResponse get = httpGet("/notebook/" + noteId + "/paragraph/" + pId);
       String getResponse = EntityUtils.toString(get.getEntity(), StandardCharsets.UTF_8);
       LOG.info("testGetParagraph response\n" + getResponse);
       assertThat("Test get method: ", get, isAllowed());
@@ -854,92 +987,104 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
 
       Map<String, Object> body = (Map<String, Object>) resp.get("body");
 
-      assertEquals(p.getId(), body.get("id"));
+      assertEquals(pId, body.get("id"));
       assertEquals("hello", body.get("title"));
       assertEquals("world", body.get("text"));
     } finally {
       //cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testMoveParagraph() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testMoveParagraph", anonymous);
-
-      Paragraph p = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p.setTitle("title1");
-      p.setText("text1");
-
-      Paragraph p2 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p2.setTitle("title2");
-      p2.setText("text2");
-
-      TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
-
-      CloseableHttpResponse post = httpPost("/notebook/" + note.getId() + "/paragraph/" + p2.getId() +
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testMoveParagraph", anonymous);
+      Paragraph p2 = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph p = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p.setTitle("title1");
+          p.setText("text1");
+          Paragraph p2tmp = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p2tmp.setTitle("title2");
+          p2tmp.setText("text2");
+          TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
+          return p2tmp;
+        });
+
+      CloseableHttpResponse post = httpPost("/notebook/" + noteId + "/paragraph/" + p2.getId() +
               "/move/" + 0, "");
       assertThat("Test post method: ", post, isAllowed());
       post.close();
 
-      Note retrNote = TestUtils.getInstance(Notebook.class).getNote(note.getId());
-      Paragraph paragraphAtIdx0 = retrNote.getParagraphs().get(0);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        retrNote -> {
+          Paragraph paragraphAtIdx0 = retrNote.getParagraphs().get(0);
+
+          assertEquals(p2.getId(), paragraphAtIdx0.getId());
+          assertEquals(p2.getTitle(), paragraphAtIdx0.getTitle());
+          assertEquals(p2.getText(), paragraphAtIdx0.getText());
+          return null;
+        });
 
-      assertEquals(p2.getId(), paragraphAtIdx0.getId());
-      assertEquals(p2.getTitle(), paragraphAtIdx0.getTitle());
-      assertEquals(p2.getText(), paragraphAtIdx0.getText());
 
-      CloseableHttpResponse post2 = httpPost("/notebook/" + note.getId() + "/paragraph/" + p2.getId() +
+      CloseableHttpResponse post2 = httpPost("/notebook/" + noteId + "/paragraph/" + p2.getId() +
               "/move/" + 10, "");
       assertThat("Test post method: ", post2, isBadRequest());
       post.close();
     } finally {
       //cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testDeleteParagraph() throws IOException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testDeleteParagraph", anonymous);
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testDeleteParagraph", anonymous);
+      Paragraph p = TestUtils.getInstance(Notebook.class).processNote(noteId,
+        note -> {
+          Paragraph ptmp = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          ptmp.setTitle("title1");
+          ptmp.setText("text1");
 
-      Paragraph p = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p.setTitle("title1");
-      p.setText("text1");
+          TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
+          return ptmp;
+        });
 
-      TestUtils.getInstance(Notebook.class).saveNote(note, anonymous);
 
-      CloseableHttpResponse delete = httpDelete("/notebook/" + note.getId() + "/paragraph/" + p.getId());
+      CloseableHttpResponse delete = httpDelete("/notebook/" + noteId + "/paragraph/" + p.getId());
       assertThat("Test delete method: ", delete, isAllowed());
       delete.close();
 
-      Note retrNote = TestUtils.getInstance(Notebook.class).getNote(note.getId());
-      Paragraph retrParagrah = retrNote.getParagraph(p.getId());
-      assertNull("paragraph should be deleted", retrParagrah);
+      TestUtils.getInstance(Notebook.class).processNote(noteId,
+        retrNote -> {
+          Paragraph retrParagrah = retrNote.getParagraph(p.getId());
+          assertNull("paragraph should be deleted", retrParagrah);
+          return null;
+        });
     } finally {
       //cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testTitleSearch() throws IOException, InterruptedException {
-    Note note = null;
+    String noteId = null;
     try {
-      note = TestUtils.getInstance(Notebook.class).createNote("note1_testTitleSearch", anonymous);
+      noteId = TestUtils.getInstance(Notebook.class).createNote("note1_testTitleSearch", anonymous);
       String jsonRequest = "{\"title\": \"testTitleSearchOfParagraph\", " +
               "\"text\": \"ThisIsToTestSearchMethodWithTitle \"}";
-      CloseableHttpResponse postNoteText = httpPost("/notebook/" + note.getId() + "/paragraph", jsonRequest);
+      CloseableHttpResponse postNoteText = httpPost("/notebook/" + noteId + "/paragraph", jsonRequest);
       postNoteText.close();
       Thread.sleep(3000);
 
@@ -960,8 +1105,8 @@ public class ZeppelinRestApiTest extends AbstractTestRestApi {
       searchNote.close();
     } finally {
       //cleanup
-      if (null != note) {
-        TestUtils.getInstance(Notebook.class).removeNote(note, anonymous);
+      if (null != noteId) {
+        TestUtils.getInstance(Notebook.class).removeNote(noteId, anonymous);
       }
     }
   }
diff --git a/zeppelin-server/src/test/java/org/apache/zeppelin/service/NotebookServiceTest.java b/zeppelin-server/src/test/java/org/apache/zeppelin/service/NotebookServiceTest.java
index 9b21150..9e5489e 100644
--- a/zeppelin-server/src/test/java/org/apache/zeppelin/service/NotebookServiceTest.java
+++ b/zeppelin-server/src/test/java/org/apache/zeppelin/service/NotebookServiceTest.java
@@ -79,6 +79,7 @@ public class NotebookServiceTest {
 
   private File notebookDir;
   private SearchService searchService;
+  private Notebook notebook;
   private ServiceContext context =
       new ServiceContext(AuthenticationInfo.ANONYMOUS, new HashSet<>());
 
@@ -113,11 +114,10 @@ public class NotebookServiceTest {
     when(mockInterpreterSetting.isUserAuthorized(any())).thenReturn(true);
     when(mockInterpreterGroup.getInterpreterSetting()).thenReturn(mockInterpreterSetting);
     when(mockInterpreterSetting.getStatus()).thenReturn(InterpreterSetting.Status.READY);
-    searchService = new LuceneSearch(zeppelinConfiguration);
     Credentials credentials = new Credentials();
-    NoteManager noteManager = new NoteManager(notebookRepo);
+    NoteManager noteManager = new NoteManager(notebookRepo, zeppelinConfiguration);
     AuthorizationService authorizationService = new AuthorizationService(noteManager, zeppelinConfiguration);
-    Notebook notebook =
+    notebook =
         new Notebook(
             zeppelinConfiguration,
             authorizationService,
@@ -125,10 +125,9 @@ public class NotebookServiceTest {
             noteManager,
             mockInterpreterFactory,
             mockInterpreterSettingManager,
-            searchService,
             credentials,
             null);
-
+    searchService = new LuceneSearch(zeppelinConfiguration, notebook);
     QuartzSchedulerService schedulerService = new QuartzSchedulerService(zeppelinConfiguration, notebook);
     schedulerService.waitForFinishInit();
     notebookService =
@@ -150,37 +149,50 @@ public class NotebookServiceTest {
   @Test
   public void testNoteOperations() throws IOException {
     // get home note
-    Note homeNote = notebookService.getHomeNote(context, callback);
-    assertNull(homeNote);
-    verify(callback).onSuccess(homeNote, context);
+    String homeNoteId = notebookService.getHomeNote(context, callback);
+    assertNull(homeNoteId);
+    notebook.processNote(homeNoteId,
+      homeNote -> {
+        verify(callback).onSuccess(homeNote, context);
+        return null;
+      });
 
     // create note
-    Note note1 = notebookService.createNote("/folder_1/note1", "test", true, context, callback);
-    assertEquals("note1", note1.getName());
-    assertEquals(1, note1.getParagraphCount());
-    verify(callback).onSuccess(note1, context);
+    String note1Id = notebookService.createNote("/folder_1/note1", "test", true, context, callback);
+    Note note1 = notebook.processNote(note1Id,
+      note1Read -> {
+        assertEquals("note1", note1Read.getName());
+        assertEquals(1, note1Read.getParagraphCount());
+        verify(callback).onSuccess(note1Read, context);
+        return note1Read;
+      });
+
 
     // create duplicated note
     reset(callback);
-    Note note2 = notebookService.createNote("/folder_1/note1", "test", true, context, callback);
-    assertNull(note2);
+    String note2Id = notebookService.createNote("/folder_1/note1", "test", true, context, callback);
+    assertNull(note2Id);
     ArgumentCaptor<Exception> exception = ArgumentCaptor.forClass(Exception.class);
     verify(callback).onFailure(exception.capture(), any(ServiceContext.class));
-    assertTrue(exception.getValue().getMessage().equals("Note '/folder_1/note1' existed"));
+    assertEquals("Note '/folder_1/note1' existed", exception.getValue().getMessage());
 
     // list note
     reset(callback);
     List<NoteInfo> notesInfo = notebookService.listNotesInfo(false, context, callback);
     assertEquals(1, notesInfo.size());
-    assertEquals(note1.getId(), notesInfo.get(0).getId());
+    assertEquals(note1Id, notesInfo.get(0).getId());
     assertEquals(note1.getName(), notesInfo.get(0).getNoteName());
     verify(callback).onSuccess(notesInfo, context);
 
     // get note
     reset(callback);
-    Note note1_copy = notebookService.getNote(note1.getId(), context, callback);
-    assertEquals(note1, note1_copy);
-    verify(callback).onSuccess(note1_copy, context);
+    notebookService.getNote(note1.getId(), context, callback,
+      note1_copy -> {
+        assertEquals(note1, note1_copy);
+        verify(callback).onSuccess(note1_copy, context);
+        return null;
+      });
+
 
     // rename note
     reset(callback);
@@ -203,15 +215,23 @@ public class NotebookServiceTest {
     assertEquals("/folder_4/new_name", notesInfo.get(0).getPath());
 
     // create another note
-    note2 = notebookService.createNote("/note2", "test", true, context, callback);
-    assertEquals("note2", note2.getName());
-    verify(callback).onSuccess(note2, context);
+    note2Id = notebookService.createNote("/note2", "test", true, context, callback);
+    notebook.processNote(note2Id,
+      note2 -> {
+        assertEquals("note2", note2.getName());
+        verify(callback).onSuccess(note2, context);
+        return null;
+      });
 
     // rename note
     reset(callback);
-    notebookService.renameNote(note2.getId(), "new_note2", true, context, callback);
-    verify(callback).onSuccess(note2, context);
-    assertEquals("new_note2", note2.getName());
+    notebookService.renameNote(note2Id, "new_note2", true, context, callback);
+    notebook.processNote(note2Id,
+      note2 -> {
+        verify(callback).onSuccess(note2, context);
+        assertEquals("new_note2", note2.getName());
+        return null;
+      });
 
     // list note
     reset(callback);
@@ -221,7 +241,7 @@ public class NotebookServiceTest {
 
     // delete note
     reset(callback);
-    notebookService.removeNote(note2.getId(), context, callback);
+    notebookService.removeNote(note2Id, context, callback);
     verify(callback).onSuccess("Delete note successfully", context);
 
     // list note again
@@ -242,16 +262,28 @@ public class NotebookServiceTest {
 
     // import note
     reset(callback);
-    Note importedNote = notebookService.importNote("/Imported Note", "{}", context, callback);
-    assertNotNull(importedNote);
-    verify(callback).onSuccess(importedNote, context);
+    String importedNoteId = notebookService.importNote("/Imported Note", "{}", context, callback);
+    notebook.processNote(importedNoteId,
+      importedNote -> {
+        assertNotNull(importedNote);
+        verify(callback).onSuccess(importedNote, context);
+        return null;
+      });
 
     // clone note
     reset(callback);
-    Note clonedNote = notebookService.cloneNote(importedNote.getId(), "/Backup/Cloned Note",
+    String clonedNoteId = notebookService.cloneNote(importedNoteId, "/Backup/Cloned Note",
         context, callback);
-    assertEquals(importedNote.getParagraphCount(), clonedNote.getParagraphCount());
-    verify(callback).onSuccess(clonedNote, context);
+    notebook.processNote(importedNoteId,
+      importedNote -> {
+        notebook.processNote(clonedNoteId,
+          clonedNote -> {
+            assertEquals(importedNote.getParagraphCount(), clonedNote.getParagraphCount());
+            verify(callback).onSuccess(clonedNote, context);
+            return null;
+          });
+        return null;
+      });
 
     // list note
     reset(callback);
@@ -260,21 +292,25 @@ public class NotebookServiceTest {
     verify(callback).onSuccess(notesInfo, context);
 
     // test moving corrupted note to trash
-    Note corruptedNote = notebookService.createNote("/folder_1/corruptedNote", "test", true, context, callback);
-    String corruptedNotePath = notebookDir.getAbsolutePath() + corruptedNote.getPath() + "_" + corruptedNote.getId() + ".zpln";
-    // corrupt note
-    FileWriter myWriter = new FileWriter(corruptedNotePath);
-    myWriter.write("{{{I'm corrupted;;;");
-    myWriter.close();
-    notebookService.moveNoteToTrash(corruptedNote.getId(), context, callback);
+    String corruptedNoteId = notebookService.createNote("/folder_1/corruptedNote", "test", true, context, callback);
+    notebook.processNote(corruptedNoteId,
+      corruptedNote -> {
+        String corruptedNotePath = notebookDir.getAbsolutePath() + corruptedNote.getPath() + "_" + corruptedNote.getId() + ".zpln";
+        // corrupt note
+        FileWriter myWriter = new FileWriter(corruptedNotePath);
+        myWriter.write("{{{I'm corrupted;;;");
+        myWriter.close();
+        return null;
+      });
+    notebookService.moveNoteToTrash(corruptedNoteId, context, callback);
+
     reset(callback);
     notesInfo = notebookService.listNotesInfo(false, context, callback);
     assertEquals(3, notesInfo.size());
     verify(callback).onSuccess(notesInfo, context);
-    notebookService.removeNote(corruptedNote.getId(), context, callback);
-
+    notebookService.removeNote(corruptedNoteId, context, callback);
     // move note to Trash
-    notebookService.moveNoteToTrash(importedNote.getId(), context, callback);
+    notebookService.moveNoteToTrash(importedNoteId, context, callback);
 
     reset(callback);
     notesInfo = notebookService.listNotesInfo(false, context, callback);
@@ -283,7 +319,7 @@ public class NotebookServiceTest {
 
     boolean moveToTrash = false;
     for (NoteInfo noteInfo : notesInfo) {
-      if (noteInfo.getId().equals(importedNote.getId())) {
+      if (noteInfo.getId().equals(importedNoteId)) {
         assertEquals("/~Trash/Imported Note", noteInfo.getPath());
         moveToTrash = true;
       }
@@ -291,18 +327,19 @@ public class NotebookServiceTest {
     assertTrue("No note is moved to trash", moveToTrash);
 
     // restore it
-    notebookService.restoreNote(importedNote.getId(), context, callback);
-    Note restoredNote = notebookService.getNote(importedNote.getId(), context, callback);
-    assertNotNull(restoredNote);
-    assertEquals("/Imported Note", restoredNote.getPath());
-
-    // move it to Trash again
-    notebookService.moveNoteToTrash(restoredNote.getId(), context, callback);
-
+    notebookService.restoreNote(importedNoteId, context, callback);
+    notebookService.getNote(importedNoteId, context, callback,
+      restoredNote -> {
+        assertNotNull(restoredNote);
+        assertEquals("/Imported Note", restoredNote.getPath());
+        return null;
+      });
+ // move it to Trash again
+    notebookService.moveNoteToTrash(importedNoteId, context, callback);
     // remove note from Trash
     reset(callback);
 
-    notebookService.removeNote(importedNote.getId(), context, callback);
+    notebookService.removeNote(importedNoteId, context, callback);
     notesInfo = notebookService.listNotesInfo(false, context, callback);
     assertEquals(1, notesInfo.size());
 
@@ -315,7 +352,7 @@ public class NotebookServiceTest {
     verify(callback).onSuccess(notesInfo, context);
     moveToTrash = false;
     for (NoteInfo noteInfo : notesInfo) {
-      if (noteInfo.getId().equals(clonedNote.getId())) {
+      if (noteInfo.getId().equals(clonedNoteId)) {
         assertEquals("/~Trash/Backup/Cloned Note", noteInfo.getPath());
         moveToTrash = true;
       }
@@ -325,9 +362,12 @@ public class NotebookServiceTest {
     // restore folder
     reset(callback);
     notebookService.restoreFolder("/~Trash/Backup", context, callback);
-    restoredNote = notebookService.getNote(clonedNote.getId(), context, callback);
-    assertNotNull(restoredNote);
-    assertEquals("/Backup/Cloned Note", restoredNote.getPath());
+    notebookService.getNote(clonedNoteId, context, callback,
+      restoredNote -> {
+        assertNotNull(restoredNote);
+        assertEquals("/Backup/Cloned Note", restoredNote.getPath());
+        return null;
+      });
 
     // move the folder to trash again
     notebookService.moveFolderToTrash("Backup", context, callback);
@@ -347,18 +387,26 @@ public class NotebookServiceTest {
 
   @Test
   public void testRenameNoteRejectsDuplicate() throws IOException {
-    Note note1 = notebookService.createNote("/folder/note1", "test", true, context, callback);
-    assertEquals("note1", note1.getName());
-    verify(callback).onSuccess(note1, context);
+    String note1Id = notebookService.createNote("/folder/note1", "test", true, context, callback);
+    notebook.processNote(note1Id,
+      note1 -> {
+        assertEquals("note1", note1.getName());
+        verify(callback).onSuccess(note1, context);
+        return null;
+      });
 
     reset(callback);
-    Note note2 = notebookService.createNote("/folder/note2", "test", true, context, callback);
-    assertEquals("note2", note2.getName());
-    verify(callback).onSuccess(note2, context);
+    String note2Id = notebookService.createNote("/folder/note2", "test", true, context, callback);
+    notebook.processNote(note2Id,
+      note2 -> {
+        assertEquals("note2", note2.getName());
+        verify(callback).onSuccess(note2, context);
+        return null;
+      });
 
     reset(callback);
     ArgumentCaptor<NotePathAlreadyExistsException> exception = ArgumentCaptor.forClass(NotePathAlreadyExistsException.class);
-    notebookService.renameNote(note1.getId(), "/folder/note2", false, context, callback);
+    notebookService.renameNote(note1Id, "/folder/note2", false, context, callback);
     verify(callback).onFailure(exception.capture(), any(ServiceContext.class));
     assertEquals("Note '/folder/note2' existed", exception.getValue().getMessage());
     verify(callback, never()).onSuccess(any(), any());
@@ -368,46 +416,66 @@ public class NotebookServiceTest {
   @Test
   public void testParagraphOperations() throws IOException {
     // create note
-    Note note1 = notebookService.createNote("note1", "python", false, context, callback);
-    assertEquals("note1", note1.getName());
-    assertEquals(0, note1.getParagraphCount());
-    verify(callback).onSuccess(note1, context);
+    String note1Id = notebookService.createNote("note1", "python", false, context, callback);
+    notebook.processNote(note1Id,
+      note1 -> {
+        assertEquals("note1", note1.getName());
+        assertEquals(0, note1.getParagraphCount());
+        verify(callback).onSuccess(note1, context);
+        return null;
+      });
+
 
     // add paragraph
     reset(callback);
-    Paragraph p = notebookService.insertParagraph(note1.getId(), 0, new HashMap<>(), context,
+    Paragraph p = notebookService.insertParagraph(note1Id, 0, new HashMap<>(), context,
         callback);
     assertNotNull(p);
     verify(callback).onSuccess(p, context);
-    assertEquals(1, note1.getParagraphCount());
+    notebook.processNote(note1Id,
+      note1 -> {
+        assertEquals(1, note1.getParagraphCount());
+        return null;
+      });
 
     // update paragraph
     reset(callback);
-    notebookService.updateParagraph(note1.getId(), p.getId(), "my_title", "my_text",
+    notebookService.updateParagraph(note1Id, p.getId(), "my_title", "my_text",
         new HashMap<>(), new HashMap<>(), context, callback);
     assertEquals("my_title", p.getTitle());
     assertEquals("my_text", p.getText());
 
     // move paragraph
     reset(callback);
-    notebookService.moveParagraph(note1.getId(), p.getId(), 0, context, callback);
-    assertEquals(p, note1.getParagraph(0));
-    verify(callback).onSuccess(p, context);
+    notebookService.moveParagraph(note1Id, p.getId(), 0, context, callback);
+    notebook.processNote(note1Id,
+      note1 -> {
+        assertEquals(p, note1.getParagraph(0));
+        verify(callback).onSuccess(p, context);
+        return null;
+      });
 
     // run paragraph asynchronously
     reset(callback);
     p.getConfig().put("colWidth", "6.0");
     p.getConfig().put("title", true);
-    boolean runStatus = notebookService.runParagraph(note1.getId(), p.getId(), "my_title", "1+1",
-        new HashMap<>(), new HashMap<>(), null, false, false, context, callback);
+    boolean runStatus = notebook.processNote(note1Id,
+      note1 -> {
+        return notebookService.runParagraph(note1, p.getId(), "my_title", "1+1",
+          new HashMap<>(), new HashMap<>(), null, false, false, context, callback);
+      });
     assertTrue(runStatus);
     verify(callback).onSuccess(p, context);
     assertEquals(2, p.getConfig().size());
 
     // run paragraph synchronously via correct code
     reset(callback);
-    runStatus = notebookService.runParagraph(note1.getId(), p.getId(), "my_title", "1+1",
-        new HashMap<>(), new HashMap<>(), null, false, true, context, callback);
+    runStatus = notebook.processNote(note1Id,
+      note1 -> {
+        return notebookService.runParagraph(note1, p.getId(), "my_title", "1+1",
+          new HashMap<>(), new HashMap<>(), null, false, true, context, callback);
+      });
+
     assertTrue(runStatus);
     verify(callback).onSuccess(p, context);
     assertEquals(2, p.getConfig().size());
@@ -415,13 +483,16 @@ public class NotebookServiceTest {
     // run all paragraphs, with null paragraph list provided
     reset(callback);
     assertTrue(notebookService.runAllParagraphs(
-            note1.getId(),
+            note1Id,
             null,
             context, callback));
 
     reset(callback);
-    runStatus = notebookService.runParagraph(note1.getId(), p.getId(), "my_title", "invalid_code",
-        new HashMap<>(), new HashMap<>(), null, false, true, context, callback);
+    runStatus = notebook.processNote(note1Id,
+      note1 -> {
+        return notebookService.runParagraph(note1, p.getId(), "my_title", "invalid_code",
+          new HashMap<>(), new HashMap<>(), null, false, true, context, callback);
+      });
     assertTrue(runStatus);
     // TODO(zjffdu) Enable it after ZEPPELIN-3699
     // assertNotNull(p.getResult());
@@ -429,7 +500,7 @@ public class NotebookServiceTest {
 
     // clean output
     reset(callback);
-    notebookService.clearParagraphOutput(note1.getId(), p.getId(), context, callback);
+    notebookService.clearParagraphOutput(note1Id, p.getId(), context, callback);
     assertNull(p.getReturn());
     verify(callback).onSuccess(p, context);
   }
diff --git a/zeppelin-server/src/test/java/org/apache/zeppelin/socket/NotebookServerTest.java b/zeppelin-server/src/test/java/org/apache/zeppelin/socket/NotebookServerTest.java
index 7e15df8..168def2 100644
--- a/zeppelin-server/src/test/java/org/apache/zeppelin/socket/NotebookServerTest.java
+++ b/zeppelin-server/src/test/java/org/apache/zeppelin/socket/NotebookServerTest.java
@@ -23,12 +23,12 @@ import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.anyString;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.reset;
-import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
@@ -36,6 +36,7 @@ import static org.mockito.Mockito.when;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashSet;
@@ -56,16 +57,16 @@ import org.apache.zeppelin.interpreter.thrift.ParagraphInfo;
 import org.apache.zeppelin.interpreter.thrift.ServiceException;
 import org.apache.zeppelin.notebook.AuthorizationService;
 import org.apache.zeppelin.notebook.Note;
+import org.apache.zeppelin.notebook.NoteInfo;
 import org.apache.zeppelin.notebook.Notebook;
 import org.apache.zeppelin.notebook.Paragraph;
+import org.apache.zeppelin.notebook.Notebook.NoteProcessor;
 import org.apache.zeppelin.notebook.repo.NotebookRepoWithVersionControl;
-import org.apache.zeppelin.notebook.scheduler.QuartzSchedulerService;
-import org.apache.zeppelin.notebook.scheduler.SchedulerService;
 import org.apache.zeppelin.common.Message;
 import org.apache.zeppelin.common.Message.OP;
 import org.apache.zeppelin.rest.AbstractTestRestApi;
 import org.apache.zeppelin.scheduler.Job;
-import org.apache.zeppelin.service.ConfigurationService;
+import org.apache.zeppelin.scheduler.Job.Status;
 import org.apache.zeppelin.service.NotebookService;
 import org.apache.zeppelin.service.ServiceContext;
 import org.apache.zeppelin.user.AuthenticationInfo;
@@ -74,6 +75,7 @@ import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 
 /** Basic REST API tests for notebookServer. */
@@ -127,19 +129,22 @@ public class NotebookServerTest extends AbstractTestRestApi {
 
     String noteName = "Note with millis " + System.currentTimeMillis();
     notebookServer.onMessage(sock1, new Message(OP.NEW_NOTE).put("name", noteName).toJson());
-    Note createdNote = null;
-    for (Note note : notebook.getAllNotes()) {
-      if (note.getName().equals(noteName)) {
-        createdNote = note;
+    NoteInfo createdNoteInfo = null;
+    for (NoteInfo noteInfo : notebook.getNotesInfo()) {
+      if (notebook.processNote(noteInfo.getId(), Note::getName).equals(noteName)) {
+        createdNoteInfo = noteInfo;
         break;
       }
     }
 
-    Message message = new Message(OP.GET_NOTE).put("id", createdNote.getId());
+    Message message = new Message(OP.GET_NOTE).put("id", createdNoteInfo.getId());
     notebookServer.onMessage(sock1, message.toJson());
     notebookServer.onMessage(sock2, message.toJson());
 
-    Paragraph paragraph = createdNote.getParagraphs().get(0);
+    Paragraph paragraph = notebook.processNote(createdNoteInfo.getId(),
+      createdNote -> {
+        return createdNote.getParagraphs().get(0);
+      });
     String paragraphId = paragraph.getId();
 
     String[] patches = new String[]{
@@ -173,7 +178,7 @@ public class NotebookServerTest extends AbstractTestRestApi {
     verify(sock1, times(++sock1SendCount)).send(anyString());
     verify(sock2, times(sock2SendCount)).send(anyString());
 
-    notebook.removeNote(createdNote, anonymous);
+    notebook.removeNote(createdNoteInfo.getId(), anonymous);
   }
 
   private void patchParagraph(NotebookSocket noteSocket, String paragraphId, String patch) {
@@ -186,39 +191,45 @@ public class NotebookServerTest extends AbstractTestRestApi {
   @Test
   public void testMakeSureNoAngularObjectBroadcastToWebsocketWhoFireTheEvent()
           throws IOException, InterruptedException {
-    Note note1 = null;
+    String note1Id = null;
     try {
       // create a notebook
-      note1 = notebook.createNote("note1", anonymous);
+      note1Id = notebook.createNote("note1", anonymous);
 
       // get reference to interpreterGroup
       InterpreterGroup interpreterGroup = null;
       List<InterpreterSetting> settings = notebook.getInterpreterSettingManager().get();
       for (InterpreterSetting setting : settings) {
         if (setting.getName().equals("md")) {
-          interpreterGroup = setting.getOrCreateInterpreterGroup("anonymous", note1.getId());
+          interpreterGroup = setting.getOrCreateInterpreterGroup("anonymous", note1Id);
           break;
         }
       }
 
-      // start interpreter process
-      Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p1.setText("%md start remote interpreter process");
-      p1.setAuthenticationInfo(anonymous);
-      note1.run(p1.getId());
-
+      notebook.processNote(note1Id,
+        note1 -> {
+          // start interpreter process
+          Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p1.setText("%md start remote interpreter process");
+          p1.setAuthenticationInfo(anonymous);
+          note1.run(p1.getId());
+          return null;
+        });
+
+      Status status = notebook.processNote(note1Id, note1-> note1.getParagraph(0).getStatus());
       // wait for paragraph finished
       while (true) {
-        if (p1.getStatus() == Job.Status.FINISHED) {
+        if (status == Job.Status.FINISHED) {
           break;
         }
         Thread.sleep(100);
+        status = notebook.processNote(note1Id, note1-> note1.getParagraph(0).getStatus());
       }
       // sleep for 1 second to make sure job running thread finish to fire event. See ZEPPELIN-3277
       Thread.sleep(1000);
 
       // add angularObject
-      interpreterGroup.getAngularObjectRegistry().add("object1", "value1", note1.getId(), null);
+      interpreterGroup.getAngularObjectRegistry().add("object1", "value1", note1Id, null);
 
       // create two sockets and open it
       NotebookSocket sock1 = createWebSocket();
@@ -231,8 +242,8 @@ public class NotebookServerTest extends AbstractTestRestApi {
       notebookServer.onOpen(sock2);
       verify(sock1, times(0)).send(anyString()); // getNote, getAngularObject
       // open the same notebook from sockets
-      notebookServer.onMessage(sock1, new Message(OP.GET_NOTE).put("id", note1.getId()).toJson());
-      notebookServer.onMessage(sock2, new Message(OP.GET_NOTE).put("id", note1.getId()).toJson());
+      notebookServer.onMessage(sock1, new Message(OP.GET_NOTE).put("id", note1Id).toJson());
+      notebookServer.onMessage(sock2, new Message(OP.GET_NOTE).put("id", note1Id).toJson());
 
       reset(sock1);
       reset(sock2);
@@ -240,7 +251,7 @@ public class NotebookServerTest extends AbstractTestRestApi {
       // update object from sock1
       notebookServer.onMessage(sock1,
               new Message(OP.ANGULAR_OBJECT_UPDATED)
-                      .put("noteId", note1.getId())
+                      .put("noteId", note1Id)
                       .put("name", "object1")
                       .put("value", "value1")
                       .put("interpreterGroupId", interpreterGroup.getId()).toJson());
@@ -250,8 +261,8 @@ public class NotebookServerTest extends AbstractTestRestApi {
       verify(sock1, times(0)).send(anyString());
       verify(sock2, times(1)).send(anyString());
     } finally {
-      if (note1 != null) {
-        notebook.removeNote(note1, anonymous);
+      if (note1Id != null) {
+        notebook.removeNote(note1Id, anonymous);
       }
     }
   }
@@ -260,32 +271,38 @@ public class NotebookServerTest extends AbstractTestRestApi {
   public void testAngularObjectSaveToNote()
       throws IOException, InterruptedException {
     // create a notebook
-    Note note1 = null;
+    String note1Id = null;
     try {
-      note1 = notebook.createNote("note1", "angular", anonymous);
+      note1Id = notebook.createNote("note1", "angular", anonymous);
 
       // get reference to interpreterGroup
       InterpreterGroup interpreterGroup = null;
-      List<InterpreterSetting> settings = note1.getBindedInterpreterSettings(new ArrayList<>());
+      List<InterpreterSetting> settings = notebook.processNote(note1Id, note1-> note1.getBindedInterpreterSettings(new ArrayList<>()));
       for (InterpreterSetting setting : settings) {
         if (setting.getName().equals("angular")) {
-          interpreterGroup = setting.getOrCreateInterpreterGroup("anonymous", note1.getId());
+          interpreterGroup = setting.getOrCreateInterpreterGroup("anonymous", note1Id);
           break;
         }
       }
 
-      // start interpreter process
-      Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p1.setText("%angular <h2>Bind here : {{COMMAND_TYPE}}</h2>");
-      p1.setAuthenticationInfo(anonymous);
-      note1.run(p1.getId());
+      String p1Id = notebook.processNote(note1Id,
+        note1 -> {
+          // start interpreter process
+          Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p1.setText("%angular <h2>Bind here : {{COMMAND_TYPE}}</h2>");
+          p1.setAuthenticationInfo(anonymous);
+          note1.run(p1.getId());
+          return p1.getId();
+        });
 
       // wait for paragraph finished
+      Status status = notebook.processNote(note1Id, note1-> note1.getParagraph(p1Id).getStatus());
       while (true) {
-        if (p1.getStatus() == Job.Status.FINISHED) {
+        if (status == Job.Status.FINISHED) {
           break;
         }
         Thread.sleep(100);
+        status = notebook.processNote(note1Id, note1-> note1.getParagraph(p1Id).getStatus());
       }
       // sleep for 1 second to make sure job running thread finish to fire event. See ZEPPELIN-3277
       Thread.sleep(1000);
@@ -296,67 +313,67 @@ public class NotebookServerTest extends AbstractTestRestApi {
       notebookServer.onOpen(sock1);
       verify(sock1, times(0)).send(anyString()); // getNote, getAngularObject
       // open the same notebook from sockets
-      notebookServer.onMessage(sock1, new Message(OP.GET_NOTE).put("id", note1.getId()).toJson());
+      notebookServer.onMessage(sock1, new Message(OP.GET_NOTE).put("id", note1Id).toJson());
 
       reset(sock1);
 
       // bind object from sock1
       notebookServer.onMessage(sock1,
               new Message(OP.ANGULAR_OBJECT_CLIENT_BIND)
-                      .put("noteId", note1.getId())
-                      .put("paragraphId", p1.getId())
+                      .put("noteId", note1Id)
+                      .put("paragraphId", p1Id)
                       .put("name", "COMMAND_TYPE")
                       .put("value", "COMMAND_TYPE_VALUE")
                       .put("interpreterGroupId", interpreterGroup.getId()).toJson());
-      List<AngularObject> list = note1.getAngularObjects("angular-shared_process");
-      assertEquals(list.size(), 1);
-      assertEquals(list.get(0).getNoteId(), note1.getId());
-      assertEquals(list.get(0).getParagraphId(), p1.getId());
-      assertEquals(list.get(0).getName(), "COMMAND_TYPE");
-      assertEquals(list.get(0).get(), "COMMAND_TYPE_VALUE");
+      List<AngularObject> list = notebook.processNote(note1Id, note1-> note1.getAngularObjects("angular-shared_process"));
+      assertEquals(1, list.size());
+      assertEquals(note1Id, list.get(0).getNoteId());
+      assertEquals(p1Id, list.get(0).getParagraphId());
+      assertEquals("COMMAND_TYPE", list.get(0).getName());
+      assertEquals("COMMAND_TYPE_VALUE", list.get(0).get());
       // Check if the interpreterGroup AngularObjectRegistry is updated
       Map<String, Map<String, AngularObject>> mapRegistry = interpreterGroup.getAngularObjectRegistry().getRegistry();
-      AngularObject ao = mapRegistry.get(note1.getId() + "_" + p1.getId()).get("COMMAND_TYPE");
-      assertEquals(ao.getName(), "COMMAND_TYPE");
-      assertEquals(ao.get(), "COMMAND_TYPE_VALUE");
+      AngularObject ao = mapRegistry.get(note1Id + "_" + p1Id).get("COMMAND_TYPE");
+      assertEquals("COMMAND_TYPE", ao.getName());
+      assertEquals("COMMAND_TYPE_VALUE", ao.get());
 
       // update bind object from sock1
       notebookServer.onMessage(sock1,
               new Message(OP.ANGULAR_OBJECT_UPDATED)
-                      .put("noteId", note1.getId())
-                      .put("paragraphId", p1.getId())
+                      .put("noteId", note1Id)
+                      .put("paragraphId", p1Id)
                       .put("name", "COMMAND_TYPE")
                       .put("value", "COMMAND_TYPE_VALUE_UPDATE")
                       .put("interpreterGroupId", interpreterGroup.getId()).toJson());
-      list = note1.getAngularObjects("angular-shared_process");
-      assertEquals(list.size(), 1);
-      assertEquals(list.get(0).getNoteId(), note1.getId());
-      assertEquals(list.get(0).getParagraphId(), p1.getId());
-      assertEquals(list.get(0).getName(), "COMMAND_TYPE");
-      assertEquals(list.get(0).get(), "COMMAND_TYPE_VALUE_UPDATE");
+      list = notebook.processNote(note1Id, note1-> note1.getAngularObjects("angular-shared_process"));
+      assertEquals(1, list.size());
+      assertEquals(note1Id, list.get(0).getNoteId());
+      assertEquals(p1Id, list.get(0).getParagraphId());
+      assertEquals("COMMAND_TYPE", list.get(0).getName());
+      assertEquals("COMMAND_TYPE_VALUE_UPDATE", list.get(0).get());
       // Check if the interpreterGroup AngularObjectRegistry is updated
       mapRegistry = interpreterGroup.getAngularObjectRegistry().getRegistry();
-      AngularObject ao1 = mapRegistry.get(note1.getId() + "_" + p1.getId()).get("COMMAND_TYPE");
-      assertEquals(ao1.getName(), "COMMAND_TYPE");
-      assertEquals(ao1.get(), "COMMAND_TYPE_VALUE_UPDATE");
+      AngularObject ao1 = mapRegistry.get(note1Id + "_" + p1Id).get("COMMAND_TYPE");
+      assertEquals("COMMAND_TYPE", ao1.getName());
+      assertEquals("COMMAND_TYPE_VALUE_UPDATE", ao1.get());
 
       // unbind object from sock1
       notebookServer.onMessage(sock1,
               new Message(OP.ANGULAR_OBJECT_CLIENT_UNBIND)
-                      .put("noteId", note1.getId())
-                      .put("paragraphId", p1.getId())
+                      .put("noteId", note1Id)
+                      .put("paragraphId", p1Id)
                       .put("name", "COMMAND_TYPE")
                       .put("value", "COMMAND_TYPE_VALUE")
                       .put("interpreterGroupId", interpreterGroup.getId()).toJson());
-      list = note1.getAngularObjects("angular-shared_process");
-      assertEquals(list.size(), 0);
+      list = notebook.processNote(note1Id, note1-> note1.getAngularObjects("angular-shared_process"));
+      assertEquals(0, list.size());
       // Check if the interpreterGroup AngularObjectRegistry is delete
       mapRegistry = interpreterGroup.getAngularObjectRegistry().getRegistry();
-      AngularObject ao2 = mapRegistry.get(note1.getId() + "_" + p1.getId()).get("COMMAND_TYPE");
+      AngularObject ao2 = mapRegistry.get(note1Id + "_" + p1Id).get("COMMAND_TYPE");
       assertNull(ao2);
     } finally {
-      if (note1 != null) {
-        notebook.removeNote(note1, anonymous);
+      if (note1Id != null) {
+        notebook.removeNote(note1Id, anonymous);
       }
     }
   }
@@ -364,39 +381,51 @@ public class NotebookServerTest extends AbstractTestRestApi {
   @Test
   public void testLoadAngularObjectFromNote() throws IOException, InterruptedException {
     // create a notebook
-    Note note1 = null;
+    String note1Id = null;
     try {
-      note1 = notebook.createNote("note1", anonymous);
+      note1Id = notebook.createNote("note1", anonymous);
 
       // get reference to interpreterGroup
       InterpreterGroup interpreterGroup = null;
       List<InterpreterSetting> settings = notebook.getInterpreterSettingManager().get();
       for (InterpreterSetting setting : settings) {
         if (setting.getName().equals("angular")) {
-          interpreterGroup = setting.getOrCreateInterpreterGroup("anonymous", note1.getId());
+          interpreterGroup = setting.getOrCreateInterpreterGroup("anonymous", note1Id);
           break;
         }
       }
+      String p1Id = notebook.processNote(note1Id,
+        note1 -> {
+          // start interpreter process
+          Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          p1.setText("%angular <h2>Bind here : {{COMMAND_TYPE}}</h2>");
+          p1.setAuthenticationInfo(anonymous);
+          note1.run(p1.getId());
+          return p1.getId();
+        });
 
-      // start interpreter process
-      Paragraph p1 = note1.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      p1.setText("%angular <h2>Bind here : {{COMMAND_TYPE}}</h2>");
-      p1.setAuthenticationInfo(anonymous);
-      note1.run(p1.getId());
 
       // wait for paragraph finished
+      Status status = notebook.processNote(note1Id, note1-> note1.getParagraph(p1Id).getStatus());
       while (true) {
-        if (p1.getStatus() == Job.Status.FINISHED) {
+        System.out.println("loop");
+        if (status == Job.Status.FINISHED) {
           break;
         }
         Thread.sleep(100);
+        status = notebook.processNote(note1Id, note1-> note1.getParagraph(p1Id).getStatus());
       }
       // sleep for 1 second to make sure job running thread finish to fire event. See ZEPPELIN-3277
       Thread.sleep(1000);
 
       // set note AngularObject
-      AngularObject ao = new AngularObject("COMMAND_TYPE", "COMMAND_TYPE_VALUE", note1.getId(), p1.getId(), null);
-      note1.addOrUpdateAngularObject("angular-shared_process", ao);
+      AngularObject ao = new AngularObject("COMMAND_TYPE", "COMMAND_TYPE_VALUE", note1Id, p1Id, null);
+      notebook.processNote(note1Id,
+        note1 -> {
+          note1.addOrUpdateAngularObject("angular-shared_process", ao);
+          return null;
+        });
+
 
       // create sockets and open it
       NotebookSocket sock1 = createWebSocket();
@@ -404,21 +433,21 @@ public class NotebookServerTest extends AbstractTestRestApi {
 
       // Check the AngularObjectRegistry of the interpreterGroup before executing GET_NOTE
       Map<String, Map<String, AngularObject>> mapRegistry1 = interpreterGroup.getAngularObjectRegistry().getRegistry();
-      assertEquals(mapRegistry1.size(), 0);
+      assertEquals(0, mapRegistry1.size());
 
       // open the notebook from sockets, AngularObjectRegistry that triggers the update of the interpreterGroup
-      notebookServer.onMessage(sock1, new Message(OP.GET_NOTE).put("id", note1.getId()).toJson());
+      notebookServer.onMessage(sock1, new Message(OP.GET_NOTE).put("id", note1Id).toJson());
       Thread.sleep(1000);
 
       // After executing GET_NOTE, check the AngularObjectRegistry of the interpreterGroup
       Map<String, Map<String, AngularObject>> mapRegistry2 = interpreterGroup.getAngularObjectRegistry().getRegistry();
-      assertEquals(mapRegistry1.size(), 2);
-      AngularObject ao1 = mapRegistry2.get(note1.getId() + "_" + p1.getId()).get("COMMAND_TYPE");
-      assertEquals(ao1.getName(), "COMMAND_TYPE");
-      assertEquals(ao1.get(), "COMMAND_TYPE_VALUE");
+      assertEquals(2, mapRegistry1.size());
+      AngularObject ao1 = mapRegistry2.get(note1Id + "_" + p1Id).get("COMMAND_TYPE");
+      assertEquals("COMMAND_TYPE", ao1.getName());
+      assertEquals("COMMAND_TYPE_VALUE", ao1.get());
     } finally {
-      if (note1 != null) {
-        notebook.removeNote(note1, anonymous);
+      if (note1Id != null) {
+        notebook.removeNote(note1Id, anonymous);
       }
     }
   }
@@ -432,54 +461,65 @@ public class NotebookServerTest extends AbstractTestRestApi {
         "\"name\": \"Test Zeppelin notebook import\",\"config\": " +
         "{}}}}";
     Message messageReceived = notebookServer.deserializeMessage(msg);
-    Note note = null;
+    String noteId = null;
     ServiceContext context = new ServiceContext(AuthenticationInfo.ANONYMOUS, new HashSet<>());
     try {
       try {
-        note = notebookServer.importNote(null, context, messageReceived);
+        noteId = notebookServer.importNote(null, context, messageReceived);
       } catch (NullPointerException e) {
         //broadcastNoteList(); failed nothing to worry.
         LOG.error("Exception in NotebookServerTest while testImportNotebook, failed nothing to " +
                 "worry ", e);
       }
 
-      assertNotEquals(null, notebook.getNote(note.getId()));
-      assertEquals("Test Zeppelin notebook import", notebook.getNote(note.getId()).getName());
-      assertEquals("Test paragraphs import", notebook.getNote(note.getId()).getParagraphs().get(0)
-              .getText());
+      notebook.processNote(noteId,
+        note -> {
+          assertNotNull(note);
+          assertEquals("Test Zeppelin notebook import", note.getName());
+          assertEquals("Test paragraphs import", note.getParagraphs().get(0)
+            .getText());
+          return null;
+        });
+
     } finally {
-      if (note != null) {
-        notebook.removeNote(note, anonymous);
+      if (noteId != null) {
+        notebook.removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testImportJupyterNote() throws IOException {
-    String jupyterNoteJson = IOUtils.toString(getClass().getResourceAsStream("/Lecture-4.ipynb"));
+    String jupyterNoteJson = IOUtils.toString(getClass().getResourceAsStream("/Lecture-4.ipynb"), StandardCharsets.UTF_8);
     String msg = "{\"op\":\"IMPORT_NOTE\",\"data\":" +
             "{\"note\": " + jupyterNoteJson + "}}";
     Message messageReceived = notebookServer.deserializeMessage(msg);
-    Note note = null;
+    String noteId = null;
     ServiceContext context = new ServiceContext(AuthenticationInfo.ANONYMOUS, new HashSet<>());
     try {
       try {
-        note = notebookServer.importNote(null, context, messageReceived);
+        noteId = notebookServer.importNote(null, context, messageReceived);
       } catch (NullPointerException e) {
         //broadcastNoteList(); failed nothing to worry.
         LOG.error("Exception in NotebookServerTest while testImportJupyterNote, failed nothing to " +
                 "worry ", e);
       }
 
-      assertNotEquals(null, notebook.getNote(note.getId()));
-      assertTrue(notebook.getNote(note.getId()).getName(),
-              notebook.getNote(note.getId()).getName().startsWith("Note converted from Jupyter_"));
-      assertEquals("md", notebook.getNote(note.getId()).getParagraphs().get(0).getIntpText());
-      assertEquals("\n# matplotlib - 2D and 3D plotting in Python",
-              notebook.getNote(note.getId()).getParagraphs().get(0).getScriptText());
+      notebook.processNote(noteId,
+        note -> {
+          assertNotNull(note);
+          assertTrue(note.getName(), note.getName().startsWith("Note converted from Jupyter_"));
+          assertEquals("md", note.getParagraphs().get(0).getIntpText());
+          assertEquals("\n# matplotlib - 2D and 3D plotting in Python",
+            note.getParagraphs().get(0).getScriptText());
+          return null;
+        });
+
+
+
     } finally {
-      if (note != null) {
-        notebook.removeNote(note, anonymous);
+      if (noteId != null) {
+        notebook.removeNote(noteId, anonymous);
       }
     }
   }
@@ -501,7 +541,7 @@ public class NotebookServerTest extends AbstractTestRestApi {
       notebookServer.setNotebookService(() -> notebookService);
       final Note note = mock(Note.class, RETURNS_DEEP_STUBS);
 
-      when(notebook.getNote("noteId")).thenReturn(note);
+      when(notebook.processNote(eq("noteId"), Mockito.any())).then(e -> e.getArgumentAt(1,NoteProcessor.class).process(note));
       final Paragraph paragraph = mock(Paragraph.class, RETURNS_DEEP_STUBS);
       when(note.getParagraph("paragraphId")).thenReturn(paragraph);
 
@@ -557,7 +597,7 @@ public class NotebookServerTest extends AbstractTestRestApi {
       notebookServer.setNotebook(() -> notebook);
       notebookServer.setNotebookService(() -> notebookService);
       final Note note = mock(Note.class, RETURNS_DEEP_STUBS);
-      when(notebook.getNote("noteId")).thenReturn(note);
+      when(notebook.processNote(eq("noteId"), Mockito.any())).then(e -> e.getArgumentAt(1,NoteProcessor.class).process(note));
       final Paragraph paragraph = mock(Paragraph.class, RETURNS_DEEP_STUBS);
       when(note.getParagraph("paragraphId")).thenReturn(paragraph);
 
@@ -626,19 +666,20 @@ public class NotebookServerTest extends AbstractTestRestApi {
     // expect the events are broadcasted properly
     verify(sock1, times(sendCount)).send(anyString());
 
-    Note createdNote = null;
-    for (Note note : notebook.getAllNotes()) {
-      if (note.getName().equals(noteName)) {
-        createdNote = note;
+    String createdNoteId = null;
+    for (NoteInfo noteInfo : notebook.getNotesInfo()) {
+      ;
+      if (notebook.processNote(noteInfo.getId(), Note::getName).equals(noteName)) {
+        createdNoteId = noteInfo.getId();
         break;
       }
     }
 
     if (settings.size() > 1) {
       assertEquals(notebook.getInterpreterSettingManager().getDefaultInterpreterSetting(
-              createdNote.getId()).getId(), defaultInterpreterId);
+        createdNoteId).getId(), defaultInterpreterId);
     }
-    notebook.removeNote(createdNote, anonymous);
+    notebook.removeNote(createdNoteId, anonymous);
   }
 
   @Test
@@ -651,10 +692,10 @@ public class NotebookServerTest extends AbstractTestRestApi {
         "\"name\": \"Test RuntimeInfos\",\"config\": " +
         "{}}}}";
     Message messageReceived = notebookServer.deserializeMessage(msg);
-    Note note = null;
+    String noteId = null;
     ServiceContext context = new ServiceContext(AuthenticationInfo.ANONYMOUS, new HashSet<>());
     try {
-      note = notebookServer.importNote(null, context, messageReceived);
+      noteId = notebookServer.importNote(null, context, messageReceived);
     } catch (NullPointerException e) {
       //broadcastNoteList(); failed nothing to worry.
       LOG.error("Exception in NotebookServerTest while testImportNotebook, failed nothing to " +
@@ -662,47 +703,52 @@ public class NotebookServerTest extends AbstractTestRestApi {
     } catch (IOException e) {
       e.printStackTrace();
     }
-
-    assertNotEquals(null, notebook.getNote(note.getId()));
-    assertNotEquals(null, note.getParagraph(0));
-
-    String nodeId = note.getId();
-    String paragraphId = note.getParagraph(0).getId();
-
     // update RuntimeInfos
     Map<String, String> infos = new java.util.HashMap<>();
-    infos.put("jobUrl", "jobUrl_value");
-    infos.put("jobLabel", "jobLabel_value");
-    infos.put("label", "SPARK JOB");
-    infos.put("tooltip", "View in Spark web UI");
-    infos.put("noteId", nodeId);
-    infos.put("paraId", paragraphId);
-
-    notebookServer.onParaInfosReceived(nodeId, paragraphId, "spark", infos);
-    Paragraph paragraph = note.getParagraph(paragraphId);
-
-    // check RuntimeInfos
-    assertTrue(paragraph.getRuntimeInfos().containsKey("jobUrl"));
-    List<Object> list = paragraph.getRuntimeInfos().get("jobUrl").getValue();
-    assertEquals(1, list.size());
-    Map<String, String> map = (Map<String, String>) list.get(0);
-    assertEquals(2, map.size());
-    assertEquals(map.get("jobUrl"), "jobUrl_value");
-    assertEquals(map.get("jobLabel"), "jobLabel_value");
+    String paragraphId = notebook.processNote(noteId,
+      note -> {
+        assertNotNull(note);
+        assertNotNull(note.getParagraph(0));
+        infos.put("jobUrl", "jobUrl_value");
+        infos.put("jobLabel", "jobLabel_value");
+        infos.put("label", "SPARK JOB");
+        infos.put("tooltip", "View in Spark web UI");
+        infos.put("noteId", note.getId());
+        infos.put("paraId", note.getParagraph(0).getId());
+        return note.getParagraph(0).getId();
+      });
+
+    notebookServer.onParaInfosReceived(noteId, paragraphId, "spark", infos);
+    notebook.processNote(noteId,
+      note -> {
+        Paragraph paragraph = note.getParagraph(paragraphId);
+        // check RuntimeInfos
+        assertTrue(paragraph.getRuntimeInfos().containsKey("jobUrl"));
+        List<Object> list = paragraph.getRuntimeInfos().get("jobUrl").getValue();
+        assertEquals(1, list.size());
+        Map<String, String> map = (Map<String, String>) list.get(0);
+        assertEquals(2, map.size());
+        assertEquals(map.get("jobUrl"), "jobUrl_value");
+        assertEquals(map.get("jobLabel"), "jobLabel_value");
+        return null;
+      });
   }
 
   @Test
   public void testGetParagraphList() throws IOException {
-    Note note = null;
+    String noteId = null;
 
     try {
-      note = notebook.createNote("note1", anonymous);
-      Paragraph p1 = note.addNewParagraph(anonymous);
-      p1.setText("%md start remote interpreter process");
-      p1.setAuthenticationInfo(anonymous);
-      notebook.saveNote(note, anonymous);
+      noteId = notebook.createNote("note1", anonymous);
+      notebook.processNote(noteId,
+        note -> {
+          Paragraph p1 = note.addNewParagraph(anonymous);
+          p1.setText("%md start remote interpreter process");
+          p1.setAuthenticationInfo(anonymous);
+          notebook.saveNote(note, anonymous);
+          return null;
+        });
 
-      String noteId = note.getId();
       String user1Id = "user1", user2Id = "user2";
 
       // test user1 can get anonymous's note
@@ -746,44 +792,49 @@ public class NotebookServerTest extends AbstractTestRestApi {
       }
       assertNotNull(user1Id + " can get " + user2Id + "'s shared note", paragraphList2);
     } finally {
-      if (null != note) {
-        notebook.removeNote(note, anonymous);
+      if (null != noteId) {
+        notebook.removeNote(noteId, anonymous);
       }
     }
   }
 
   @Test
   public void testNoteRevision() throws IOException {
-    Note note = null;
+    String noteId = null;
 
     try {
-      note = notebook.createNote("note1", anonymous);
-      assertEquals(0, note.getParagraphCount());
-      NotebookRepoWithVersionControl.Revision firstRevision = notebook.checkpointNote(note.getId(), note.getPath(), "first commit", AuthenticationInfo.ANONYMOUS);
-      List<NotebookRepoWithVersionControl.Revision> revisionList = notebook.listRevisionHistory(note.getId(), note.getPath(), AuthenticationInfo.ANONYMOUS);
-      assertEquals(1, revisionList.size());
-      assertEquals(firstRevision.id, revisionList.get(0).id);
-      assertEquals("first commit", revisionList.get(0).message);
-
-      // add one new paragraph and commit it
-      note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
-      notebook.saveNote(note, AuthenticationInfo.ANONYMOUS);
-      assertEquals(1, note.getParagraphCount());
-      NotebookRepoWithVersionControl.Revision secondRevision = notebook.checkpointNote(note.getId(), note.getPath(), "second commit", AuthenticationInfo.ANONYMOUS);
-
-      revisionList = notebook.listRevisionHistory(note.getId(), note.getPath(), AuthenticationInfo.ANONYMOUS);
-      assertEquals(2, revisionList.size());
-      assertEquals(secondRevision.id, revisionList.get(0).id);
-      assertEquals("second commit", revisionList.get(0).message);
-      assertEquals(firstRevision.id, revisionList.get(1).id);
-      assertEquals("first commit", revisionList.get(1).message);
-
-      // checkout the first commit
-      note = notebook.getNoteByRevision(note.getId(), note.getPath(), firstRevision.id, AuthenticationInfo.ANONYMOUS);
-      assertEquals(0, note.getParagraphCount());
+      noteId = notebook.createNote("note1", anonymous);
+      notebook.processNote(noteId,
+        note -> {
+          assertEquals(0, note.getParagraphCount());
+          NotebookRepoWithVersionControl.Revision firstRevision = notebook.checkpointNote(note.getId(), note.getPath(), "first commit", AuthenticationInfo.ANONYMOUS);
+          List<NotebookRepoWithVersionControl.Revision> revisionList = notebook.listRevisionHistory(note.getId(), note.getPath(), AuthenticationInfo.ANONYMOUS);
+          assertEquals(1, revisionList.size());
+          assertEquals(firstRevision.id, revisionList.get(0).id);
+          assertEquals("first commit", revisionList.get(0).message);
+
+          // add one new paragraph and commit it
+          note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
+          notebook.saveNote(note, AuthenticationInfo.ANONYMOUS);
+          assertEquals(1, note.getParagraphCount());
+          NotebookRepoWithVersionControl.Revision secondRevision = notebook.checkpointNote(note.getId(), note.getPath(), "second commit", AuthenticationInfo.ANONYMOUS);
+
+          revisionList = notebook.listRevisionHistory(note.getId(), note.getPath(), AuthenticationInfo.ANONYMOUS);
+          assertEquals(2, revisionList.size());
+          assertEquals(secondRevision.id, revisionList.get(0).id);
+          assertEquals("second commit", revisionList.get(0).message);
+          assertEquals(firstRevision.id, revisionList.get(1).id);
+          assertEquals("first commit", revisionList.get(1).message);
+
+          // checkout the first commit
+          note = notebook.getNoteByRevision(note.getId(), note.getPath(), firstRevision.id, AuthenticationInfo.ANONYMOUS);
+          assertEquals(0, note.getParagraphCount());
+          return null;
+        });
+
     } finally {
-      if (null != note) {
-        notebook.removeNote(note, anonymous);
+      if (null != noteId) {
+        notebook.removeNote(noteId, anonymous);
       }
     }
   }
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 c558c37..ad10e11 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
@@ -371,42 +371,39 @@ public class HeliumApplicationFactory implements ApplicationEventListener, NoteE
     if (notebook == null) {
       return null;
     }
-
-    Note note = null;
     try {
-      note = notebook.getNote(noteId);
-      if (note == null) {
-        logger.warn("Note " + noteId + " not found");
-        return null;
-      }
+      return notebook.processNote(noteId,
+        note -> {
+          if (note == null) {
+            logger.warn("Note {} not found", noteId);
+            return null;
+          }
+          Paragraph paragraph = note.getParagraph(paragraphId);
+          if (paragraph == null) {
+            logger.error("Can't get paragraph {}", paragraphId);
+            return null;
+          }
+          return paragraph.getApplicationState(appId);
+        });
     } catch (IOException e) {
       logger.error("Can't get note {}", noteId);
       return null;
     }
-
-    Paragraph paragraph = note.getParagraph(paragraphId);
-    if (paragraph == null) {
-      logger.error("Can't get paragraph {}", paragraphId);
-      return null;
-    }
-
-    ApplicationState appFound = paragraph.getApplicationState(appId);
-
-    return appFound;
   }
 
   @Override
-  public void onNoteRemove(Note note, AuthenticationInfo subject) throws IOException {
+  public void onNoteRemove(Note note, AuthenticationInfo subject) {
+    // do nothing
   }
 
   @Override
-  public void onNoteCreate(Note note, AuthenticationInfo subject) throws IOException {
-
+  public void onNoteCreate(Note note, AuthenticationInfo subject) {
+    // do nothing
   }
 
   @Override
-  public void onNoteUpdate(Note note, AuthenticationInfo subject) throws IOException {
-
+  public void onNoteUpdate(Note note, AuthenticationInfo subject) {
+    // do nothing
   }
 
   @Override
@@ -420,12 +417,12 @@ public class HeliumApplicationFactory implements ApplicationEventListener, NoteE
 
   @Override
   public void onParagraphCreate(Paragraph p) {
-
+    // do nothing
   }
 
   @Override
-  public void onParagraphUpdate(Paragraph p) throws IOException {
-
+  public void onParagraphUpdate(Paragraph p) {
+    // do nothing
   }
 
   @Override
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSetting.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSetting.java
index bff9273..9ec613c 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSetting.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSetting.java
@@ -1206,16 +1206,19 @@ public class InterpreterSetting {
 
   private ExecutionContext getExecutionContext(String user, String noteId) {
     try {
-      Note note = getInterpreterSettingManager().getNotebook().getNote(noteId);
-      if (note == null) {
-        throw new RuntimeException("No such note: " + noteId);
-      } else {
-        ExecutionContext context = note.getExecutionContext();
-        context.setUser(user);
-        return context;
-      }
+      return getInterpreterSettingManager().getNotebook().processNote(noteId,
+        note -> {
+          if (note == null) {
+            throw new RuntimeException("No such note: " + noteId);
+          } else {
+            ExecutionContext context = note.getExecutionContext();
+            context.setUser(user);
+            return context;
+          }
+        });
     } catch (IOException e) {
       throw new RuntimeException("Fail to getExecutionContext", e);
     }
+
   }
 }
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSettingManager.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSettingManager.java
index b3cd8ae..6d30006 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSettingManager.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterSettingManager.java
@@ -544,8 +544,10 @@ public class InterpreterSettingManager implements NoteEventListener, ClusterEven
 
   public InterpreterSetting getDefaultInterpreterSetting(String noteId) {
     try {
-      Note note = notebook.getNote(noteId);
-      InterpreterSetting interpreterSetting = interpreterSettings.get(note.getDefaultInterpreterGroup());
+      InterpreterSetting interpreterSetting = notebook.processNote(noteId,
+        note -> {
+          return interpreterSettings.get(note.getDefaultInterpreterGroup());
+        });
       if (interpreterSetting == null) {
         interpreterSetting = get().get(0);
       }
@@ -957,11 +959,13 @@ public class InterpreterSettingManager implements NoteEventListener, ClusterEven
   // restart in note page
   public void restart(String settingId, String user, String noteId) throws InterpreterException {
     try {
-      Note note = notebook.getNote(noteId);
-      if (note == null) {
-        throw new InterpreterException("No such note: " + noteId);
-      }
-      ExecutionContext executionContext = note.getExecutionContext();
+      ExecutionContext executionContext = notebook.processNote(noteId,
+        note -> {
+          if (note == null) {
+            throw new IOException("No such note: " + noteId);
+          }
+          return note.getExecutionContext();
+        });
       executionContext.setUser(user);
       restart(settingId, executionContext);
     } catch (IOException e) {
@@ -1116,7 +1120,7 @@ public class InterpreterSettingManager implements NoteEventListener, ClusterEven
   }
 
   @Override
-  public void onNoteRemove(Note note, AuthenticationInfo subject) throws IOException {
+  public void onNoteRemove(Note note, AuthenticationInfo subject) {
     // stop all associated interpreters
     if (note.getParagraphs() != null) {
       for (Paragraph paragraph : note.getParagraphs()) {
@@ -1181,40 +1185,38 @@ public class InterpreterSettingManager implements NoteEventListener, ClusterEven
   }
 
   @Override
-  public void onNoteCreate(Note note, AuthenticationInfo subject) throws IOException {
-
+  public void onNoteCreate(Note note, AuthenticationInfo subject) {
+    // do nothing
   }
 
   @Override
-  public void onNoteUpdate(Note note, AuthenticationInfo subject) throws IOException {
-
+  public void onNoteUpdate(Note note, AuthenticationInfo subject) {
+    // do nothing
   }
 
   @Override
-  public void onParagraphRemove(Paragraph p) throws IOException {
-
+  public void onParagraphRemove(Paragraph p) {
+    // do nothing
   }
 
   @Override
-  public void onParagraphCreate(Paragraph p) throws IOException {
-
+  public void onParagraphCreate(Paragraph p) {
+    // do nothing
   }
 
   @Override
-  public void onParagraphUpdate(Paragraph p) throws IOException {
-
+  public void onParagraphUpdate(Paragraph p) {
+    // do nothing
   }
 
   @Override
-  public void onParagraphStatusChange(Paragraph p, Job.Status status) throws IOException {
-
+  public void onParagraphStatusChange(Paragraph p, Job.Status status) {
+    // do nothing
   }
 
   @Override
   public void onClusterEvent(String msg) {
-    if (LOGGER.isDebugEnabled()) {
-      LOGGER.debug("onClusterEvent : {}", msg);
-    }
+    LOGGER.debug("onClusterEvent : {}", msg);
 
     try {
       ClusterMessage message = ClusterMessage.deserializeMessage(msg);
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/RemoteInterpreterEventServer.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/RemoteInterpreterEventServer.java
index b8cd73f..d2649ab 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/RemoteInterpreterEventServer.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/RemoteInterpreterEventServer.java
@@ -298,11 +298,14 @@ public class RemoteInterpreterEventServer implements RemoteInterpreterEventServi
 
     if (angularObject.getNoteId() != null) {
       try {
-        Note note = interpreterSettingManager.getNotebook().getNote(angularObject.getNoteId());
-        if (note != null) {
-          note.addOrUpdateAngularObject(intpGroupId, angularObject);
-          interpreterSettingManager.getNotebook().saveNote(note, AuthenticationInfo.ANONYMOUS);
-        }
+        interpreterSettingManager.getNotebook().processNote(angularObject.getNoteId(),
... 6621 lines suppressed ...