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

[1/2] incubator-zeppelin git commit: [ZEPPELIN-713] Expand ResourceId and put InterpreterResult into ResourcePool

Repository: incubator-zeppelin
Updated Branches:
  refs/heads/master e2122bc9c -> 85a2ad376


http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourceSet.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourceSet.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourceSet.java
index a03655b..bbc3f06 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourceSet.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourceSet.java
@@ -72,4 +72,34 @@ public class ResourceSet extends LinkedList<Resource> {
     }
     return result;
   }
+
+  public ResourceSet filterByNoteId(String noteId) {
+    ResourceSet result = new ResourceSet();
+    for (Resource r : this) {
+      if (equals(r.getResourceId().getNoteId(), noteId)) {
+        result.add(r);
+      }
+    }
+    return result;
+  }
+
+  public ResourceSet filterByParagraphId(String paragraphId) {
+    ResourceSet result = new ResourceSet();
+    for (Resource r : this) {
+      if (equals(r.getResourceId().getParagraphId(), paragraphId)) {
+        result.add(r);
+      }
+    }
+    return result;
+  }
+
+  private boolean equals(String a, String b) {
+    if (a == null && b == null) {
+      return true;
+    } else if (a != null && b != null) {
+      return a.equals(b);
+    } else {
+      return false;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/WellKnownResourceName.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/WellKnownResourceName.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/WellKnownResourceName.java
new file mode 100644
index 0000000..2d14fd4
--- /dev/null
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/WellKnownResourceName.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.zeppelin.resource;
+
+/**
+ * Well known resource names in ResourcePool
+ */
+public enum WellKnownResourceName {
+  ParagraphResult("zeppelin.paragraph.result");     // paragraph run result
+
+  String name;
+  WellKnownResourceName(String name) {
+    this.name = name;
+  }
+
+  public String toString() {
+    return name;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/main/thrift/RemoteInterpreterService.thrift
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/thrift/RemoteInterpreterService.thrift b/zeppelin-interpreter/src/main/thrift/RemoteInterpreterService.thrift
index 224433d..3a70caa 100644
--- a/zeppelin-interpreter/src/main/thrift/RemoteInterpreterService.thrift
+++ b/zeppelin-interpreter/src/main/thrift/RemoteInterpreterService.thrift
@@ -78,7 +78,9 @@ service RemoteInterpreterService {
   // get all resources in the interpreter process
   list<string> resoucePoolGetAll();
   // get value of resource
-  binary resourceGet(1: string resourceName);
+  binary resourceGet(1: string noteId, 2: string paragraphId, 3: string resourceName);
+  // remove resource
+  bool resourceRemove(1: string noteId, 2: string paragraphId, 3:string resourceName);
 
   void angularObjectUpdate(1: string name, 2: string noteId, 3: string paragraphId, 4: string
   object);

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/test/java/org/apache/zeppelin/interpreter/remote/mock/MockInterpreterResourcePool.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/test/java/org/apache/zeppelin/interpreter/remote/mock/MockInterpreterResourcePool.java b/zeppelin-interpreter/src/test/java/org/apache/zeppelin/interpreter/remote/mock/MockInterpreterResourcePool.java
index 1db68ad..3826b90 100644
--- a/zeppelin-interpreter/src/test/java/org/apache/zeppelin/interpreter/remote/mock/MockInterpreterResourcePool.java
+++ b/zeppelin-interpreter/src/test/java/org/apache/zeppelin/interpreter/remote/mock/MockInterpreterResourcePool.java
@@ -29,6 +29,7 @@ import org.apache.zeppelin.interpreter.InterpreterContext;
 import org.apache.zeppelin.interpreter.InterpreterPropertyBuilder;
 import org.apache.zeppelin.interpreter.InterpreterResult;
 import org.apache.zeppelin.interpreter.InterpreterResult.Code;
+import org.apache.zeppelin.resource.Resource;
 import org.apache.zeppelin.resource.ResourcePool;
 
 public class MockInterpreterResourcePool extends Interpreter {
@@ -61,9 +62,18 @@ public class MockInterpreterResourcePool extends Interpreter {
   public InterpreterResult interpret(String st, InterpreterContext context) {
     String[] stmt = st.split(" ");
     String cmd = stmt[0];
+    String noteId = null;
+    String paragraphId = null;
     String name = null;
     if (stmt.length >= 2) {
-      name = stmt[1];
+      String[] npn = stmt[1].split(":");
+      if (npn.length == 3) {
+        noteId = npn[0];
+        paragraphId = npn[1];
+        name = npn[2];
+      } else {
+        name = stmt[1];
+      }
     }
     String value = null;
     if (stmt.length == 3) {
@@ -73,11 +83,16 @@ public class MockInterpreterResourcePool extends Interpreter {
     ResourcePool resourcePool = context.getResourcePool();
     Object ret = null;
     if (cmd.equals("put")) {
-      resourcePool.put(name, value);
+      resourcePool.put(noteId, paragraphId, name, value);
     } else if (cmd.equalsIgnoreCase("get")) {
-      ret = resourcePool.get(name).get();
+      Resource resource = resourcePool.get(noteId, paragraphId, name);
+      if (resource != null) {
+        ret = resourcePool.get(noteId, paragraphId, name).get();
+      } else {
+        ret = "";
+      }
     } else if (cmd.equals("remove")) {
-      ret = resourcePool.remove(name);
+      ret = resourcePool.remove(noteId, paragraphId, name);
     } else if (cmd.equals("getAll")) {
       ret = resourcePool.getAll();
     }

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/test/java/org/apache/zeppelin/resource/DistributedResourcePoolTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/test/java/org/apache/zeppelin/resource/DistributedResourcePoolTest.java b/zeppelin-interpreter/src/test/java/org/apache/zeppelin/resource/DistributedResourcePoolTest.java
index a99fde2..e49b437 100644
--- a/zeppelin-interpreter/src/test/java/org/apache/zeppelin/resource/DistributedResourcePoolTest.java
+++ b/zeppelin-interpreter/src/test/java/org/apache/zeppelin/resource/DistributedResourcePoolTest.java
@@ -136,12 +136,13 @@ public class DistributedResourcePoolTest {
     InterpreterResult ret;
     intp1.interpret("put key1 value1", context);
     intp2.interpret("put key2 value2", context);
+    int numInterpreterResult = 2;
 
     ret = intp1.interpret("getAll", context);
-    assertEquals(2, gson.fromJson(ret.message(), ResourceSet.class).size());
+    assertEquals(numInterpreterResult + 2, gson.fromJson(ret.message(), ResourceSet.class).size());
 
     ret = intp2.interpret("getAll", context);
-    assertEquals(2, gson.fromJson(ret.message(), ResourceSet.class).size());
+    assertEquals(numInterpreterResult + 2, gson.fromJson(ret.message(), ResourceSet.class).size());
 
     ret = intp1.interpret("get key1", context);
     assertEquals("value1", gson.fromJson(ret.message(), String.class));
@@ -201,4 +202,44 @@ public class DistributedResourcePoolTest {
     assertEquals("value1", pool1.getAll().get(0).get());
     assertEquals("value2", pool1.getAll().get(1).get());
   }
+
+  @Test
+  public void testResourcePoolUtils() {
+    Gson gson = new Gson();
+    InterpreterResult ret;
+
+    // when create some resources
+    intp1.interpret("put note1:paragraph1:key1 value1", context);
+    intp1.interpret("put note1:paragraph2:key1 value2", context);
+    intp2.interpret("put note2:paragraph1:key1 value1", context);
+    intp2.interpret("put note2:paragraph2:key2 value2", context);
+
+    int numInterpreterResult = 2;
+
+    // then get all resources.
+    assertEquals(numInterpreterResult + 4, ResourcePoolUtils.getAllResources().size());
+
+    // when remove all resources from note1
+    ResourcePoolUtils.removeResourcesBelongsToNote("note1");
+
+    // then resources should be removed.
+    assertEquals(numInterpreterResult + 2, ResourcePoolUtils.getAllResources().size());
+    assertEquals("", gson.fromJson(
+        intp1.interpret("get note1:paragraph1:key1", context).message(),
+        String.class));
+    assertEquals("", gson.fromJson(
+        intp1.interpret("get note1:paragraph2:key1", context).message(),
+        String.class));
+
+
+    // when remove all resources from note2:paragraph1
+    ResourcePoolUtils.removeResourcesBelongsToParagraph("note2", "paragraph1");
+
+    // then 1
+    assertEquals(numInterpreterResult + 1, ResourcePoolUtils.getAllResources().size());
+    assertEquals("value2", gson.fromJson(
+        intp1.interpret("get note2:paragraph2:key2", context).message(),
+        String.class));
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterFactory.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterFactory.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterFactory.java
index 3221054..f0fa385 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterFactory.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/interpreter/InterpreterFactory.java
@@ -434,6 +434,8 @@ public class InterpreterFactory {
       angularObjectRegistry = new AngularObjectRegistry(
           id,
           angularObjectRegistryListener);
+
+      // TODO(moon) : create distributed resource pool for local interpreters and set
     }
 
     interpreterGroup.setAngularObjectRegistry(angularObjectRegistry);

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Note.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Note.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Note.java
index b0470c8..6a09735 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Note.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Note.java
@@ -32,6 +32,7 @@ import org.apache.zeppelin.interpreter.*;
 import org.apache.zeppelin.interpreter.remote.RemoteAngularObjectRegistry;
 import org.apache.zeppelin.notebook.repo.NotebookRepo;
 import org.apache.zeppelin.notebook.utility.IdHashes;
+import org.apache.zeppelin.resource.ResourcePoolUtils;
 import org.apache.zeppelin.scheduler.Job;
 import org.apache.zeppelin.scheduler.Job.Status;
 import org.apache.zeppelin.scheduler.JobListener;
@@ -208,6 +209,8 @@ public class Note implements Serializable, JobListener {
    * @return a paragraph that was deleted, or <code>null</code> otherwise
    */
   public Paragraph removeParagraph(String paragraphId) {
+    removeAllAngularObjectInParagraph(paragraphId);
+    ResourcePoolUtils.removeResourcesBelongsToParagraph(id(), paragraphId);
     synchronized (paragraphs) {
       Iterator<Paragraph> i = paragraphs.iterator();
       while (i.hasNext()) {
@@ -220,7 +223,7 @@ public class Note implements Serializable, JobListener {
       }
     }
 
-    removeAllAngularObjectInParagraph(paragraphId);
+
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Notebook.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Notebook.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Notebook.java
index 8a14b87..4827bff 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Notebook.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/Notebook.java
@@ -40,6 +40,7 @@ import org.apache.zeppelin.interpreter.InterpreterSetting;
 import org.apache.zeppelin.interpreter.remote.RemoteAngularObjectRegistry;
 import org.apache.zeppelin.notebook.repo.NotebookRepo;
 import org.apache.zeppelin.notebook.repo.NotebookRepoSync;
+import org.apache.zeppelin.resource.ResourcePoolUtils;
 import org.apache.zeppelin.scheduler.SchedulerFactory;
 import org.apache.zeppelin.search.SearchService;
 import org.quartz.CronScheduleBuilder;
@@ -307,6 +308,8 @@ public class Notebook {
       }
     }
 
+    ResourcePoolUtils.removeResourcesBelongsToNote(id);
+
     try {
       note.unpersist();
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/mock/MockInterpreter1.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/mock/MockInterpreter1.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/mock/MockInterpreter1.java
index 079846c..cf0a613 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/mock/MockInterpreter1.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/mock/MockInterpreter1.java
@@ -45,13 +45,20 @@ public class MockInterpreter1 extends Interpreter{
 
 	@Override
 	public InterpreterResult interpret(String st, InterpreterContext context) {
+		InterpreterResult result;
 
 		if ("getId".equals(st)) {
 			// get unique id of this interpreter instance
-			return new InterpreterResult(InterpreterResult.Code.SUCCESS, "" + this.hashCode());
+			result = new InterpreterResult(InterpreterResult.Code.SUCCESS, "" + this.hashCode());
 		} else {
-			return new InterpreterResult(InterpreterResult.Code.SUCCESS, "repl1: " + st);
+			result = new InterpreterResult(InterpreterResult.Code.SUCCESS, "repl1: " + st);
 		}
+
+		if (context.getResourcePool() != null) {
+			context.getResourcePool().put(context.getNoteId(), context.getParagraphId(), "result", result);
+		}
+
+		return result;
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/mock/MockInterpreter2.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/mock/MockInterpreter2.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/mock/MockInterpreter2.java
index dd465a5..bae4b8d 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/mock/MockInterpreter2.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/interpreter/mock/MockInterpreter2.java
@@ -45,7 +45,19 @@ public class MockInterpreter2 extends Interpreter{
 
 	@Override
 	public InterpreterResult interpret(String st, InterpreterContext context) {
-		return new InterpreterResult(InterpreterResult.Code.SUCCESS, "repl2: "+st);
+		InterpreterResult result;
+
+		if ("getId".equals(st)) {
+			// get unique id of this interpreter instance
+			result = new InterpreterResult(InterpreterResult.Code.SUCCESS, "" + this.hashCode());
+		} else {
+			result = new InterpreterResult(InterpreterResult.Code.SUCCESS, "repl2: " + st);
+		}
+
+		if (context.getResourcePool() != null) {
+			context.getResourcePool().put(context.getNoteId(), context.getParagraphId(), "result", result);
+		}
+		return result;
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NotebookTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NotebookTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NotebookTest.java
index 06889d6..3c89c35 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NotebookTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NotebookTest.java
@@ -38,6 +38,8 @@ import org.apache.zeppelin.interpreter.mock.MockInterpreter1;
 import org.apache.zeppelin.interpreter.mock.MockInterpreter2;
 import org.apache.zeppelin.notebook.repo.NotebookRepo;
 import org.apache.zeppelin.notebook.repo.VFSNotebookRepo;
+import org.apache.zeppelin.resource.LocalResourcePool;
+import org.apache.zeppelin.resource.ResourcePoolUtils;
 import org.apache.zeppelin.scheduler.Job;
 import org.apache.zeppelin.scheduler.Job.Status;
 import org.apache.zeppelin.scheduler.JobListener;
@@ -326,6 +328,33 @@ public class NotebookTest implements JobListenerFactory{
   }
 
   @Test
+  public void testResourceRemovealOnParagraphNoteRemove() throws IOException {
+    Note note = notebook.createNote();
+    note.getNoteReplLoader().setInterpreters(factory.getDefaultInterpreterSettingList());
+    for (InterpreterGroup intpGroup : InterpreterGroup.getAll()) {
+      intpGroup.setResourcePool(new LocalResourcePool(intpGroup.getId()));
+    }
+    Paragraph p1 = note.addParagraph();
+    p1.setText("hello");
+    Paragraph p2 = note.addParagraph();
+    p2.setText("%mock2 world");
+
+    note.runAll();
+    while(p1.isTerminated()==false || p1.getResult()==null) Thread.yield();
+    while(p2.isTerminated()==false || p2.getResult()==null) Thread.yield();
+
+    assertEquals(2, ResourcePoolUtils.getAllResources().size());
+
+    // remove a paragraph
+    note.removeParagraph(p1.getId());
+    assertEquals(1, ResourcePoolUtils.getAllResources().size());
+
+    // remove note
+    notebook.removeNote(note.id());
+    assertEquals(0, ResourcePoolUtils.getAllResources().size());
+  }
+
+  @Test
   public void testAngularObjectRemovalOnNotebookRemove() throws InterruptedException,
       IOException {
     // create a note and a paragraph


[2/2] incubator-zeppelin git commit: [ZEPPELIN-713] Expand ResourceId and put InterpreterResult into ResourcePool

Posted by mo...@apache.org.
[ZEPPELIN-713] Expand ResourceId and put InterpreterResult into ResourcePool

### What is this PR for?
ResourcePool is introduced by ZEPPELIN-619.
However, it didn't provide any automatic resource release on paragraph / note removal unlike paragraph scope angular object which is quite helpful in practical use.

This PR adds noteId and paragraphId to ResourceId, so resource can be identified by noteId and paragraphId in addition to resourcePoolId and name.
By using informations in ResourceId, automatically release resource when note / paragraph remove.

So now, it's possible to put InterpreterResult into the resource pool safely, which might be useful for ZEPPELIN-374 and ZEPPELIN-533.

### What type of PR is it?
Improvement

### Todos
* [x] - Add noteId and paragraphId to ResourceId
* [x] - Release resource when note / paragraph remove
* [x] - Put InterpreterResult into resource pool
* [x] - Unit test

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

### How should this be tested?
You can get resourcePool from InterpreterContext and print items inside of spark interpreter. (note. resource pool is not spark interpreter specific feature. But so far spark interpreter provides user api to access it)

You can see InterpreterResult is automatically shared to resourcePool and they're removed on note/paragraph remove.

```
%md
hello world

%spark
// print items in resource pool.
import collection.JavaConversions._
val pool = z.getInterpreterContext.getResourcePool
pool.getAll.foreach(i=>println(i + " " + i.get))
```

### Screenshots (if appropriate)

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

Author: Lee moon soo <mo...@apache.org>

Closes #763 from Leemoonsoo/ZEPPELIN-713 and squashes the following commits:

473dd6f [Lee moon soo] Update test
b2e4fcc [Lee moon soo] put interpreter result into resource pool
11d138c [Lee moon soo] Remove resource on note / paragraph removal


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

Branch: refs/heads/master
Commit: 85a2ad376640aa62db683d9ffbe2d74f51e88b59
Parents: e2122bc
Author: Lee moon soo <mo...@apache.org>
Authored: Sat Mar 5 12:43:33 2016 -0800
Committer: Lee moon soo <mo...@apache.org>
Committed: Tue Mar 15 11:38:03 2016 -0700

----------------------------------------------------------------------
 .../remote/RemoteInterpreterEventPoller.java    |    5 +-
 .../remote/RemoteInterpreterServer.java         |   26 +-
 .../thrift/RemoteInterpreterContext.java        |    4 +-
 .../thrift/RemoteInterpreterEvent.java          |    4 +-
 .../thrift/RemoteInterpreterEventType.java      |    2 +-
 .../thrift/RemoteInterpreterResult.java         |    4 +-
 .../thrift/RemoteInterpreterService.java        | 1338 +++++++++++++++++-
 .../resource/DistributedResourcePool.java       |   34 +
 .../zeppelin/resource/LocalResourcePool.java    |   19 +
 .../apache/zeppelin/resource/ResourceId.java    |   34 +-
 .../apache/zeppelin/resource/ResourcePool.java  |   30 +
 .../zeppelin/resource/ResourcePoolUtils.java    |  136 ++
 .../apache/zeppelin/resource/ResourceSet.java   |   30 +
 .../resource/WellKnownResourceName.java         |   33 +
 .../main/thrift/RemoteInterpreterService.thrift |    4 +-
 .../mock/MockInterpreterResourcePool.java       |   23 +-
 .../resource/DistributedResourcePoolTest.java   |   45 +-
 .../interpreter/InterpreterFactory.java         |    2 +
 .../java/org/apache/zeppelin/notebook/Note.java |    5 +-
 .../org/apache/zeppelin/notebook/Notebook.java  |    3 +
 .../interpreter/mock/MockInterpreter1.java      |   11 +-
 .../interpreter/mock/MockInterpreter2.java      |   14 +-
 .../apache/zeppelin/notebook/NotebookTest.java  |   29 +
 23 files changed, 1787 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterEventPoller.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterEventPoller.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterEventPoller.java
index 8600c78..8c891c2 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterEventPoller.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterEventPoller.java
@@ -260,7 +260,10 @@ public class RemoteInterpreterEventPoller extends Thread {
       boolean broken = false;
       try {
         client = remoteInterpreterProcess.getClient();
-        ByteBuffer res = client.resourceGet(resourceId.getName());
+        ByteBuffer res = client.resourceGet(
+            resourceId.getNoteId(),
+            resourceId.getParagraphId(),
+            resourceId.getName());
         Object o = Resource.deserializeObject(res);
         return o;
       } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterServer.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterServer.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterServer.java
index 09ef391..243095f 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterServer.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterServer.java
@@ -343,12 +343,22 @@ public class RemoteInterpreterServer
         }
 
         String interpreterResultMessage = result.message();
+
+        InterpreterResult combinedResult;
         if (interpreterResultMessage != null && !interpreterResultMessage.isEmpty()) {
           message += interpreterResultMessage;
-          return new InterpreterResult(result.code(), result.type(), message);
+          combinedResult = new InterpreterResult(result.code(), result.type(), message);
         } else {
-          return new InterpreterResult(result.code(), outputType, message);
+          combinedResult = new InterpreterResult(result.code(), outputType, message);
         }
+
+        // put result into resource pool
+        context.getResourcePool().put(
+            context.getNoteId(),
+            context.getParagraphId(),
+            WellKnownResourceName.ParagraphResult.toString(),
+            combinedResult);
+        return combinedResult;
       } finally {
         InterpreterContext.remove();
       }
@@ -651,9 +661,17 @@ public class RemoteInterpreterServer
   }
 
   @Override
-  public ByteBuffer resourceGet(String resourceName) throws TException {
+  public boolean resourceRemove(String noteId, String paragraphId, String resourceName)
+      throws TException {
+    Resource resource = resourcePool.remove(noteId, paragraphId, resourceName);
+    return resource != null;
+  }
+
+  @Override
+  public ByteBuffer resourceGet(String noteId, String paragraphId, String resourceName)
+      throws TException {
     logger.debug("Request resourceGet {} from ZeppelinServer", resourceName);
-    Resource resource = resourcePool.get(resourceName, false);
+    Resource resource = resourcePool.get(noteId, paragraphId, resourceName, false);
 
     if (resource == null || resource.get() == null || !resource.isSerializable()) {
       return ByteBuffer.allocate(0);

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterContext.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterContext.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterContext.java
index adcde0f..d3087c7 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterContext.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterContext.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.3)
+ * Autogenerated by Thrift Compiler (0.9.2)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-02-16")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2016-3-4")
 public class RemoteInterpreterContext implements org.apache.thrift.TBase<RemoteInterpreterContext, RemoteInterpreterContext._Fields>, java.io.Serializable, Cloneable, Comparable<RemoteInterpreterContext> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RemoteInterpreterContext");
 

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEvent.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEvent.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEvent.java
index 3df9946..c7d3ffc 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEvent.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEvent.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.3)
+ * Autogenerated by Thrift Compiler (0.9.2)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-02-16")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2016-3-4")
 public class RemoteInterpreterEvent implements org.apache.thrift.TBase<RemoteInterpreterEvent, RemoteInterpreterEvent._Fields>, java.io.Serializable, Cloneable, Comparable<RemoteInterpreterEvent> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RemoteInterpreterEvent");
 

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEventType.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEventType.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEventType.java
index 664e5ad..7cb7963 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEventType.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterEventType.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.3)
+ * Autogenerated by Thrift Compiler (0.9.2)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterResult.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterResult.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterResult.java
index f6ac9b5..a51d6d7 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterResult.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterResult.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.3)
+ * Autogenerated by Thrift Compiler (0.9.2)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-02-16")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2016-3-4")
 public class RemoteInterpreterResult implements org.apache.thrift.TBase<RemoteInterpreterResult, RemoteInterpreterResult._Fields>, java.io.Serializable, Cloneable, Comparable<RemoteInterpreterResult> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RemoteInterpreterResult");
 

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterService.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterService.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterService.java
index 6b74b40..e8b6b12 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterService.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/thrift/RemoteInterpreterService.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.3)
+ * Autogenerated by Thrift Compiler (0.9.2)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-02-16")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2016-3-4")
 public class RemoteInterpreterService {
 
   public interface Iface {
@@ -84,7 +84,9 @@ public class RemoteInterpreterService {
 
     public List<String> resoucePoolGetAll() throws org.apache.thrift.TException;
 
-    public ByteBuffer resourceGet(String resourceName) throws org.apache.thrift.TException;
+    public ByteBuffer resourceGet(String noteId, String paragraphId, String resourceName) throws org.apache.thrift.TException;
+
+    public boolean resourceRemove(String noteId, String paragraphId, String resourceName) throws org.apache.thrift.TException;
 
     public void angularObjectUpdate(String name, String noteId, String paragraphId, String object) throws org.apache.thrift.TException;
 
@@ -124,7 +126,9 @@ public class RemoteInterpreterService {
 
     public void resoucePoolGetAll(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
-    public void resourceGet(String resourceName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+    public void resourceGet(String noteId, String paragraphId, String resourceName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void resourceRemove(String noteId, String paragraphId, String resourceName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void angularObjectUpdate(String name, String noteId, String paragraphId, String object, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
@@ -470,15 +474,17 @@ public class RemoteInterpreterService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "resoucePoolGetAll failed: unknown result");
     }
 
-    public ByteBuffer resourceGet(String resourceName) throws org.apache.thrift.TException
+    public ByteBuffer resourceGet(String noteId, String paragraphId, String resourceName) throws org.apache.thrift.TException
     {
-      send_resourceGet(resourceName);
+      send_resourceGet(noteId, paragraphId, resourceName);
       return recv_resourceGet();
     }
 
-    public void send_resourceGet(String resourceName) throws org.apache.thrift.TException
+    public void send_resourceGet(String noteId, String paragraphId, String resourceName) throws org.apache.thrift.TException
     {
       resourceGet_args args = new resourceGet_args();
+      args.setNoteId(noteId);
+      args.setParagraphId(paragraphId);
       args.setResourceName(resourceName);
       sendBase("resourceGet", args);
     }
@@ -493,6 +499,31 @@ public class RemoteInterpreterService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "resourceGet failed: unknown result");
     }
 
+    public boolean resourceRemove(String noteId, String paragraphId, String resourceName) throws org.apache.thrift.TException
+    {
+      send_resourceRemove(noteId, paragraphId, resourceName);
+      return recv_resourceRemove();
+    }
+
+    public void send_resourceRemove(String noteId, String paragraphId, String resourceName) throws org.apache.thrift.TException
+    {
+      resourceRemove_args args = new resourceRemove_args();
+      args.setNoteId(noteId);
+      args.setParagraphId(paragraphId);
+      args.setResourceName(resourceName);
+      sendBase("resourceRemove", args);
+    }
+
+    public boolean recv_resourceRemove() throws org.apache.thrift.TException
+    {
+      resourceRemove_result result = new resourceRemove_result();
+      receiveBase(result, "resourceRemove");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "resourceRemove failed: unknown result");
+    }
+
     public void angularObjectUpdate(String name, String noteId, String paragraphId, String object) throws org.apache.thrift.TException
     {
       send_angularObjectUpdate(name, noteId, paragraphId, object);
@@ -1072,23 +1103,29 @@ public class RemoteInterpreterService {
       }
     }
 
-    public void resourceGet(String resourceName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void resourceGet(String noteId, String paragraphId, String resourceName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      resourceGet_call method_call = new resourceGet_call(resourceName, resultHandler, this, ___protocolFactory, ___transport);
+      resourceGet_call method_call = new resourceGet_call(noteId, paragraphId, resourceName, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
     public static class resourceGet_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String noteId;
+      private String paragraphId;
       private String resourceName;
-      public resourceGet_call(String resourceName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public resourceGet_call(String noteId, String paragraphId, String resourceName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
+        this.noteId = noteId;
+        this.paragraphId = paragraphId;
         this.resourceName = resourceName;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
         prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("resourceGet", org.apache.thrift.protocol.TMessageType.CALL, 0));
         resourceGet_args args = new resourceGet_args();
+        args.setNoteId(noteId);
+        args.setParagraphId(paragraphId);
         args.setResourceName(resourceName);
         args.write(prot);
         prot.writeMessageEnd();
@@ -1104,6 +1141,44 @@ public class RemoteInterpreterService {
       }
     }
 
+    public void resourceRemove(String noteId, String paragraphId, String resourceName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      resourceRemove_call method_call = new resourceRemove_call(noteId, paragraphId, resourceName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class resourceRemove_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String noteId;
+      private String paragraphId;
+      private String resourceName;
+      public resourceRemove_call(String noteId, String paragraphId, String resourceName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.noteId = noteId;
+        this.paragraphId = paragraphId;
+        this.resourceName = resourceName;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("resourceRemove", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        resourceRemove_args args = new resourceRemove_args();
+        args.setNoteId(noteId);
+        args.setParagraphId(paragraphId);
+        args.setResourceName(resourceName);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public boolean getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_resourceRemove();
+      }
+    }
+
     public void angularObjectUpdate(String name, String noteId, String paragraphId, String object, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       angularObjectUpdate_call method_call = new angularObjectUpdate_call(name, noteId, paragraphId, object, resultHandler, this, ___protocolFactory, ___transport);
@@ -1252,6 +1327,7 @@ public class RemoteInterpreterService {
       processMap.put("resourceResponseGet", new resourceResponseGet());
       processMap.put("resoucePoolGetAll", new resoucePoolGetAll());
       processMap.put("resourceGet", new resourceGet());
+      processMap.put("resourceRemove", new resourceRemove());
       processMap.put("angularObjectUpdate", new angularObjectUpdate());
       processMap.put("angularObjectAdd", new angularObjectAdd());
       processMap.put("angularObjectRemove", new angularObjectRemove());
@@ -1554,7 +1630,28 @@ public class RemoteInterpreterService {
 
       public resourceGet_result getResult(I iface, resourceGet_args args) throws org.apache.thrift.TException {
         resourceGet_result result = new resourceGet_result();
-        result.success = iface.resourceGet(args.resourceName);
+        result.success = iface.resourceGet(args.noteId, args.paragraphId, args.resourceName);
+        return result;
+      }
+    }
+
+    public static class resourceRemove<I extends Iface> extends org.apache.thrift.ProcessFunction<I, resourceRemove_args> {
+      public resourceRemove() {
+        super("resourceRemove");
+      }
+
+      public resourceRemove_args getEmptyArgsInstance() {
+        return new resourceRemove_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public resourceRemove_result getResult(I iface, resourceRemove_args args) throws org.apache.thrift.TException {
+        resourceRemove_result result = new resourceRemove_result();
+        result.success = iface.resourceRemove(args.noteId, args.paragraphId, args.resourceName);
+        result.setSuccessIsSet(true);
         return result;
       }
     }
@@ -1647,6 +1744,7 @@ public class RemoteInterpreterService {
       processMap.put("resourceResponseGet", new resourceResponseGet());
       processMap.put("resoucePoolGetAll", new resoucePoolGetAll());
       processMap.put("resourceGet", new resourceGet());
+      processMap.put("resourceRemove", new resourceRemove());
       processMap.put("angularObjectUpdate", new angularObjectUpdate());
       processMap.put("angularObjectAdd", new angularObjectAdd());
       processMap.put("angularObjectRemove", new angularObjectRemove());
@@ -2408,7 +2506,59 @@ public class RemoteInterpreterService {
       }
 
       public void start(I iface, resourceGet_args args, org.apache.thrift.async.AsyncMethodCallback<ByteBuffer> resultHandler) throws TException {
-        iface.resourceGet(args.resourceName,resultHandler);
+        iface.resourceGet(args.noteId, args.paragraphId, args.resourceName,resultHandler);
+      }
+    }
+
+    public static class resourceRemove<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, resourceRemove_args, Boolean> {
+      public resourceRemove() {
+        super("resourceRemove");
+      }
+
+      public resourceRemove_args getEmptyArgsInstance() {
+        return new resourceRemove_args();
+      }
+
+      public AsyncMethodCallback<Boolean> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Boolean>() { 
+          public void onComplete(Boolean o) {
+            resourceRemove_result result = new resourceRemove_result();
+            result.success = o;
+            result.setSuccessIsSet(true);
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            resourceRemove_result result = new resourceRemove_result();
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, resourceRemove_args args, org.apache.thrift.async.AsyncMethodCallback<Boolean> resultHandler) throws TException {
+        iface.resourceRemove(args.noteId, args.paragraphId, args.resourceName,resultHandler);
       }
     }
 
@@ -7566,7 +7716,7 @@ public class RemoteInterpreterService {
     public Object getFieldValue(_Fields field) {
       switch (field) {
       case SUCCESS:
-        return getSuccess();
+        return Integer.valueOf(getSuccess());
 
       }
       throw new IllegalStateException();
@@ -8898,7 +9048,7 @@ public class RemoteInterpreterService {
         return getBuf();
 
       case CURSOR:
-        return getCursor();
+        return Integer.valueOf(getCursor());
 
       }
       throw new IllegalStateException();
@@ -13667,7 +13817,9 @@ public class RemoteInterpreterService {
   public static class resourceGet_args implements org.apache.thrift.TBase<resourceGet_args, resourceGet_args._Fields>, java.io.Serializable, Cloneable, Comparable<resourceGet_args>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("resourceGet_args");
 
-    private static final org.apache.thrift.protocol.TField RESOURCE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("resourceName", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField NOTE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("noteId", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField PARAGRAPH_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("paragraphId", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField RESOURCE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("resourceName", org.apache.thrift.protocol.TType.STRING, (short)3);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
@@ -13675,11 +13827,15 @@ public class RemoteInterpreterService {
       schemes.put(TupleScheme.class, new resourceGet_argsTupleSchemeFactory());
     }
 
+    public String noteId; // required
+    public String paragraphId; // required
     public String resourceName; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      RESOURCE_NAME((short)1, "resourceName");
+      NOTE_ID((short)1, "noteId"),
+      PARAGRAPH_ID((short)2, "paragraphId"),
+      RESOURCE_NAME((short)3, "resourceName");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -13694,7 +13850,11 @@ public class RemoteInterpreterService {
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // RESOURCE_NAME
+          case 1: // NOTE_ID
+            return NOTE_ID;
+          case 2: // PARAGRAPH_ID
+            return PARAGRAPH_ID;
+          case 3: // RESOURCE_NAME
             return RESOURCE_NAME;
           default:
             return null;
@@ -13739,6 +13899,10 @@ public class RemoteInterpreterService {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.NOTE_ID, new org.apache.thrift.meta_data.FieldMetaData("noteId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.PARAGRAPH_ID, new org.apache.thrift.meta_data.FieldMetaData("paragraphId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       tmpMap.put(_Fields.RESOURCE_NAME, new org.apache.thrift.meta_data.FieldMetaData("resourceName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
@@ -13749,9 +13913,13 @@ public class RemoteInterpreterService {
     }
 
     public resourceGet_args(
+      String noteId,
+      String paragraphId,
       String resourceName)
     {
       this();
+      this.noteId = noteId;
+      this.paragraphId = paragraphId;
       this.resourceName = resourceName;
     }
 
@@ -13759,6 +13927,12 @@ public class RemoteInterpreterService {
      * Performs a deep copy on <i>other</i>.
      */
     public resourceGet_args(resourceGet_args other) {
+      if (other.isSetNoteId()) {
+        this.noteId = other.noteId;
+      }
+      if (other.isSetParagraphId()) {
+        this.paragraphId = other.paragraphId;
+      }
       if (other.isSetResourceName()) {
         this.resourceName = other.resourceName;
       }
@@ -13770,9 +13944,59 @@ public class RemoteInterpreterService {
 
     @Override
     public void clear() {
+      this.noteId = null;
+      this.paragraphId = null;
       this.resourceName = null;
     }
 
+    public String getNoteId() {
+      return this.noteId;
+    }
+
+    public resourceGet_args setNoteId(String noteId) {
+      this.noteId = noteId;
+      return this;
+    }
+
+    public void unsetNoteId() {
+      this.noteId = null;
+    }
+
+    /** Returns true if field noteId is set (has been assigned a value) and false otherwise */
+    public boolean isSetNoteId() {
+      return this.noteId != null;
+    }
+
+    public void setNoteIdIsSet(boolean value) {
+      if (!value) {
+        this.noteId = null;
+      }
+    }
+
+    public String getParagraphId() {
+      return this.paragraphId;
+    }
+
+    public resourceGet_args setParagraphId(String paragraphId) {
+      this.paragraphId = paragraphId;
+      return this;
+    }
+
+    public void unsetParagraphId() {
+      this.paragraphId = null;
+    }
+
+    /** Returns true if field paragraphId is set (has been assigned a value) and false otherwise */
+    public boolean isSetParagraphId() {
+      return this.paragraphId != null;
+    }
+
+    public void setParagraphIdIsSet(boolean value) {
+      if (!value) {
+        this.paragraphId = null;
+      }
+    }
+
     public String getResourceName() {
       return this.resourceName;
     }
@@ -13799,6 +14023,22 @@ public class RemoteInterpreterService {
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
+      case NOTE_ID:
+        if (value == null) {
+          unsetNoteId();
+        } else {
+          setNoteId((String)value);
+        }
+        break;
+
+      case PARAGRAPH_ID:
+        if (value == null) {
+          unsetParagraphId();
+        } else {
+          setParagraphId((String)value);
+        }
+        break;
+
       case RESOURCE_NAME:
         if (value == null) {
           unsetResourceName();
@@ -13812,6 +14052,12 @@ public class RemoteInterpreterService {
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
+      case NOTE_ID:
+        return getNoteId();
+
+      case PARAGRAPH_ID:
+        return getParagraphId();
+
       case RESOURCE_NAME:
         return getResourceName();
 
@@ -13826,6 +14072,10 @@ public class RemoteInterpreterService {
       }
 
       switch (field) {
+      case NOTE_ID:
+        return isSetNoteId();
+      case PARAGRAPH_ID:
+        return isSetParagraphId();
       case RESOURCE_NAME:
         return isSetResourceName();
       }
@@ -13845,6 +14095,24 @@ public class RemoteInterpreterService {
       if (that == null)
         return false;
 
+      boolean this_present_noteId = true && this.isSetNoteId();
+      boolean that_present_noteId = true && that.isSetNoteId();
+      if (this_present_noteId || that_present_noteId) {
+        if (!(this_present_noteId && that_present_noteId))
+          return false;
+        if (!this.noteId.equals(that.noteId))
+          return false;
+      }
+
+      boolean this_present_paragraphId = true && this.isSetParagraphId();
+      boolean that_present_paragraphId = true && that.isSetParagraphId();
+      if (this_present_paragraphId || that_present_paragraphId) {
+        if (!(this_present_paragraphId && that_present_paragraphId))
+          return false;
+        if (!this.paragraphId.equals(that.paragraphId))
+          return false;
+      }
+
       boolean this_present_resourceName = true && this.isSetResourceName();
       boolean that_present_resourceName = true && that.isSetResourceName();
       if (this_present_resourceName || that_present_resourceName) {
@@ -13861,6 +14129,16 @@ public class RemoteInterpreterService {
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
+      boolean present_noteId = true && (isSetNoteId());
+      list.add(present_noteId);
+      if (present_noteId)
+        list.add(noteId);
+
+      boolean present_paragraphId = true && (isSetParagraphId());
+      list.add(present_paragraphId);
+      if (present_paragraphId)
+        list.add(paragraphId);
+
       boolean present_resourceName = true && (isSetResourceName());
       list.add(present_resourceName);
       if (present_resourceName)
@@ -13877,6 +14155,26 @@ public class RemoteInterpreterService {
 
       int lastComparison = 0;
 
+      lastComparison = Boolean.valueOf(isSetNoteId()).compareTo(other.isSetNoteId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetNoteId()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.noteId, other.noteId);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetParagraphId()).compareTo(other.isSetParagraphId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetParagraphId()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.paragraphId, other.paragraphId);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       lastComparison = Boolean.valueOf(isSetResourceName()).compareTo(other.isSetResourceName());
       if (lastComparison != 0) {
         return lastComparison;
@@ -13907,8 +14205,24 @@ public class RemoteInterpreterService {
       StringBuilder sb = new StringBuilder("resourceGet_args(");
       boolean first = true;
 
-      sb.append("resourceName:");
-      if (this.resourceName == null) {
+      sb.append("noteId:");
+      if (this.noteId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.noteId);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("paragraphId:");
+      if (this.paragraphId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.paragraphId);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("resourceName:");
+      if (this.resourceName == null) {
         sb.append("null");
       } else {
         sb.append(this.resourceName);
@@ -13957,7 +14271,23 @@ public class RemoteInterpreterService {
             break;
           }
           switch (schemeField.id) {
-            case 1: // RESOURCE_NAME
+            case 1: // NOTE_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.noteId = iprot.readString();
+                struct.setNoteIdIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // PARAGRAPH_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.paragraphId = iprot.readString();
+                struct.setParagraphIdIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // RESOURCE_NAME
               if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
                 struct.resourceName = iprot.readString();
                 struct.setResourceNameIsSet(true);
@@ -13980,6 +14310,16 @@ public class RemoteInterpreterService {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.noteId != null) {
+          oprot.writeFieldBegin(NOTE_ID_FIELD_DESC);
+          oprot.writeString(struct.noteId);
+          oprot.writeFieldEnd();
+        }
+        if (struct.paragraphId != null) {
+          oprot.writeFieldBegin(PARAGRAPH_ID_FIELD_DESC);
+          oprot.writeString(struct.paragraphId);
+          oprot.writeFieldEnd();
+        }
         if (struct.resourceName != null) {
           oprot.writeFieldBegin(RESOURCE_NAME_FIELD_DESC);
           oprot.writeString(struct.resourceName);
@@ -14003,10 +14343,22 @@ public class RemoteInterpreterService {
       public void write(org.apache.thrift.protocol.TProtocol prot, resourceGet_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
-        if (struct.isSetResourceName()) {
+        if (struct.isSetNoteId()) {
           optionals.set(0);
         }
-        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetParagraphId()) {
+          optionals.set(1);
+        }
+        if (struct.isSetResourceName()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetNoteId()) {
+          oprot.writeString(struct.noteId);
+        }
+        if (struct.isSetParagraphId()) {
+          oprot.writeString(struct.paragraphId);
+        }
         if (struct.isSetResourceName()) {
           oprot.writeString(struct.resourceName);
         }
@@ -14015,8 +14367,16 @@ public class RemoteInterpreterService {
       @Override
       public void read(org.apache.thrift.protocol.TProtocol prot, resourceGet_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(1);
+        BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
+          struct.noteId = iprot.readString();
+          struct.setNoteIdIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.paragraphId = iprot.readString();
+          struct.setParagraphIdIsSet(true);
+        }
+        if (incoming.get(2)) {
           struct.resourceName = iprot.readString();
           struct.setResourceNameIsSet(true);
         }
@@ -14396,6 +14756,938 @@ public class RemoteInterpreterService {
 
   }
 
+  public static class resourceRemove_args implements org.apache.thrift.TBase<resourceRemove_args, resourceRemove_args._Fields>, java.io.Serializable, Cloneable, Comparable<resourceRemove_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("resourceRemove_args");
+
+    private static final org.apache.thrift.protocol.TField NOTE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("noteId", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField PARAGRAPH_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("paragraphId", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField RESOURCE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("resourceName", org.apache.thrift.protocol.TType.STRING, (short)3);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new resourceRemove_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new resourceRemove_argsTupleSchemeFactory());
+    }
+
+    public String noteId; // required
+    public String paragraphId; // required
+    public String resourceName; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      NOTE_ID((short)1, "noteId"),
+      PARAGRAPH_ID((short)2, "paragraphId"),
+      RESOURCE_NAME((short)3, "resourceName");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // NOTE_ID
+            return NOTE_ID;
+          case 2: // PARAGRAPH_ID
+            return PARAGRAPH_ID;
+          case 3: // RESOURCE_NAME
+            return RESOURCE_NAME;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.NOTE_ID, new org.apache.thrift.meta_data.FieldMetaData("noteId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.PARAGRAPH_ID, new org.apache.thrift.meta_data.FieldMetaData("paragraphId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.RESOURCE_NAME, new org.apache.thrift.meta_data.FieldMetaData("resourceName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(resourceRemove_args.class, metaDataMap);
+    }
+
+    public resourceRemove_args() {
+    }
+
+    public resourceRemove_args(
+      String noteId,
+      String paragraphId,
+      String resourceName)
+    {
+      this();
+      this.noteId = noteId;
+      this.paragraphId = paragraphId;
+      this.resourceName = resourceName;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public resourceRemove_args(resourceRemove_args other) {
+      if (other.isSetNoteId()) {
+        this.noteId = other.noteId;
+      }
+      if (other.isSetParagraphId()) {
+        this.paragraphId = other.paragraphId;
+      }
+      if (other.isSetResourceName()) {
+        this.resourceName = other.resourceName;
+      }
+    }
+
+    public resourceRemove_args deepCopy() {
+      return new resourceRemove_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.noteId = null;
+      this.paragraphId = null;
+      this.resourceName = null;
+    }
+
+    public String getNoteId() {
+      return this.noteId;
+    }
+
+    public resourceRemove_args setNoteId(String noteId) {
+      this.noteId = noteId;
+      return this;
+    }
+
+    public void unsetNoteId() {
+      this.noteId = null;
+    }
+
+    /** Returns true if field noteId is set (has been assigned a value) and false otherwise */
+    public boolean isSetNoteId() {
+      return this.noteId != null;
+    }
+
+    public void setNoteIdIsSet(boolean value) {
+      if (!value) {
+        this.noteId = null;
+      }
+    }
+
+    public String getParagraphId() {
+      return this.paragraphId;
+    }
+
+    public resourceRemove_args setParagraphId(String paragraphId) {
+      this.paragraphId = paragraphId;
+      return this;
+    }
+
+    public void unsetParagraphId() {
+      this.paragraphId = null;
+    }
+
+    /** Returns true if field paragraphId is set (has been assigned a value) and false otherwise */
+    public boolean isSetParagraphId() {
+      return this.paragraphId != null;
+    }
+
+    public void setParagraphIdIsSet(boolean value) {
+      if (!value) {
+        this.paragraphId = null;
+      }
+    }
+
+    public String getResourceName() {
+      return this.resourceName;
+    }
+
+    public resourceRemove_args setResourceName(String resourceName) {
+      this.resourceName = resourceName;
+      return this;
+    }
+
+    public void unsetResourceName() {
+      this.resourceName = null;
+    }
+
+    /** Returns true if field resourceName is set (has been assigned a value) and false otherwise */
+    public boolean isSetResourceName() {
+      return this.resourceName != null;
+    }
+
+    public void setResourceNameIsSet(boolean value) {
+      if (!value) {
+        this.resourceName = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case NOTE_ID:
+        if (value == null) {
+          unsetNoteId();
+        } else {
+          setNoteId((String)value);
+        }
+        break;
+
+      case PARAGRAPH_ID:
+        if (value == null) {
+          unsetParagraphId();
+        } else {
+          setParagraphId((String)value);
+        }
+        break;
+
+      case RESOURCE_NAME:
+        if (value == null) {
+          unsetResourceName();
+        } else {
+          setResourceName((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case NOTE_ID:
+        return getNoteId();
+
+      case PARAGRAPH_ID:
+        return getParagraphId();
+
+      case RESOURCE_NAME:
+        return getResourceName();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case NOTE_ID:
+        return isSetNoteId();
+      case PARAGRAPH_ID:
+        return isSetParagraphId();
+      case RESOURCE_NAME:
+        return isSetResourceName();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof resourceRemove_args)
+        return this.equals((resourceRemove_args)that);
+      return false;
+    }
+
+    public boolean equals(resourceRemove_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_noteId = true && this.isSetNoteId();
+      boolean that_present_noteId = true && that.isSetNoteId();
+      if (this_present_noteId || that_present_noteId) {
+        if (!(this_present_noteId && that_present_noteId))
+          return false;
+        if (!this.noteId.equals(that.noteId))
+          return false;
+      }
+
+      boolean this_present_paragraphId = true && this.isSetParagraphId();
+      boolean that_present_paragraphId = true && that.isSetParagraphId();
+      if (this_present_paragraphId || that_present_paragraphId) {
+        if (!(this_present_paragraphId && that_present_paragraphId))
+          return false;
+        if (!this.paragraphId.equals(that.paragraphId))
+          return false;
+      }
+
+      boolean this_present_resourceName = true && this.isSetResourceName();
+      boolean that_present_resourceName = true && that.isSetResourceName();
+      if (this_present_resourceName || that_present_resourceName) {
+        if (!(this_present_resourceName && that_present_resourceName))
+          return false;
+        if (!this.resourceName.equals(that.resourceName))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_noteId = true && (isSetNoteId());
+      list.add(present_noteId);
+      if (present_noteId)
+        list.add(noteId);
+
+      boolean present_paragraphId = true && (isSetParagraphId());
+      list.add(present_paragraphId);
+      if (present_paragraphId)
+        list.add(paragraphId);
+
+      boolean present_resourceName = true && (isSetResourceName());
+      list.add(present_resourceName);
+      if (present_resourceName)
+        list.add(resourceName);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(resourceRemove_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetNoteId()).compareTo(other.isSetNoteId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetNoteId()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.noteId, other.noteId);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetParagraphId()).compareTo(other.isSetParagraphId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetParagraphId()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.paragraphId, other.paragraphId);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetResourceName()).compareTo(other.isSetResourceName());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetResourceName()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.resourceName, other.resourceName);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("resourceRemove_args(");
+      boolean first = true;
+
+      sb.append("noteId:");
+      if (this.noteId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.noteId);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("paragraphId:");
+      if (this.paragraphId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.paragraphId);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("resourceName:");
+      if (this.resourceName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.resourceName);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class resourceRemove_argsStandardSchemeFactory implements SchemeFactory {
+      public resourceRemove_argsStandardScheme getScheme() {
+        return new resourceRemove_argsStandardScheme();
+      }
+    }
+
+    private static class resourceRemove_argsStandardScheme extends StandardScheme<resourceRemove_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, resourceRemove_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // NOTE_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.noteId = iprot.readString();
+                struct.setNoteIdIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // PARAGRAPH_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.paragraphId = iprot.readString();
+                struct.setParagraphIdIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // RESOURCE_NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.resourceName = iprot.readString();
+                struct.setResourceNameIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, resourceRemove_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.noteId != null) {
+          oprot.writeFieldBegin(NOTE_ID_FIELD_DESC);
+          oprot.writeString(struct.noteId);
+          oprot.writeFieldEnd();
+        }
+        if (struct.paragraphId != null) {
+          oprot.writeFieldBegin(PARAGRAPH_ID_FIELD_DESC);
+          oprot.writeString(struct.paragraphId);
+          oprot.writeFieldEnd();
+        }
+        if (struct.resourceName != null) {
+          oprot.writeFieldBegin(RESOURCE_NAME_FIELD_DESC);
+          oprot.writeString(struct.resourceName);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class resourceRemove_argsTupleSchemeFactory implements SchemeFactory {
+      public resourceRemove_argsTupleScheme getScheme() {
+        return new resourceRemove_argsTupleScheme();
+      }
+    }
+
+    private static class resourceRemove_argsTupleScheme extends TupleScheme<resourceRemove_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, resourceRemove_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetNoteId()) {
+          optionals.set(0);
+        }
+        if (struct.isSetParagraphId()) {
+          optionals.set(1);
+        }
+        if (struct.isSetResourceName()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetNoteId()) {
+          oprot.writeString(struct.noteId);
+        }
+        if (struct.isSetParagraphId()) {
+          oprot.writeString(struct.paragraphId);
+        }
+        if (struct.isSetResourceName()) {
+          oprot.writeString(struct.resourceName);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, resourceRemove_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(3);
+        if (incoming.get(0)) {
+          struct.noteId = iprot.readString();
+          struct.setNoteIdIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.paragraphId = iprot.readString();
+          struct.setParagraphIdIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.resourceName = iprot.readString();
+          struct.setResourceNameIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class resourceRemove_result implements org.apache.thrift.TBase<resourceRemove_result, resourceRemove_result._Fields>, java.io.Serializable, Cloneable, Comparable<resourceRemove_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("resourceRemove_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new resourceRemove_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new resourceRemove_resultTupleSchemeFactory());
+    }
+
+    public boolean success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __SUCCESS_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(resourceRemove_result.class, metaDataMap);
+    }
+
+    public resourceRemove_result() {
+    }
+
+    public resourceRemove_result(
+      boolean success)
+    {
+      this();
+      this.success = success;
+      setSuccessIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public resourceRemove_result(resourceRemove_result other) {
+      __isset_bitfield = other.__isset_bitfield;
+      this.success = other.success;
+    }
+
+    public resourceRemove_result deepCopy() {
+      return new resourceRemove_result(this);
+    }
+
+    @Override
+    public void clear() {
+      setSuccessIsSet(false);
+      this.success = false;
+    }
+
+    public boolean isSuccess() {
+      return this.success;
+    }
+
+    public resourceRemove_result setSuccess(boolean success) {
+      this.success = success;
+      setSuccessIsSet(true);
+      return this;
+    }
+
+    public void unsetSuccess() {
+      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((Boolean)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return Boolean.valueOf(isSuccess());
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof resourceRemove_result)
+        return this.equals((resourceRemove_result)that);
+      return false;
+    }
+
+    public boolean equals(resourceRemove_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true;
+      boolean that_present_success = true;
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (this.success != that.success)
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true;
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(resourceRemove_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("resourceRemove_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      sb.append(this.success);
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class resourceRemove_resultStandardSchemeFactory implements SchemeFactory {
+      public resourceRemove_resultStandardScheme getScheme() {
+        return new resourceRemove_resultStandardScheme();
+      }
+    }
+
+    private static class resourceRemove_resultStandardScheme extends StandardScheme<resourceRemove_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, resourceRemove_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+                struct.success = iprot.readBool();
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, resourceRemove_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.isSetSuccess()) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          oprot.writeBool(struct.success);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class resourceRemove_resultTupleSchemeFactory implements SchemeFactory {
+      public resourceRemove_resultTupleScheme getScheme() {
+        return new resourceRemove_resultTupleScheme();
+      }
+    }
+
+    private static class resourceRemove_resultTupleScheme extends TupleScheme<resourceRemove_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, resourceRemove_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          oprot.writeBool(struct.success);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, resourceRemove_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = iprot.readBool();
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+  }
+
   public static class angularObjectUpdate_args implements org.apache.thrift.TBase<angularObjectUpdate_args, angularObjectUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<angularObjectUpdate_args>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("angularObjectUpdate_args");
 

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/DistributedResourcePool.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/DistributedResourcePool.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/DistributedResourcePool.java
index 3f03b92..ba31f01 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/DistributedResourcePool.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/DistributedResourcePool.java
@@ -33,6 +33,11 @@ public class DistributedResourcePool extends LocalResourcePool {
     return get(name, true);
   }
 
+  @Override
+  public Resource get(String noteId, String paragraphId, String name) {
+    return get(noteId, paragraphId, name, true);
+  }
+
   /**
    * get resource by name.
    * @param name
@@ -58,6 +63,35 @@ public class DistributedResourcePool extends LocalResourcePool {
     }
   }
 
+  /**
+   * get resource by name.
+   * @param name
+   * @param remote false only return from local resource
+   * @return null if resource not found.
+   */
+  public Resource get(String noteId, String paragraphId, String name, boolean remote) {
+    // try local first
+    Resource resource = super.get(noteId, paragraphId, name);
+    if (resource != null) {
+      return resource;
+    }
+
+    if (remote) {
+      ResourceSet resources = connector.getAllResources()
+          .filterByNoteId(noteId)
+          .filterByParagraphId(paragraphId)
+          .filterByName(name);
+
+      if (resources.isEmpty()) {
+        return null;
+      } else {
+        return resources.get(0);
+      }
+    } else {
+      return null;
+    }
+  }
+
   @Override
   public ResourceSet getAll() {
     return getAll(true);

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/LocalResourcePool.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/LocalResourcePool.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/LocalResourcePool.java
index cc5f7e9..ef269e4 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/LocalResourcePool.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/LocalResourcePool.java
@@ -53,6 +53,12 @@ public class LocalResourcePool implements ResourcePool {
   }
 
   @Override
+  public Resource get(String noteId, String paragraphId, String name) {
+    ResourceId resourceId = new ResourceId(resourcePoolId, noteId, paragraphId, name);
+    return resources.get(resourceId);
+  }
+
+  @Override
   public ResourceSet getAll() {
     return new ResourceSet(resources.values());
   }
@@ -71,7 +77,20 @@ public class LocalResourcePool implements ResourcePool {
   }
 
   @Override
+  public void put(String noteId, String paragraphId, String name, Object object) {
+    ResourceId resourceId = new ResourceId(resourcePoolId, noteId, paragraphId, name);
+
+    Resource resource = new Resource(resourceId, object);
+    resources.put(resourceId, resource);
+  }
+
+  @Override
   public Resource remove(String name) {
     return resources.remove(new ResourceId(resourcePoolId, name));
   }
+
+  @Override
+  public Resource remove(String noteId, String paragraphId, String name) {
+    return resources.remove(new ResourceId(resourcePoolId, noteId, paragraphId, name));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourceId.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourceId.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourceId.java
index a0d55e3..3fb3102 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourceId.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourceId.java
@@ -22,9 +22,20 @@ package org.apache.zeppelin.resource;
 public class ResourceId {
   private final String resourcePoolId;
   private final String name;
+  private final String noteId;
+  private final String paragraphId;
 
   ResourceId(String resourcePoolId, String name) {
     this.resourcePoolId = resourcePoolId;
+    this.noteId = null;
+    this.paragraphId = null;
+    this.name = name;
+  }
+
+  ResourceId(String resourcePoolId, String noteId, String paragraphId, String name) {
+    this.resourcePoolId = resourcePoolId;
+    this.noteId = noteId;
+    this.paragraphId = paragraphId;
     this.name = name;
   }
 
@@ -36,16 +47,35 @@ public class ResourceId {
     return name;
   }
 
+  public String getNoteId() {
+    return noteId;
+  }
+
+  public String getParagraphId() {
+    return paragraphId;
+  }
+
   @Override
   public int hashCode() {
-    return (resourcePoolId + name).hashCode();
+    return (resourcePoolId + noteId + paragraphId + name).hashCode();
   }
 
   @Override
   public boolean equals(Object o) {
     if (o instanceof ResourceId) {
       ResourceId r = (ResourceId) o;
-      return (r.name.equals(name) && r.resourcePoolId.equals(resourcePoolId));
+      return equals(r.name, name) && equals(r.resourcePoolId, resourcePoolId) &&
+          equals(r.noteId, noteId) && equals(r.paragraphId, paragraphId);
+    } else {
+      return false;
+    }
+  }
+
+  private boolean equals(String a, String b) {
+    if (a == null && b == null) {
+      return true;
+    } else if (a != null && b != null) {
+      return a.equals(b);
     } else {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourcePool.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourcePool.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourcePool.java
index 6328b8d..a1c28de 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourcePool.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourcePool.java
@@ -34,6 +34,15 @@ public interface ResourcePool {
   public Resource get(String name);
 
   /**
+   * Get resource from name
+   * @param noteId
+   * @param paragraphId
+   * @param name Resource name
+   * @return null if resource not found
+   */
+  public Resource get(String noteId, String paragraphId, String name);
+
+  /**
    * Get all resources
    * @return
    */
@@ -47,9 +56,30 @@ public interface ResourcePool {
   public void put(String name, Object object);
 
   /**
+   * Put an object into resource pool
+   * Given noteId and paragraphId is identifying resource along with name.
+   * Object will be automatically removed on related note or paragraph removal.
+   *
+   * @param noteId
+   * @param paragraphId
+   * @param name
+   * @param object
+   */
+  public void put(String noteId, String paragraphId, String name, Object object);
+
+  /**
    * Remove object
    * @param name Resource name to remove
    * @return removed Resource. null if resource not found
    */
   public Resource remove(String name);
+
+  /**
+   * Remove object
+   * @param noteId
+   * @param paragraphId
+   * @param name Resource name to remove
+   * @return removed Resource. null if resource not found
+   */
+  public Resource remove(String noteId, String paragraphId, String name);
 }

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/85a2ad37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourcePoolUtils.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourcePoolUtils.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourcePoolUtils.java
new file mode 100644
index 0000000..ecd84d4
--- /dev/null
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/resource/ResourcePoolUtils.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zeppelin.resource;
+
+import com.google.gson.Gson;
+import org.apache.zeppelin.interpreter.InterpreterGroup;
+import org.apache.zeppelin.interpreter.remote.RemoteInterpreterProcess;
+import org.apache.zeppelin.interpreter.thrift.RemoteInterpreterService;
+import org.slf4j.Logger;
+
+import java.util.List;
+
+/**
+ * Utilities for ResourcePool
+ */
+public class ResourcePoolUtils {
+  static Logger logger = org.slf4j.LoggerFactory.getLogger(ResourcePoolUtils.class);
+
+  public static ResourceSet getAllResources() {
+    return getAllResourcesExcept(null);
+  }
+
+  public static ResourceSet getAllResourcesExcept(String interpreterGroupExcludsion) {
+    ResourceSet resourceSet = new ResourceSet();
+    for (InterpreterGroup intpGroup : InterpreterGroup.getAll()) {
+      if (interpreterGroupExcludsion != null &&
+          intpGroup.getId().equals(interpreterGroupExcludsion)) {
+        continue;
+      }
+
+      RemoteInterpreterProcess remoteInterpreterProcess = intpGroup.getRemoteInterpreterProcess();
+      if (remoteInterpreterProcess == null) {
+        ResourcePool localPool = intpGroup.getResourcePool();
+        if (localPool != null) {
+          resourceSet.addAll(localPool.getAll());
+        }
+      } else if (remoteInterpreterProcess.isRunning()) {
+        RemoteInterpreterService.Client client = null;
+        boolean broken = false;
+        try {
+          client = remoteInterpreterProcess.getClient();
+          List<String> resourceList = client.resoucePoolGetAll();
+          Gson gson = new Gson();
+          for (String res : resourceList) {
+            resourceSet.add(gson.fromJson(res, Resource.class));
+          }
+        } catch (Exception e) {
+          logger.error(e.getMessage(), e);
+          broken = true;
+        } finally {
+          if (client != null) {
+            intpGroup.getRemoteInterpreterProcess().releaseClient(client, broken);
+          }
+        }
+      }
+    }
+    return resourceSet;
+  }
+
+  public static void removeResourcesBelongsToNote(String noteId) {
+    removeResourcesBelongsToParagraph(noteId, null);
+  }
+
+  public static void removeResourcesBelongsToParagraph(String noteId, String paragraphId) {
+    for (InterpreterGroup intpGroup : InterpreterGroup.getAll()) {
+      ResourceSet resourceSet = new ResourceSet();
+      RemoteInterpreterProcess remoteInterpreterProcess = intpGroup.getRemoteInterpreterProcess();
+      if (remoteInterpreterProcess == null) {
+        ResourcePool localPool = intpGroup.getResourcePool();
+        if (localPool != null) {
+          resourceSet.addAll(localPool.getAll());
+        }
+        if (noteId != null) {
+          resourceSet = resourceSet.filterByNoteId(noteId);
+        }
+        if (paragraphId != null) {
+          resourceSet = resourceSet.filterByParagraphId(paragraphId);
+        }
+
+        for (Resource r : resourceSet) {
+          localPool.remove(
+              r.getResourceId().getNoteId(),
+              r.getResourceId().getParagraphId(),
+              r.getResourceId().getName());
+        }
+      } else if (remoteInterpreterProcess.isRunning()) {
+        RemoteInterpreterService.Client client = null;
+        boolean broken = false;
+        try {
+          client = remoteInterpreterProcess.getClient();
+          List<String> resourceList = client.resoucePoolGetAll();
+          Gson gson = new Gson();
+          for (String res : resourceList) {
+            resourceSet.add(gson.fromJson(res, Resource.class));
+          }
+
+          if (noteId != null) {
+            resourceSet = resourceSet.filterByNoteId(noteId);
+          }
+          if (paragraphId != null) {
+            resourceSet = resourceSet.filterByParagraphId(paragraphId);
+          }
+
+          for (Resource r : resourceSet) {
+            client.resourceRemove(
+                r.getResourceId().getNoteId(),
+                r.getResourceId().getParagraphId(),
+                r.getResourceId().getName());
+          }
+        } catch (Exception e) {
+          logger.error(e.getMessage(), e);
+          broken = true;
+        } finally {
+          if (client != null) {
+            intpGroup.getRemoteInterpreterProcess().releaseClient(client, broken);
+          }
+        }
+      }
+    }
+  }
+}