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 2015/10/14 11:16:41 UTC

incubator-zeppelin git commit: [Zeppelin 329] Notebook copied in file system level should be reflected in notebook list

Repository: incubator-zeppelin
Updated Branches:
  refs/heads/master 7fb04784b -> 4b8512bcf


[Zeppelin 329] Notebook copied in file system level should be reflected in notebook list

This PR enables Zeppelin to reflect addition/changes/removal of `{ZEPPELIN_NOTEBOOK_DIR}/{notebookId}/note.json` on the file system without restarting ZeppelinServer.

To activate this functionality, user needs to set `ZEPPELIN_NOTEBOOK_GET_FROM_REPO` to be true in conf/zeppelin-env.sh.

Author: Mina Lee <mi...@nflabs.com>

Closes #331 from minahlee/ZEPPELIN-329 and squashes the following commits:

41d8a10 [Mina Lee] Change env variable name
82bbb57 [Mina Lee] Fix indentation
34f412b [Mina Lee] ZEPPELIN-329 Notebook copied in file system level should be reflected in notebook list


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

Branch: refs/heads/master
Commit: 4b8512bcf9179831624ed45fc5b7dd1a661f64d4
Parents: 7fb0478
Author: Mina Lee <mi...@nflabs.com>
Authored: Mon Oct 5 19:34:25 2015 +0900
Committer: Lee moon soo <mo...@apache.org>
Committed: Wed Oct 14 11:17:17 2015 +0200

----------------------------------------------------------------------
 .../zeppelin/conf/ZeppelinConfiguration.java    |   4 +
 .../org/apache/zeppelin/notebook/Notebook.java  |  24 ++
 .../apache/zeppelin/notebook/NotebookTest.java  | 299 +++++++++-------
 .../src/test/resources/2A94M5J1Z/note.json      | 341 +++++++++++++++++++
 4 files changed, 540 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/4b8512bc/zeppelin-zengine/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
index 1027f3d..19ddceb 100755
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/conf/ZeppelinConfiguration.java
@@ -422,6 +422,10 @@ public class ZeppelinConfiguration extends XMLConfiguration {
     ZEPPELIN_NOTEBOOK_S3_BUCKET("zeppelin.notebook.s3.bucket", "zeppelin"),
     ZEPPELIN_NOTEBOOK_S3_USER("zeppelin.notebook.s3.user", "user"),
     ZEPPELIN_NOTEBOOK_STORAGE("zeppelin.notebook.storage", VFSNotebookRepo.class.getName()),
+    // Notebook list and contents will be always loaded from repository if set true.
+    // If set false, modified notebooks or new notebooks added on file system level
+    // won't be reflected on Zeppelin till user restarts Zeppelin.
+    ZEPPELIN_NOTEBOOK_RELOAD_FROM_STORAGE("zeppelin.notebook.reloadAllNotesFromStorage", false),
     ZEPPELIN_INTERPRETER_REMOTE_RUNNER("zeppelin.interpreter.remoterunner", "bin/interpreter.sh"),
     // Decide when new note is created, interpreter settings will be binded automatically or not.
     ZEPPELIN_NOTEBOOK_AUTO_INTERPRETER_BINDING("zeppelin.notebook.autoInterpreterBinding", true),

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/4b8512bc/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 03c9638..63ff250 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
@@ -263,6 +263,23 @@ public class Notebook {
     }
   }
 
+  /**
+   * Reload all notes from repository after clearing `notes`
+   * to reflect the changes of added/deleted/modified notebooks on file system level.
+   *
+   * @return
+   * @throws IOException
+   */
+  private void reloadAllNotes() throws IOException {
+    synchronized (notes) {
+      notes.clear();
+    }
+    List<NoteInfo> noteInfos = notebookRepo.list();
+    for (NoteInfo info : noteInfos) {
+      loadNoteFromRepo(info.getId());
+    }
+  }
+
   class SnapshotAngularObject {
     String intpGroupId;
     AngularObject angularObject;
@@ -288,6 +305,13 @@ public class Notebook {
   }
 
   public List<Note> getAllNotes() {
+    if (conf.getBoolean(ConfVars.ZEPPELIN_NOTEBOOK_RELOAD_FROM_STORAGE)) {
+      try {
+        reloadAllNotes();
+      } catch (IOException e) {
+        logger.error("Cannot reload notes from storage", e);
+      }
+    }
     synchronized (notes) {
       List<Note> noteList = new ArrayList<Note>(notes.values());
       Collections.sort(noteList, new Comparator() {

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/4b8512bc/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 08a0098..883dc64 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
@@ -24,8 +24,10 @@ import static org.junit.Assert.assertNull;
 import java.io.File;
 import java.io.IOException;
 import java.util.Date;
+import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.zeppelin.conf.ZeppelinConfiguration;
 import org.apache.zeppelin.conf.ZeppelinConfiguration.ConfVars;
 import org.apache.zeppelin.display.AngularObjectRegistry;
@@ -46,29 +48,29 @@ import org.quartz.SchedulerException;
 
 public class NotebookTest implements JobListenerFactory{
 
-	private File tmpDir;
-	private ZeppelinConfiguration conf;
-	private SchedulerFactory schedulerFactory;
-	private File notebookDir;
-	private Notebook notebook;
-	private NotebookRepo notebookRepo;
+  private File tmpDir;
+  private ZeppelinConfiguration conf;
+  private SchedulerFactory schedulerFactory;
+  private File notebookDir;
+  private Notebook notebook;
+  private NotebookRepo notebookRepo;
   private InterpreterFactory factory;
 
-	@Before
-	public void setUp() throws Exception {
-		tmpDir = new File(System.getProperty("java.io.tmpdir")+"/ZeppelinLTest_"+System.currentTimeMillis());
-		tmpDir.mkdirs();
-		new File(tmpDir, "conf").mkdirs();
-		notebookDir = new File(System.getProperty("java.io.tmpdir")+"/ZeppelinLTest_"+System.currentTimeMillis()+"/notebook");
-		notebookDir.mkdirs();
+  @Before
+  public void setUp() throws Exception {
+    tmpDir = new File(System.getProperty("java.io.tmpdir")+"/ZeppelinLTest_"+System.currentTimeMillis());
+    tmpDir.mkdirs();
+    new File(tmpDir, "conf").mkdirs();
+    notebookDir = new File(System.getProperty("java.io.tmpdir")+"/ZeppelinLTest_"+System.currentTimeMillis()+"/notebook");
+    notebookDir.mkdirs();
 
     System.setProperty(ConfVars.ZEPPELIN_HOME.getVarName(), tmpDir.getAbsolutePath());
-		System.setProperty(ConfVars.ZEPPELIN_NOTEBOOK_DIR.getVarName(), notebookDir.getAbsolutePath());
-		System.setProperty(ConfVars.ZEPPELIN_INTERPRETERS.getVarName(), "org.apache.zeppelin.interpreter.mock.MockInterpreter1,org.apache.zeppelin.interpreter.mock.MockInterpreter2");
+    System.setProperty(ConfVars.ZEPPELIN_NOTEBOOK_DIR.getVarName(), notebookDir.getAbsolutePath());
+    System.setProperty(ConfVars.ZEPPELIN_INTERPRETERS.getVarName(), "org.apache.zeppelin.interpreter.mock.MockInterpreter1,org.apache.zeppelin.interpreter.mock.MockInterpreter2");
 
-		conf = ZeppelinConfiguration.create();
+    conf = ZeppelinConfiguration.create();
 
-		this.schedulerFactory = new SchedulerFactory();
+    this.schedulerFactory = new SchedulerFactory();
 
     MockInterpreter1.register("mock1", "org.apache.zeppelin.interpreter.mock.MockInterpreter1");
     MockInterpreter2.register("mock2", "org.apache.zeppelin.interpreter.mock.MockInterpreter2");
@@ -76,90 +78,131 @@ public class NotebookTest implements JobListenerFactory{
     factory = new InterpreterFactory(conf, new InterpreterOption(false), null);
 
     notebookRepo = new VFSNotebookRepo(conf);
-		notebook = new Notebook(conf, notebookRepo, schedulerFactory, factory, this);
-	}
-
-	@After
-	public void tearDown() throws Exception {
-		delete(tmpDir);
-	}
-
-	@Test
-	public void testSelectingReplImplementation() throws IOException {
-		Note note = notebook.createNote();
-		note.getNoteReplLoader().setInterpreters(factory.getDefaultInterpreterSettingList());
-
-		// run with defatul repl
-		Paragraph p1 = note.addParagraph();
-		p1.setText("hello world");
-		note.run(p1.getId());
-		while(p1.isTerminated()==false || p1.getResult()==null) Thread.yield();
-		assertEquals("repl1: hello world", p1.getResult().message());
-
-		// run with specific repl
-		Paragraph p2 = note.addParagraph();
-		p2.setText("%mock2 hello world");
-		note.run(p2.getId());
-		while(p2.isTerminated()==false || p2.getResult()==null) Thread.yield();
-		assertEquals("repl2: hello world", p2.getResult().message());
-	}
-
-	@Test
-	public void testPersist() throws IOException, SchedulerException{
-		Note note = notebook.createNote();
-
-		// run with default repl
-		Paragraph p1 = note.addParagraph();
-		p1.setText("hello world");
-		note.persist();
-
-		Notebook notebook2 = new Notebook(conf, notebookRepo, schedulerFactory, new InterpreterFactory(conf, null), this);
-		assertEquals(1, notebook2.getAllNotes().size());
-	}
-
-	@Test
-	public void testRunAll() throws IOException {
-		Note note = notebook.createNote();
+    notebook = new Notebook(conf, notebookRepo, schedulerFactory, factory, this);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    delete(tmpDir);
+  }
+
+  @Test
+  public void testSelectingReplImplementation() throws IOException {
+    Note note = notebook.createNote();
     note.getNoteReplLoader().setInterpreters(factory.getDefaultInterpreterSettingList());
 
-		Paragraph p1 = note.addParagraph();
-		p1.setText("p1");
-		Paragraph p2 = note.addParagraph();
-		p2.setText("p2");
-		assertEquals(null, p2.getResult());
-		note.runAll();
-
-		while(p2.isTerminated()==false || p2.getResult()==null) Thread.yield();
-		assertEquals("repl1: p2", p2.getResult().message());
-	}
-
-	@Test
-	public void testSchedule() throws InterruptedException, IOException{
-		// create a note and a paragraph
-		Note note = notebook.createNote();
+    // run with defatul repl
+    Paragraph p1 = note.addParagraph();
+    p1.setText("hello world");
+    note.run(p1.getId());
+    while(p1.isTerminated()==false || p1.getResult()==null) Thread.yield();
+    assertEquals("repl1: hello world", p1.getResult().message());
+
+    // run with specific repl
+    Paragraph p2 = note.addParagraph();
+    p2.setText("%mock2 hello world");
+    note.run(p2.getId());
+    while(p2.isTerminated()==false || p2.getResult()==null) Thread.yield();
+    assertEquals("repl2: hello world", p2.getResult().message());
+  }
+
+  @Test
+  public void testGetAllNotes() throws IOException {
+    // get all notes after copy the {notebookId}/note.json into notebookDir
+    File srcDir = new File("src/test/resources/2A94M5J1Z");
+    File destDir = new File(notebookDir.getAbsolutePath() + "/2A94M5J1Z");
+
+    try {
+      FileUtils.copyDirectory(srcDir, destDir);
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+
+    Note copiedNote = notebookRepo.get("2A94M5J1Z");
+
+    // when ZEPPELIN_NOTEBOOK_GET_FROM_REPO set to be false
+    System.setProperty(ConfVars.ZEPPELIN_NOTEBOOK_RELOAD_FROM_STORAGE.getVarName(), "false");
+    List<Note> notes = notebook.getAllNotes();
+    assertEquals(notes.size(), 0);
+
+    // when ZEPPELIN_NOTEBOOK_GET_FROM_REPO set to be true
+    System.setProperty(ConfVars.ZEPPELIN_NOTEBOOK_RELOAD_FROM_STORAGE.getVarName(), "true");
+    notes = notebook.getAllNotes();
+    assertEquals(notes.size(), 1);
+    assertEquals(notes.get(0).id(), copiedNote.id());
+    assertEquals(notes.get(0).getName(), copiedNote.getName());
+    assertEquals(notes.get(0).getParagraphs(), copiedNote.getParagraphs());
+
+    // get all notes after remove the {notebookId}/note.json from notebookDir
+    // when ZEPPELIN_NOTEBOOK_GET_FROM_REPO set to be false
+    System.setProperty(ConfVars.ZEPPELIN_NOTEBOOK_RELOAD_FROM_STORAGE.getVarName(), "false");
+    // delete the notebook
+    FileUtils.deleteDirectory(destDir);
+    notes = notebook.getAllNotes();
+    assertEquals(notes.size(), 1);
+
+    // when ZEPPELIN_NOTEBOOK_GET_FROM_REPO set to be true
+    System.setProperty(ConfVars.ZEPPELIN_NOTEBOOK_RELOAD_FROM_STORAGE.getVarName(), "true");
+    notes = notebook.getAllNotes();
+    assertEquals(notes.size(), 0);
+  }
+
+  @Test
+  public void testPersist() throws IOException, SchedulerException{
+    Note note = notebook.createNote();
+
+    // run with default repl
+    Paragraph p1 = note.addParagraph();
+    p1.setText("hello world");
+    note.persist();
+
+    Notebook notebook2 = new Notebook(conf, notebookRepo, schedulerFactory, new InterpreterFactory(conf, null), this);
+    assertEquals(1, notebook2.getAllNotes().size());
+  }
+
+  @Test
+  public void testRunAll() throws IOException {
+    Note note = notebook.createNote();
     note.getNoteReplLoader().setInterpreters(factory.getDefaultInterpreterSettingList());
 
-		Paragraph p = note.addParagraph();
-		p.setText("p1");
-		Date dateFinished = p.getDateFinished();
-		assertNull(dateFinished);
-
-		// set cron scheduler, once a second
-		Map<String, Object> config = note.getConfig();
-		config.put("cron", "* * * * * ?");
-		note.setConfig(config);
-		notebook.refreshCron(note.id());
-		Thread.sleep(1*1000);
-		dateFinished = p.getDateFinished();
-		assertNotNull(dateFinished);
-
-		// remove cron scheduler.
-		config.put("cron", null);
-		note.setConfig(config);
-		notebook.refreshCron(note.id());
-		Thread.sleep(1*1000);
-		assertEquals(dateFinished, p.getDateFinished());
-	}
+    Paragraph p1 = note.addParagraph();
+    p1.setText("p1");
+    Paragraph p2 = note.addParagraph();
+    p2.setText("p2");
+    assertEquals(null, p2.getResult());
+    note.runAll();
+
+    while(p2.isTerminated()==false || p2.getResult()==null) Thread.yield();
+    assertEquals("repl1: p2", p2.getResult().message());
+  }
+
+  @Test
+  public void testSchedule() throws InterruptedException, IOException{
+    // create a note and a paragraph
+    Note note = notebook.createNote();
+    note.getNoteReplLoader().setInterpreters(factory.getDefaultInterpreterSettingList());
+
+    Paragraph p = note.addParagraph();
+    p.setText("p1");
+    Date dateFinished = p.getDateFinished();
+    assertNull(dateFinished);
+
+    // set cron scheduler, once a second
+    Map<String, Object> config = note.getConfig();
+    config.put("cron", "* * * * * ?");
+    note.setConfig(config);
+    notebook.refreshCron(note.id());
+    Thread.sleep(1*1000);
+    dateFinished = p.getDateFinished();
+    assertNotNull(dateFinished);
+
+    // remove cron scheduler.
+    config.put("cron", null);
+    note.setConfig(config);
+    notebook.refreshCron(note.id());
+    Thread.sleep(1*1000);
+    assertEquals(dateFinished, p.getDateFinished());
+  }
 
   @Test
   public void testAngularObjectRemovalOnNotebookRemove() throws InterruptedException,
@@ -184,7 +227,7 @@ public class NotebookTest implements JobListenerFactory{
     assertNull(registry.get("o1", note.id()));
     // global object sould be remained
     assertNotNull(registry.get("o2", null));
-	}
+  }
 
   @Test
   public void testAngularObjectRemovalOnInterpreterRestart() throws InterruptedException,
@@ -214,34 +257,34 @@ public class NotebookTest implements JobListenerFactory{
     notebook.removeNote(note.id());
   }
 
-	private void delete(File file){
-		if(file.isFile()) file.delete();
-		else if(file.isDirectory()){
-			File [] files = file.listFiles();
-			if(files!=null && files.length>0){
-				for(File f : files){
-					delete(f);
-				}
-			}
-			file.delete();
-		}
-	}
-
-	@Override
-	public JobListener getParagraphJobListener(Note note) {
-		return new JobListener(){
-
-			@Override
-			public void onProgressUpdate(Job job, int progress) {
-			}
-
-			@Override
-			public void beforeStatusChange(Job job, Status before, Status after) {
-			}
-
-			@Override
-			public void afterStatusChange(Job job, Status before, Status after) {
-			}
-		};
-	}
+  private void delete(File file){
+    if(file.isFile()) file.delete();
+    else if(file.isDirectory()){
+      File [] files = file.listFiles();
+      if(files!=null && files.length>0){
+        for(File f : files){
+          delete(f);
+        }
+      }
+      file.delete();
+    }
+  }
+
+  @Override
+  public JobListener getParagraphJobListener(Note note) {
+    return new JobListener(){
+
+      @Override
+      public void onProgressUpdate(Job job, int progress) {
+      }
+
+      @Override
+      public void beforeStatusChange(Job job, Status before, Status after) {
+      }
+
+      @Override
+      public void afterStatusChange(Job job, Status before, Status after) {
+      }
+    };
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/4b8512bc/zeppelin-zengine/src/test/resources/2A94M5J1Z/note.json
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/resources/2A94M5J1Z/note.json b/zeppelin-zengine/src/test/resources/2A94M5J1Z/note.json
new file mode 100644
index 0000000..785ccea
--- /dev/null
+++ b/zeppelin-zengine/src/test/resources/2A94M5J1Z/note.json
@@ -0,0 +1,341 @@
+{
+  "paragraphs": [
+    {
+      "text": "%md\n## Welcome to Zeppelin.\n##### This is a live tutorial, you can run the code yourself. (Shift-Enter to Run)",
+      "config": {
+        "colWidth": 12.0,
+        "graph": {
+          "mode": "table",
+          "height": 300.0,
+          "optionOpen": false,
+          "keys": [],
+          "values": [],
+          "groups": [],
+          "scatter": {}
+        },
+        "editorHide": true
+      },
+      "settings": {
+        "params": {},
+        "forms": {}
+      },
+      "jobName": "paragraph_1423836981412_-1007008116",
+      "id": "20150213-231621_168813393",
+      "result": {
+        "code": "SUCCESS",
+        "type": "HTML",
+        "msg": "\u003ch2\u003eWelcome to Zeppelin.\u003c/h2\u003e\n\u003ch5\u003eThis is a live tutorial, you can run the code yourself. (Shift-Enter to Run)\u003c/h5\u003e\n"
+      },
+      "dateCreated": "Feb 13, 2015 11:16:21 PM",
+      "dateStarted": "Apr 1, 2015 9:11:09 PM",
+      "dateFinished": "Apr 1, 2015 9:11:10 PM",
+      "status": "FINISHED",
+      "progressUpdateIntervalMs": 500
+    },
+    {
+      "title": "Load data into table",
+      "text": "import org.apache.commons.io.IOUtils\nimport java.net.URL\nimport java.nio.charset.Charset\n\n// Zeppelin creates and injects sc (SparkContext) and sqlContext (HiveContext or SqlContext)\n// So you don\u0027t need create them manually\n\n// load bank data\nval bankText \u003d sc.parallelize(\n    IOUtils.toString(\n        new URL(\"https://s3.amazonaws.com/apache-zeppelin/tutorial/bank/bank.csv\"),\n        Charset.forName(\"utf8\")).split(\"\\n\"))\n\ncase class Bank(age: Integer, job: String, marital: String, education: String, balance: Integer)\n\nval bank \u003d bankText.map(s \u003d\u003e s.split(\";\")).filter(s \u003d\u003e s(0) !\u003d \"\\\"age\\\"\").map(\n    s \u003d\u003e Bank(s(0).toInt, \n            s(1).replaceAll(\"\\\"\", \"\"),\n            s(2).replaceAll(\"\\\"\", \"\"),\n            s(3).replaceAll(\"\\\"\", \"\"),\n            s(5).replaceAll(\"\\\"\", \"\").toInt\n        )\n).toDF()\nbank.registerTempTable(\"bank\")",
+      "config": {
+        "colWidth": 12.0,
+        "graph": {
+          "mode": "table",
+          "height": 300.0,
+          "optionOpen": false,
+          "keys": [],
+          "values": [],
+          "groups": [],
+          "scatter": {}
+        },
+        "title": true
+      },
+      "settings": {
+        "params": {},
+        "forms": {}
+      },
+      "jobName": "paragraph_1423500779206_-1502780787",
+      "id": "20150210-015259_1403135953",
+      "result": {
+        "code": "SUCCESS",
+        "type": "TEXT",
+        "msg": "import org.apache.commons.io.IOUtils\nimport java.net.URL\nimport java.nio.charset.Charset\nbankText: org.apache.spark.rdd.RDD[String] \u003d ParallelCollectionRDD[32] at parallelize at \u003cconsole\u003e:65\ndefined class Bank\nbank: org.apache.spark.sql.DataFrame \u003d [age: int, job: string, marital: string, education: string, balance: int]\n"
+      },
+      "dateCreated": "Feb 10, 2015 1:52:59 AM",
+      "dateStarted": "Jul 3, 2015 1:43:40 PM",
+      "dateFinished": "Jul 3, 2015 1:43:45 PM",
+      "status": "FINISHED",
+      "progressUpdateIntervalMs": 500
+    },
+    {
+      "text": "%sql \nselect age, count(1) value\nfrom bank \nwhere age \u003c 30 \ngroup by age \norder by age",
+      "config": {
+        "colWidth": 4.0,
+        "graph": {
+          "mode": "multiBarChart",
+          "height": 300.0,
+          "optionOpen": false,
+          "keys": [
+            {
+              "name": "age",
+              "index": 0.0,
+              "aggr": "sum"
+            }
+          ],
+          "values": [
+            {
+              "name": "value",
+              "index": 1.0,
+              "aggr": "sum"
+            }
+          ],
+          "groups": [],
+          "scatter": {
+            "xAxis": {
+              "name": "age",
+              "index": 0.0,
+              "aggr": "sum"
+            },
+            "yAxis": {
+              "name": "value",
+              "index": 1.0,
+              "aggr": "sum"
+            }
+          }
+        }
+      },
+      "settings": {
+        "params": {},
+        "forms": {}
+      },
+      "jobName": "paragraph_1423500782552_-1439281894",
+      "id": "20150210-015302_1492795503",
+      "result": {
+        "code": "SUCCESS",
+        "type": "TABLE",
+        "msg": "age\tvalue\n19\t4\n20\t3\n21\t7\n22\t9\n23\t20\n24\t24\n25\t44\n26\t77\n27\t94\n28\t103\n29\t97\n"
+      },
+      "dateCreated": "Feb 10, 2015 1:53:02 AM",
+      "dateStarted": "Jul 3, 2015 1:43:17 PM",
+      "dateFinished": "Jul 3, 2015 1:43:23 PM",
+      "status": "FINISHED",
+      "progressUpdateIntervalMs": 500
+    },
+    {
+      "text": "%sql \nselect age, count(1) value \nfrom bank \nwhere age \u003c ${maxAge\u003d30} \ngroup by age \norder by age",
+      "config": {
+        "colWidth": 4.0,
+        "graph": {
+          "mode": "multiBarChart",
+          "height": 300.0,
+          "optionOpen": false,
+          "keys": [
+            {
+              "name": "age",
+              "index": 0.0,
+              "aggr": "sum"
+            }
+          ],
+          "values": [
+            {
+              "name": "value",
+              "index": 1.0,
+              "aggr": "sum"
+            }
+          ],
+          "groups": [],
+          "scatter": {
+            "xAxis": {
+              "name": "age",
+              "index": 0.0,
+              "aggr": "sum"
+            },
+            "yAxis": {
+              "name": "value",
+              "index": 1.0,
+              "aggr": "sum"
+            }
+          }
+        }
+      },
+      "settings": {
+        "params": {
+          "maxAge": "35"
+        },
+        "forms": {
+          "maxAge": {
+            "name": "maxAge",
+            "defaultValue": "30",
+            "hidden": false
+          }
+        }
+      },
+      "jobName": "paragraph_1423720444030_-1424110477",
+      "id": "20150212-145404_867439529",
+      "result": {
+        "code": "SUCCESS",
+        "type": "TABLE",
+        "msg": "age\tvalue\n19\t4\n20\t3\n21\t7\n22\t9\n23\t20\n24\t24\n25\t44\n26\t77\n27\t94\n28\t103\n29\t97\n30\t150\n31\t199\n32\t224\n33\t186\n34\t231\n"
+      },
+      "dateCreated": "Feb 12, 2015 2:54:04 PM",
+      "dateStarted": "Jul 3, 2015 1:43:28 PM",
+      "dateFinished": "Jul 3, 2015 1:43:29 PM",
+      "status": "FINISHED",
+      "progressUpdateIntervalMs": 500
+    },
+    {
+      "text": "%sql \nselect age, count(1) value \nfrom bank \nwhere marital\u003d\"${marital\u003dsingle,single|divorced|married}\" \ngroup by age \norder by age",
+      "config": {
+        "colWidth": 4.0,
+        "graph": {
+          "mode": "multiBarChart",
+          "height": 300.0,
+          "optionOpen": false,
+          "keys": [
+            {
+              "name": "age",
+              "index": 0.0,
+              "aggr": "sum"
+            }
+          ],
+          "values": [
+            {
+              "name": "value",
+              "index": 1.0,
+              "aggr": "sum"
+            }
+          ],
+          "groups": [],
+          "scatter": {
+            "xAxis": {
+              "name": "age",
+              "index": 0.0,
+              "aggr": "sum"
+            },
+            "yAxis": {
+              "name": "value",
+              "index": 1.0,
+              "aggr": "sum"
+            }
+          }
+        }
+      },
+      "settings": {
+        "params": {
+          "marital": "single"
+        },
+        "forms": {
+          "marital": {
+            "name": "marital",
+            "defaultValue": "single",
+            "options": [
+              {
+                "value": "single"
+              },
+              {
+                "value": "divorced"
+              },
+              {
+                "value": "married"
+              }
+            ],
+            "hidden": false
+          }
+        }
+      },
+      "jobName": "paragraph_1423836262027_-210588283",
+      "id": "20150213-230422_1600658137",
+      "result": {
+        "code": "SUCCESS",
+        "type": "TABLE",
+        "msg": "age\tvalue\n19\t4\n20\t3\n21\t7\n22\t9\n23\t17\n24\t13\n25\t33\n26\t56\n27\t64\n28\t78\n29\t56\n30\t92\n31\t86\n32\t105\n33\t61\n34\t75\n35\t46\n36\t50\n37\t43\n38\t44\n39\t30\n40\t25\n41\t19\n42\t23\n43\t21\n44\t20\n45\t15\n46\t14\n47\t12\n48\t12\n49\t11\n50\t8\n51\t6\n52\t9\n53\t4\n55\t3\n56\t3\n57\t2\n58\t7\n59\t2\n60\t5\n66\t2\n69\t1\n"
+      },
+      "dateCreated": "Feb 13, 2015 11:04:22 PM",
+      "dateStarted": "Jul 3, 2015 1:43:33 PM",
+      "dateFinished": "Jul 3, 2015 1:43:34 PM",
+      "status": "FINISHED",
+      "progressUpdateIntervalMs": 500
+    },
+    {
+      "text": "%md\n## Congratulations, it\u0027s done.\n##### You can create your own notebook in \u0027Notebook\u0027 menu. Good luck!",
+      "config": {
+        "colWidth": 12.0,
+        "graph": {
+          "mode": "table",
+          "height": 300.0,
+          "optionOpen": false,
+          "keys": [],
+          "values": [],
+          "groups": [],
+          "scatter": {}
+        },
+        "editorHide": true
+      },
+      "settings": {
+        "params": {},
+        "forms": {}
+      },
+      "jobName": "paragraph_1423836268492_216498320",
+      "id": "20150213-230428_1231780373",
+      "result": {
+        "code": "SUCCESS",
+        "type": "HTML",
+        "msg": "\u003ch2\u003eCongratulations, it\u0027s done.\u003c/h2\u003e\n\u003ch5\u003eYou can create your own notebook in \u0027Notebook\u0027 menu. Good luck!\u003c/h5\u003e\n"
+      },
+      "dateCreated": "Feb 13, 2015 11:04:28 PM",
+      "dateStarted": "Apr 1, 2015 9:12:18 PM",
+      "dateFinished": "Apr 1, 2015 9:12:18 PM",
+      "status": "FINISHED",
+      "progressUpdateIntervalMs": 500
+    },
+    {
+      "text": "%md\n\nAbout bank data\n\n```\nCitation Request:\n  This dataset is public available for research. The details are described in [Moro et al., 2011]. \n  Please include this citation if you plan to use this database:\n\n  [Moro et al., 2011] S. Moro, R. Laureano and P. Cortez. Using Data Mining for Bank Direct Marketing: An Application of the CRISP-DM Methodology. \n  In P. Novais et al. (Eds.), Proceedings of the European Simulation and Modelling Conference - ESM\u00272011, pp. 117-121, GuimarĂ£es, Portugal, October, 2011. EUROSIS.\n\n  Available at: [pdf] http://hdl.handle.net/1822/14838\n                [bib] http://www3.dsi.uminho.pt/pcortez/bib/2011-esm-1.txt\n```",
+      "config": {
+        "colWidth": 12.0,
+        "graph": {
+          "mode": "table",
+          "height": 300.0,
+          "optionOpen": false,
+          "keys": [],
+          "values": [],
+          "groups": [],
+          "scatter": {}
+        },
+        "editorHide": true
+      },
+      "settings": {
+        "params": {},
+        "forms": {}
+      },
+      "jobName": "paragraph_1427420818407_872443482",
+      "id": "20150326-214658_12335843",
+      "result": {
+        "code": "SUCCESS",
+        "type": "HTML",
+        "msg": "\u003cp\u003eAbout bank data\u003c/p\u003e\n\u003cpre\u003e\u003ccode\u003eCitation Request:\n  This dataset is public available for research. The details are described in [Moro et al., 2011]. \n  Please include this citation if you plan to use this database:\n\n  [Moro et al., 2011] S. Moro, R. Laureano and P. Cortez. Using Data Mining for Bank Direct Marketing: An Application of the CRISP-DM Methodology. \n  In P. Novais et al. (Eds.), Proceedings of the European Simulation and Modelling Conference - ESM\u00272011, pp. 117-121, GuimarĂ£es, Portugal, October, 2011. EUROSIS.\n\n  Available at: [pdf] http://hdl.handle.net/1822/14838\n                [bib] http://www3.dsi.uminho.pt/pcortez/bib/2011-esm-1.txt\n\u003c/code\u003e\u003c/pre\u003e\n"
+      },
+      "dateCreated": "Mar 26, 2015 9:46:58 PM",
+      "dateStarted": "Jul 3, 2015 1:44:56 PM",
+      "dateFinished": "Jul 3, 2015 1:44:56 PM",
+      "status": "FINISHED",
+      "progressUpdateIntervalMs": 500
+    },
+    {
+      "config": {},
+      "settings": {
+        "params": {},
+        "forms": {}
+      },
+      "jobName": "paragraph_1435955447812_-158639899",
+      "id": "20150703-133047_853701097",
+      "dateCreated": "Jul 3, 2015 1:30:47 PM",
+      "status": "READY",
+      "progressUpdateIntervalMs": 500
+    }
+  ],
+  "name": "Zeppelin Tutorial",
+  "id": "2A94M5J1Z",
+  "angularObjects": {},
+  "config": {
+    "looknfeel": "default"
+  },
+  "info": {}
+}
\ No newline at end of file