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

zeppelin git commit: ZEPELIN-3732. Remove JobListenerFactory

Repository: zeppelin
Updated Branches:
  refs/heads/master 82b875cf9 -> 041c8d377


ZEPELIN-3732. Remove JobListenerFactory

### What is this PR for?
This is a refactoring PR. It just remove JobListenerFactory and make NotebookServer as the `ParagraphJobListener` and pass it to Paragraph instead we use `JobListenerFactory` to create JobListener for each Paragraph which is inefficient.

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

### Todos
* [ ] - Task

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

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

### Screenshots (if appropriate)

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

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

Closes #3149 from zjffdu/ZEPPELIN-3732 and squashes the following commits:

d0745b2be [Jeff Zhang] ZEPELIN-3732. Remove JobListenerFactory


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

Branch: refs/heads/master
Commit: 041c8d37782cf05a6a8ddb1103f3f741166111ec
Parents: 82b875c
Author: Jeff Zhang <zj...@apache.org>
Authored: Sat Aug 18 17:19:39 2018 +0800
Committer: Jeff Zhang <zj...@apache.org>
Committed: Mon Aug 20 08:43:43 2018 +0800

----------------------------------------------------------------------
 .../java/org/apache/zeppelin/scheduler/Job.java |   2 +-
 .../apache/zeppelin/scheduler/JobListener.java  |   6 +-
 .../notebook/repo/NotebookRepoSyncTest.java     |  38 +++---
 .../apache/zeppelin/rest/NotebookRestApi.java   |   2 +-
 .../apache/zeppelin/socket/NotebookServer.java  | 124 ++++++++-----------
 .../zeppelin/notebook/JobListenerFactory.java   |  25 ----
 .../java/org/apache/zeppelin/notebook/Note.java |  77 ++----------
 .../org/apache/zeppelin/notebook/Notebook.java  |  15 ++-
 .../zeppelin/notebook/ParagraphJobListener.java |   8 +-
 .../helium/HeliumApplicationFactoryTest.java    |  44 +++----
 .../apache/zeppelin/notebook/FolderTest.java    |  10 +-
 .../zeppelin/notebook/FolderViewTest.java       |   4 +-
 .../org/apache/zeppelin/notebook/NoteTest.java  |  29 ++---
 .../apache/zeppelin/notebook/NotebookTest.java  |  43 +++----
 14 files changed, 159 insertions(+), 268 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/zeppelin/blob/041c8d37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/scheduler/Job.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/scheduler/Job.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/scheduler/Job.java
index d16a8ba..61c54ae 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/scheduler/Job.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/scheduler/Job.java
@@ -137,7 +137,7 @@ public abstract class Job<T> {
     Status before = this.status;
     Status after = status;
     this.status = status;
-    if (listener != null && before != after) {
+    if (listener != null && before != null && before != after) {
       listener.onStatusChange(this, before, after);
     }
   }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/041c8d37/zeppelin-interpreter/src/main/java/org/apache/zeppelin/scheduler/JobListener.java
----------------------------------------------------------------------
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/scheduler/JobListener.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/scheduler/JobListener.java
index 00f8d81..dba2004 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/scheduler/JobListener.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/scheduler/JobListener.java
@@ -20,8 +20,8 @@ package org.apache.zeppelin.scheduler;
 /**
  * Listener for job execution.
  */
-public interface JobListener {
-  void onProgressUpdate(Job job, int progress);
+public interface JobListener<T extends Job> {
+  void onProgressUpdate(T job, int progress);
 
-  void onStatusChange(Job job, Job.Status before, Job.Status after);
+  void onStatusChange(T job, Job.Status before, Job.Status after);
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/041c8d37/zeppelin-plugins/notebookrepo/git/src/test/java/org/apache/zeppelin/notebook/repo/NotebookRepoSyncTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-plugins/notebookrepo/git/src/test/java/org/apache/zeppelin/notebook/repo/NotebookRepoSyncTest.java b/zeppelin-plugins/notebookrepo/git/src/test/java/org/apache/zeppelin/notebook/repo/NotebookRepoSyncTest.java
index 5cc7289..68e2615 100644
--- a/zeppelin-plugins/notebookrepo/git/src/test/java/org/apache/zeppelin/notebook/repo/NotebookRepoSyncTest.java
+++ b/zeppelin-plugins/notebookrepo/git/src/test/java/org/apache/zeppelin/notebook/repo/NotebookRepoSyncTest.java
@@ -27,7 +27,6 @@ import org.apache.zeppelin.interpreter.InterpreterFactory;
 import org.apache.zeppelin.interpreter.InterpreterResultMessage;
 import org.apache.zeppelin.interpreter.InterpreterSettingManager;
 import org.apache.zeppelin.interpreter.remote.RemoteInterpreterProcessListener;
-import org.apache.zeppelin.notebook.JobListenerFactory;
 import org.apache.zeppelin.notebook.Note;
 import org.apache.zeppelin.notebook.Notebook;
 import org.apache.zeppelin.notebook.NotebookAuthorization;
@@ -61,7 +60,7 @@ import static org.mockito.Mockito.mock;
 
 
 //TODO(zjffdu) move it to zeppelin-zengine
-public class NotebookRepoSyncTest implements JobListenerFactory {
+public class NotebookRepoSyncTest implements ParagraphJobListener {
 
   private File mainZepDir;
   private ZeppelinConfiguration conf;
@@ -411,34 +410,31 @@ public class NotebookRepoSyncTest implements JobListenerFactory {
       }
   }
 
-  @Override
-  public ParagraphJobListener getParagraphJobListener(Note note) {
-    return new ParagraphJobListener(){
 
-      @Override
-      public void onOutputAppend(Paragraph paragraph, int idx, String output) {
 
-      }
+  @Override
+  public void onOutputAppend(Paragraph paragraph, int idx, String output) {
 
-      @Override
-      public void onOutputUpdate(Paragraph paragraph, int idx, InterpreterResultMessage msg) {
+  }
 
-      }
+  @Override
+  public void onOutputUpdate(Paragraph paragraph, int idx, InterpreterResultMessage msg) {
 
-      @Override
-      public void onOutputUpdateAll(Paragraph paragraph, List<InterpreterResultMessage> msgs) {
+  }
 
-      }
+  @Override
+  public void onOutputUpdateAll(Paragraph paragraph, List<InterpreterResultMessage> msgs) {
 
-      @Override
-      public void onProgressUpdate(Job job, int progress) {
-      }
+  }
+
+  @Override
+  public void onProgressUpdate(Paragraph paragraph, int progress) {
+  }
 
-      @Override
-      public void onStatusChange(Job job, Status before, Status after) {
+  @Override
+  public void onStatusChange(Paragraph paragraph, Status before, Status after) {
 
-      }
-    };
   }
 
+
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/041c8d37/zeppelin-server/src/main/java/org/apache/zeppelin/rest/NotebookRestApi.java
----------------------------------------------------------------------
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/rest/NotebookRestApi.java b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/NotebookRestApi.java
index d187c4d..bcf89ea 100644
--- a/zeppelin-server/src/main/java/org/apache/zeppelin/rest/NotebookRestApi.java
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/NotebookRestApi.java
@@ -394,7 +394,7 @@ public class NotebookRestApi extends AbstractRestApi {
   @Path("{noteId}")
   @ZeppelinApi
   public Response cloneNote(@PathParam("noteId") String noteId, String message)
-      throws IOException, CloneNotSupportedException, IllegalArgumentException {
+      throws IOException, IllegalArgumentException {
     LOG.info("clone note by JSON {}", message);
     checkIfUserCanWrite(noteId, "Insufficient privileges you cannot clone this note");
     NewNoteRequest request = NewNoteRequest.fromJson(message);

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/041c8d37/zeppelin-server/src/main/java/org/apache/zeppelin/socket/NotebookServer.java
----------------------------------------------------------------------
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/socket/NotebookServer.java b/zeppelin-server/src/main/java/org/apache/zeppelin/socket/NotebookServer.java
index af65517..e1efae6 100644
--- a/zeppelin-server/src/main/java/org/apache/zeppelin/socket/NotebookServer.java
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/socket/NotebookServer.java
@@ -41,7 +41,6 @@ import org.apache.zeppelin.interpreter.remote.RemoteAngularObjectRegistry;
 import org.apache.zeppelin.interpreter.remote.RemoteInterpreterProcessListener;
 import org.apache.zeppelin.interpreter.thrift.InterpreterCompletion;
 import org.apache.zeppelin.notebook.Folder;
-import org.apache.zeppelin.notebook.JobListenerFactory;
 import org.apache.zeppelin.notebook.Note;
 import org.apache.zeppelin.notebook.Notebook;
 import org.apache.zeppelin.notebook.NotebookAuthorization;
@@ -54,7 +53,6 @@ import org.apache.zeppelin.notebook.socket.Message;
 import org.apache.zeppelin.notebook.socket.Message.OP;
 import org.apache.zeppelin.notebook.socket.WatcherMessage;
 import org.apache.zeppelin.rest.exception.ForbiddenException;
-import org.apache.zeppelin.scheduler.Job;
 import org.apache.zeppelin.scheduler.Job.Status;
 import org.apache.zeppelin.server.ZeppelinServer;
 import org.apache.zeppelin.service.ConfigurationService;
@@ -107,10 +105,10 @@ import java.util.regex.Pattern;
  */
 public class NotebookServer extends WebSocketServlet
     implements NotebookSocketListener,
-    JobListenerFactory,
     AngularObjectRegistryListener,
     RemoteInterpreterProcessListener,
     ApplicationEventListener,
+    ParagraphJobListener,
     NotebookServerMBean {
 
   /**
@@ -2117,95 +2115,79 @@ public class NotebookServer extends WebSocketServlet
 
   }
 
-  /**
-   * Need description here.
-   */
-  public static class ParagraphListenerImpl implements ParagraphJobListener {
-    private NotebookServer notebookServer;
-    private Note note;
 
-    public ParagraphListenerImpl(NotebookServer notebookServer, Note note) {
-      this.notebookServer = notebookServer;
-      this.note = note;
-    }
 
-    @Override
-    public void onProgressUpdate(Job job, int progress) {
-      notebookServer.broadcast(note.getId(),
-          new Message(OP.PROGRESS).put("id", job.getId()).put("progress", progress));
-    }
+  @Override
+  public void onProgressUpdate(Paragraph p, int progress) {
+    broadcast(p.getNote().getId(),
+        new Message(OP.PROGRESS).put("id", p.getId()).put("progress", progress));
+  }
 
-    @Override
-    public void onStatusChange(Job job, Status before, Status after) {
-      if (after == Status.ERROR) {
-        if (job.getException() != null) {
-          LOG.error("Error", job.getException());
-        }
+  @Override
+  public void onStatusChange(Paragraph p, Status before, Status after) {
+    if (after == Status.ERROR) {
+      if (p.getException() != null) {
+        LOG.error("Error", p.getException());
       }
+    }
 
-      if (job.isTerminated()) {
-        if (job.getStatus() == Status.FINISHED) {
-          LOG.info("Job {} is finished successfully, status: {}", job.getId(), job.getStatus());
-        } else {
-          LOG.warn("Job {} is finished, status: {}, exception: {}, result: {}", job.getId(),
-              job.getStatus(), job.getException(), job.getReturn());
-        }
-
-        try {
-          //TODO(khalid): may change interface for JobListener and pass subject from interpreter
-          note.persist(job instanceof Paragraph ? ((Paragraph) job).getAuthenticationInfo() : null);
-        } catch (IOException e) {
-          LOG.error(e.toString(), e);
-        }
-      }
-      if (job instanceof Paragraph) {
-        Paragraph p = (Paragraph) job;
-        p.setStatusToUserParagraph(job.getStatus());
-        notebookServer.broadcastParagraph(note, p);
+    if (p.isTerminated()) {
+      if (p.getStatus() == Status.FINISHED) {
+        LOG.info("Job {} is finished successfully, status: {}", p.getId(), p.getStatus());
+      } else {
+        LOG.warn("Job {} is finished, status: {}, exception: {}, result: {}", p.getId(),
+            p.getStatus(), p.getException(), p.getReturn());
       }
+
       try {
-        notebookServer.broadcastUpdateNoteJobInfo(System.currentTimeMillis() - 5000);
+        p.getNote().persist(p.getAuthenticationInfo());
       } catch (IOException e) {
-        LOG.error("can not broadcast for job manager {}", e);
+        LOG.error(e.toString(), e);
       }
     }
 
-    /**
-     * This callback is for paragraph that runs on RemoteInterpreterProcess.
-     */
-    @Override
-    public void onOutputAppend(Paragraph paragraph, int idx, String output) {
-      Message msg =
-          new Message(OP.PARAGRAPH_APPEND_OUTPUT).put("noteId", paragraph.getNote().getId())
-              .put("paragraphId", paragraph.getId()).put("data", output);
+    p.setStatusToUserParagraph(p.getStatus());
+    broadcastParagraph(p.getNote(), p);
+    for (NotebookEventListener listener : notebook().getNotebookEventListeners()) {
+      listener.onParagraphStatusChange(p, after);
+    }
 
-      notebookServer.broadcast(paragraph.getNote().getId(), msg);
+    try {
+      broadcastUpdateNoteJobInfo(System.currentTimeMillis() - 5000);
+    } catch (IOException e) {
+      LOG.error("can not broadcast for job manager {}", e);
     }
+  }
 
-    /**
-     * This callback is for paragraph that runs on RemoteInterpreterProcess.
-     */
-    @Override
-    public void onOutputUpdate(Paragraph paragraph, int idx, InterpreterResultMessage result) {
-      String output = result.getData();
-      Message msg =
-          new Message(OP.PARAGRAPH_UPDATE_OUTPUT).put("noteId", paragraph.getNote().getId())
-              .put("paragraphId", paragraph.getId()).put("data", output);
 
-      notebookServer.broadcast(paragraph.getNote().getId(), msg);
-    }
+  /**
+   * This callback is for paragraph that runs on RemoteInterpreterProcess.
+   */
+  @Override
+  public void onOutputAppend(Paragraph paragraph, int idx, String output) {
+    Message msg =
+        new Message(OP.PARAGRAPH_APPEND_OUTPUT).put("noteId", paragraph.getNote().getId())
+            .put("paragraphId", paragraph.getId()).put("data", output);
+    broadcast(paragraph.getNote().getId(), msg);
+  }
 
-    @Override
-    public void onOutputUpdateAll(Paragraph paragraph, List<InterpreterResultMessage> msgs) {
-      // TODO
-    }
+  /**
+   * This callback is for paragraph that runs on RemoteInterpreterProcess.
+   */
+  @Override
+  public void onOutputUpdate(Paragraph paragraph, int idx, InterpreterResultMessage result) {
+    Message msg =
+        new Message(OP.PARAGRAPH_UPDATE_OUTPUT).put("noteId", paragraph.getNote().getId())
+            .put("paragraphId", paragraph.getId()).put("data", result.getData());
+    broadcast(paragraph.getNote().getId(), msg);
   }
 
   @Override
-  public ParagraphJobListener getParagraphJobListener(Note note) {
-    return new ParagraphListenerImpl(this, note);
+  public void onOutputUpdateAll(Paragraph paragraph, List<InterpreterResultMessage> msgs) {
+    // TODO
   }
 
+
   public NotebookEventListener getNotebookInformationListener() {
     return new NotebookInformationListener(this);
   }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/041c8d37/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/JobListenerFactory.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/JobListenerFactory.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/JobListenerFactory.java
deleted file mode 100644
index 1387730..0000000
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/JobListenerFactory.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * 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.notebook;
-
-/**
- * TODO(moon): provide description.
- */
-public interface JobListenerFactory {
-  public ParagraphJobListener getParagraphJobListener(Note note);
-}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/041c8d37/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 cea5c28..437b59b 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
@@ -64,7 +64,7 @@ import static java.lang.String.format;
 /**
  * Binded interpreters for a note
  */
-public class Note implements ParagraphJobListener, JsonSerializable {
+public class Note implements JsonSerializable {
   private static final Logger logger = LoggerFactory.getLogger(Note.class);
   private static final long serialVersionUID = 7920699076577612429L;
   private static Gson gson = new GsonBuilder()
@@ -93,7 +93,7 @@ public class Note implements ParagraphJobListener, JsonSerializable {
 
   private transient InterpreterFactory factory;
   private transient InterpreterSettingManager interpreterSettingManager;
-  private transient JobListenerFactory jobListenerFactory;
+  private transient ParagraphJobListener paragraphJobListener;
   private transient NotebookRepo repo;
   private transient SearchService index;
   private transient ScheduledFuture delayedPersist;
@@ -120,14 +120,14 @@ public class Note implements ParagraphJobListener, JsonSerializable {
   }
 
   public Note(String name, String defaultInterpreterGroup, NotebookRepo repo, InterpreterFactory factory,
-      InterpreterSettingManager interpreterSettingManager, JobListenerFactory jlFactory,
+      InterpreterSettingManager interpreterSettingManager, ParagraphJobListener paragraphJobListener,
       SearchService noteIndex, Credentials credentials, NoteEventListener noteEventListener) {
     this.name = name;
     this.defaultInterpreterGroup = defaultInterpreterGroup;
     this.repo = repo;
     this.factory = factory;
     this.interpreterSettingManager = interpreterSettingManager;
-    this.jobListenerFactory = jlFactory;
+    this.paragraphJobListener = paragraphJobListener;
     this.index = noteIndex;
     this.noteEventListener = noteEventListener;
     this.credentials = credentials;
@@ -293,7 +293,7 @@ public class Note implements ParagraphJobListener, JsonSerializable {
     boolean foundParagraph = false;
     for (Paragraph ownParagraph : paragraphs) {
       if (paragraph.getId().equals(ownParagraph.getId())) {
-        paragraph.setListener(this.jobListenerFactory.getParagraphJobListener(this));
+        paragraph.setListener(paragraphJobListener);
         foundParagraph = true;
       }
     }
@@ -305,8 +305,8 @@ public class Note implements ParagraphJobListener, JsonSerializable {
     }
   }
 
-  void setJobListenerFactory(JobListenerFactory jobListenerFactory) {
-    this.jobListenerFactory = jobListenerFactory;
+  void setParagraphJobListener(ParagraphJobListener paragraphJobListener) {
+    this.paragraphJobListener = paragraphJobListener;
   }
 
   void setNotebookRepo(NotebookRepo repo) {
@@ -366,7 +366,8 @@ public class Note implements ParagraphJobListener, JsonSerializable {
   void addCloneParagraph(Paragraph srcParagraph, AuthenticationInfo subject) {
 
     // Keep paragraph original ID
-    final Paragraph newParagraph = new Paragraph(srcParagraph.getId(), this, this, factory);
+    final Paragraph newParagraph = new Paragraph(srcParagraph.getId(), this,
+        paragraphJobListener, factory);
 
     Map<String, Object> config = new HashMap<>(srcParagraph.getConfig());
     Map<String, Object> param = srcParagraph.settings.getParams();
@@ -415,7 +416,7 @@ public class Note implements ParagraphJobListener, JsonSerializable {
   }
 
   private Paragraph createParagraph(int index, AuthenticationInfo authenticationInfo) {
-    Paragraph p = new Paragraph(this, this, factory);
+    Paragraph p = new Paragraph(this, paragraphJobListener, factory);
     p.setAuthenticationInfo(authenticationInfo);
     setParagraphMagic(p, index);
     return p;
@@ -692,7 +693,7 @@ public class Note implements ParagraphJobListener, JsonSerializable {
    */
   public boolean run(String paragraphId, boolean blocking) {
     Paragraph p = getParagraph(paragraphId);
-    p.setListener(jobListenerFactory.getParagraphJobListener(this));
+    p.setListener(this.paragraphJobListener);
     return p.execute(blocking);
   }
 
@@ -722,7 +723,7 @@ public class Note implements ParagraphJobListener, JsonSerializable {
 
   public List<InterpreterCompletion> completion(String paragraphId, String buffer, int cursor) {
     Paragraph p = getParagraph(paragraphId);
-    p.setListener(jobListenerFactory.getParagraphJobListener(this));
+    p.setListener(this.paragraphJobListener);
 
     return p.completion(buffer, cursor);
   }
@@ -887,60 +888,6 @@ public class Note implements ParagraphJobListener, JsonSerializable {
   public void setInfo(Map<String, Object> info) {
     this.info = info;
   }
-  
-  @Override
-  public void onStatusChange(Job job, Status before, Status after) {
-    if (jobListenerFactory != null) {
-      ParagraphJobListener listener = jobListenerFactory.getParagraphJobListener(this);
-      if (listener != null) {
-        listener.onStatusChange(job, before, after);
-      }
-    }
-
-    if (noteEventListener != null) {
-      noteEventListener.onParagraphStatusChange((Paragraph) job, after);
-    }
-  }
-
-  @Override
-  public void onProgressUpdate(Job job, int progress) {
-    if (jobListenerFactory != null) {
-      ParagraphJobListener listener = jobListenerFactory.getParagraphJobListener(this);
-      if (listener != null) {
-        listener.onProgressUpdate(job, progress);
-      }
-    }
-  }
-
-  @Override
-  public void onOutputAppend(Paragraph paragraph, int idx, String output) {
-    if (jobListenerFactory != null) {
-      ParagraphJobListener listener = jobListenerFactory.getParagraphJobListener(this);
-      if (listener != null) {
-        listener.onOutputAppend(paragraph, idx, output);
-      }
-    }
-  }
-
-  @Override
-  public void onOutputUpdate(Paragraph paragraph, int idx, InterpreterResultMessage msg) {
-    if (jobListenerFactory != null) {
-      ParagraphJobListener listener = jobListenerFactory.getParagraphJobListener(this);
-      if (listener != null) {
-        listener.onOutputUpdate(paragraph, idx, msg);
-      }
-    }
-  }
-
-  @Override
-  public void onOutputUpdateAll(Paragraph paragraph, List<InterpreterResultMessage> msgs) {
-    if (jobListenerFactory != null) {
-      ParagraphJobListener listener = jobListenerFactory.getParagraphJobListener(this);
-      if (listener != null) {
-        listener.onOutputUpdateAll(paragraph, msgs);
-      }
-    }
-  }
 
   void setNoteEventListener(NoteEventListener noteEventListener) {
     this.noteEventListener = noteEventListener;

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/041c8d37/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 86feb36..80b076f 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
@@ -43,7 +43,6 @@ import org.apache.zeppelin.interpreter.InterpreterException;
 import org.apache.zeppelin.interpreter.InterpreterFactory;
 import org.apache.zeppelin.interpreter.InterpreterGroup;
 import org.apache.zeppelin.interpreter.InterpreterNotFoundException;
-import org.apache.zeppelin.interpreter.InterpreterResult;
 import org.apache.zeppelin.interpreter.InterpreterSetting;
 import org.apache.zeppelin.interpreter.InterpreterSettingManager;
 import org.apache.zeppelin.interpreter.ManagedInterpreterGroup;
@@ -89,7 +88,7 @@ public class Notebook implements NoteEventListener {
   private ZeppelinConfiguration conf;
   private StdSchedulerFactory quertzSchedFact;
   private org.quartz.Scheduler quartzSched;
-  private JobListenerFactory jobListenerFactory;
+  private ParagraphJobListener paragraphJobListener;
   private NotebookRepo notebookRepo;
   private SearchService noteSearchService;
   private NotebookAuthorization notebookAuthorization;
@@ -106,7 +105,7 @@ public class Notebook implements NoteEventListener {
    */
   public Notebook(ZeppelinConfiguration conf, NotebookRepo notebookRepo,
       SchedulerFactory schedulerFactory, InterpreterFactory replFactory,
-      InterpreterSettingManager interpreterSettingManager, JobListenerFactory jobListenerFactory,
+      InterpreterSettingManager interpreterSettingManager, ParagraphJobListener paragraphJobListener,
       SearchService noteSearchService, NotebookAuthorization notebookAuthorization,
       Credentials credentials) throws IOException, SchedulerException {
     this.conf = conf;
@@ -114,7 +113,7 @@ public class Notebook implements NoteEventListener {
     this.schedulerFactory = schedulerFactory;
     this.replFactory = replFactory;
     this.interpreterSettingManager = interpreterSettingManager;
-    this.jobListenerFactory = jobListenerFactory;
+    this.paragraphJobListener = paragraphJobListener;
     this.noteSearchService = noteSearchService;
     this.notebookAuthorization = notebookAuthorization;
     this.credentials = credentials;
@@ -152,7 +151,7 @@ public class Notebook implements NoteEventListener {
       throws IOException {
     Note note =
         new Note(name, defaultInterpreterGroup, notebookRepo, replFactory, interpreterSettingManager,
-            jobListenerFactory, noteSearchService, credentials, this);
+            paragraphJobListener, noteSearchService, credentials, this);
     note.setNoteNameListener(folders);
 
     synchronized (notes) {
@@ -420,7 +419,7 @@ public class Notebook implements NoteEventListener {
     note.setInterpreterFactory(replFactory);
     note.setInterpreterSettingManager(interpreterSettingManager);
 
-    note.setJobListenerFactory(jobListenerFactory);
+    note.setParagraphJobListener(this.paragraphJobListener);
     note.setNotebookRepo(notebookRepo);
     note.setCronSupported(getConf());
 
@@ -552,6 +551,10 @@ public class Notebook implements NoteEventListener {
     return folders.renameFolder(oldFolderId, newFolderId);
   }
 
+  public List<NotebookEventListener> getNotebookEventListeners() {
+    return notebookEventListeners;
+  }
+
   public List<Note> getNotesUnderFolder(String folderId) {
     return folders.getFolder(folderId).getNotesRecursively();
   }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/041c8d37/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/ParagraphJobListener.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/ParagraphJobListener.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/ParagraphJobListener.java
index 8329751..8743fb7 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/ParagraphJobListener.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/ParagraphJobListener.java
@@ -27,8 +27,8 @@ import java.util.List;
 /**
  * Listen paragraph update
  */
-public interface ParagraphJobListener extends JobListener {
-  public void onOutputAppend(Paragraph paragraph, int idx, String output);
-  public void onOutputUpdate(Paragraph paragraph, int idx, InterpreterResultMessage msg);
-  public void onOutputUpdateAll(Paragraph paragraph, List<InterpreterResultMessage> msgs);
+public interface ParagraphJobListener extends JobListener<Paragraph> {
+  void onOutputAppend(Paragraph paragraph, int idx, String output);
+  void onOutputUpdate(Paragraph paragraph, int idx, InterpreterResultMessage msg);
+  void onOutputUpdateAll(Paragraph paragraph, List<InterpreterResultMessage> msgs);
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/041c8d37/zeppelin-zengine/src/test/java/org/apache/zeppelin/helium/HeliumApplicationFactoryTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/helium/HeliumApplicationFactoryTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/helium/HeliumApplicationFactoryTest.java
index 3d3df9f..67a5947 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/helium/HeliumApplicationFactoryTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/helium/HeliumApplicationFactoryTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.zeppelin.helium;
 
-import org.apache.zeppelin.conf.ZeppelinConfiguration;
 import org.apache.zeppelin.interpreter.AbstractInterpreterTest;
 import org.apache.zeppelin.interpreter.Interpreter;
 import org.apache.zeppelin.interpreter.InterpreterException;
@@ -24,13 +23,11 @@ import org.apache.zeppelin.interpreter.InterpreterNotFoundException;
 import org.apache.zeppelin.interpreter.InterpreterResultMessage;
 import org.apache.zeppelin.interpreter.InterpreterSetting;
 import org.apache.zeppelin.notebook.ApplicationState;
-import org.apache.zeppelin.notebook.JobListenerFactory;
 import org.apache.zeppelin.notebook.Note;
 import org.apache.zeppelin.notebook.Notebook;
 import org.apache.zeppelin.notebook.NotebookAuthorization;
 import org.apache.zeppelin.notebook.Paragraph;
 import org.apache.zeppelin.notebook.ParagraphJobListener;
-//import org.apache.zeppelin.notebook.repo.VFSNotebookRepo;
 import org.apache.zeppelin.notebook.repo.NotebookRepo;
 import org.apache.zeppelin.scheduler.Job;
 import org.apache.zeppelin.scheduler.SchedulerFactory;
@@ -42,14 +39,15 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.LinkedList;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 
-public class HeliumApplicationFactoryTest extends AbstractInterpreterTest implements JobListenerFactory {
+
+public class HeliumApplicationFactoryTest extends AbstractInterpreterTest
+    implements ParagraphJobListener {
 
   private SchedulerFactory schedulerFactory;
   private NotebookRepo notebookRepo;
@@ -287,33 +285,29 @@ public class HeliumApplicationFactoryTest extends AbstractInterpreterTest implem
     notebook.removeNote(note1.getId(), anonymous);
   }
 
-  @Override
-  public ParagraphJobListener getParagraphJobListener(Note note) {
-    return new ParagraphJobListener() {
-      @Override
-      public void onOutputAppend(Paragraph paragraph, int idx, String output) {
 
-      }
+    @Override
+    public void onOutputAppend(Paragraph paragraph, int idx, String output) {
 
-      @Override
-      public void onOutputUpdate(Paragraph paragraph, int idx, InterpreterResultMessage msg) {
+    }
 
-      }
+    @Override
+    public void onOutputUpdate(Paragraph paragraph, int idx, InterpreterResultMessage msg) {
 
-      @Override
-      public void onOutputUpdateAll(Paragraph paragraph, List<InterpreterResultMessage> msgs) {
+    }
 
-      }
+    @Override
+    public void onOutputUpdateAll(Paragraph paragraph, List<InterpreterResultMessage> msgs) {
 
-      @Override
-      public void onProgressUpdate(Job job, int progress) {
+    }
 
-      }
+    @Override
+    public void onProgressUpdate(Paragraph paragraph, int progress) {
 
-      @Override
-      public void onStatusChange(Job job, Job.Status before, Job.Status after) {
+    }
 
-      }
-    };
-  }
+    @Override
+    public void onStatusChange(Paragraph paragraph, Job.Status before, Job.Status after) {
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/041c8d37/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/FolderTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/FolderTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/FolderTest.java
index 0e86ad6..d756e14 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/FolderTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/FolderTest.java
@@ -40,7 +40,7 @@ public class FolderTest {
   NotebookRepo repo;
 
   @Mock
-  JobListenerFactory jobListenerFactory;
+  ParagraphJobListener paragraphJobListener;
 
   @Mock
   SearchService index;
@@ -71,13 +71,13 @@ public class FolderTest {
 
   @Before
   public void createFolderAndNotes() {
-    note1 = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    note1 = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
     note1.setName("this/is/a/folder/note1");
 
-    note2 = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    note2 = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
     note2.setName("this/is/a/folder/note2");
 
-    note3 = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    note3 = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
     note3.setName("this/is/a/folder/note3");
 
     folder = new Folder("this/is/a/folder");
@@ -118,7 +118,7 @@ public class FolderTest {
 
   @Test
   public void addNoteTest() {
-    Note note4 = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    Note note4 = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
     note4.setName("this/is/a/folder/note4");
 
     folder.addNote(note4);

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/041c8d37/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/FolderViewTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/FolderViewTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/FolderViewTest.java
index 9521355..e127be8 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/FolderViewTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/FolderViewTest.java
@@ -42,7 +42,7 @@ public class FolderViewTest {
   NotebookRepo repo;
 
   @Mock
-  JobListenerFactory jobListenerFactory;
+  ParagraphJobListener paragraphJobListener;
 
   @Mock
   SearchService index;
@@ -89,7 +89,7 @@ public class FolderViewTest {
   Note abNote2;
 
   private Note createNote() {
-    Note note = new Note("test", "test", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    Note note = new Note("test", "test", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
     note.setNoteNameListener(folderView);
     return note;
   }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/041c8d37/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NoteTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NoteTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NoteTest.java
index a3ce868..d8e7f13 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NoteTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/NoteTest.java
@@ -18,7 +18,6 @@
 package org.apache.zeppelin.notebook;
 
 import com.google.common.collect.Lists;
-import com.google.gson.Gson;
 import org.apache.zeppelin.display.AngularObject;
 import org.apache.zeppelin.display.ui.TextBox;
 import org.apache.zeppelin.interpreter.Interpreter;
@@ -37,8 +36,6 @@ import org.mockito.ArgumentCaptor;
 import org.mockito.Mock;
 import org.mockito.runners.MockitoJUnitRunner;
 
-import java.util.Date;
-
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
@@ -48,7 +45,7 @@ public class NoteTest {
   NotebookRepo repo;
 
   @Mock
-  JobListenerFactory jobListenerFactory;
+  ParagraphJobListener paragraphJobListener;
 
   @Mock
   SearchService index;
@@ -79,7 +76,7 @@ public class NoteTest {
     when(interpreter.getScheduler()).thenReturn(scheduler);
 
     String pText = "%spark sc.version";
-    Note note = new Note("test", "test", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    Note note = new Note("test", "test", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
 
     Paragraph p = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
     p.setText(pText);
@@ -95,7 +92,7 @@ public class NoteTest {
 
   @Test
   public void addParagraphWithEmptyReplNameTest() {
-    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
 
     Paragraph p = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
     assertNull(p.getText());
@@ -105,7 +102,7 @@ public class NoteTest {
   public void addParagraphWithLastReplNameTest() throws InterpreterNotFoundException {
     when(interpreterFactory.getInterpreter(anyString(), anyString(), eq("spark"), anyString())).thenReturn(interpreter);
 
-    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
     Paragraph p1 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
     p1.setText("%spark ");
     Paragraph p2 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
@@ -117,7 +114,7 @@ public class NoteTest {
   public void insertParagraphWithLastReplNameTest() throws InterpreterNotFoundException {
     when(interpreterFactory.getInterpreter(anyString(), anyString(), eq("spark"), anyString())).thenReturn(interpreter);
 
-    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
     Paragraph p1 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
     p1.setText("%spark ");
     Paragraph p2 = note.insertNewParagraph(note.getParagraphs().size(), AuthenticationInfo.ANONYMOUS);
@@ -129,7 +126,7 @@ public class NoteTest {
   public void insertParagraphWithInvalidReplNameTest() throws InterpreterNotFoundException {
     when(interpreterFactory.getInterpreter(anyString(), anyString(), eq("invalid"), anyString())).thenReturn(null);
 
-    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
     Paragraph p1 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
     p1.setText("%invalid ");
     Paragraph p2 = note.insertNewParagraph(note.getParagraphs().size(), AuthenticationInfo.ANONYMOUS);
@@ -139,7 +136,7 @@ public class NoteTest {
 
   @Test
   public void insertParagraphwithUser() {
-    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
     Paragraph p = note.insertNewParagraph(note.getParagraphs().size(), AuthenticationInfo.ANONYMOUS);
     assertEquals("anonymous", p.getUser());
   }
@@ -149,7 +146,7 @@ public class NoteTest {
     when(interpreterFactory.getInterpreter(anyString(), anyString(), eq("md"), anyString())).thenReturn(interpreter);
     when(interpreter.getScheduler()).thenReturn(scheduler);
 
-    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
     Paragraph p1 = note.addNewParagraph(AuthenticationInfo.ANONYMOUS);
     InterpreterResult result = new InterpreterResult(InterpreterResult.Code.SUCCESS, InterpreterResult.Type.TEXT, "result");
     p1.setResult(result);
@@ -165,7 +162,7 @@ public class NoteTest {
 
   @Test
   public void getFolderIdTest() {
-    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
     // Ordinary case test
     note.setName("this/is/a/folder/noteName");
     assertEquals("this/is/a/folder", note.getFolderId());
@@ -181,7 +178,7 @@ public class NoteTest {
 
   @Test
   public void getNameWithoutPathTest() {
-    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
     // Notes in the root folder
     note.setName("noteOnRootFolder");
     assertEquals("noteOnRootFolder", note.getNameWithoutPath());
@@ -196,7 +193,7 @@ public class NoteTest {
 
   @Test
   public void isTrashTest() {
-    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
     // Notes in the root folder
     note.setName("noteOnRootFolder");
     assertFalse(note.isTrash());
@@ -220,7 +217,7 @@ public class NoteTest {
 
   @Test
   public void personalizedModeReturnDifferentParagraphInstancePerUser() {
-    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
 
     String user1 = "user1";
     String user2 = "user2";
@@ -235,7 +232,7 @@ public class NoteTest {
   }
 
   public void testNoteJson() {
-    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, jobListenerFactory, index, credentials, noteEventListener);
+    Note note = new Note("test", "", repo, interpreterFactory, interpreterSettingManager, paragraphJobListener, index, credentials, noteEventListener);
     note.setName("/test_note");
     note.getConfig().put("config_1", "value_1");
     note.getInfo().put("info_1", "value_1");

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/041c8d37/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 3e9958d..c5a7512 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
@@ -74,7 +74,7 @@ import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 
 
-public class NotebookTest extends AbstractInterpreterTest implements JobListenerFactory {
+public class NotebookTest extends AbstractInterpreterTest implements ParagraphJobListener {
   private static final Logger logger = LoggerFactory.getLogger(NotebookTest.class);
 
   private SchedulerFactory schedulerFactory;
@@ -1458,38 +1458,35 @@ public class NotebookTest extends AbstractInterpreterTest implements JobListener
     }
   }
 
-  @Override
-  public ParagraphJobListener getParagraphJobListener(Note note) {
-    return new ParagraphJobListener() {
 
-      @Override
-      public void onOutputAppend(Paragraph paragraph, int idx, String output) {
 
-      }
+  @Override
+  public void onOutputAppend(Paragraph paragraph, int idx, String output) {
 
-      @Override
-      public void onOutputUpdate(Paragraph paragraph, int idx, InterpreterResultMessage msg) {
+  }
 
-      }
+  @Override
+  public void onOutputUpdate(Paragraph paragraph, int idx, InterpreterResultMessage msg) {
 
-      @Override
-      public void onOutputUpdateAll(Paragraph paragraph, List<InterpreterResultMessage> msgs) {
+  }
 
-      }
+  @Override
+  public void onOutputUpdateAll(Paragraph paragraph, List<InterpreterResultMessage> msgs) {
 
-      @Override
-      public void onProgressUpdate(Job job, int progress) {
-      }
+  }
 
-      @Override
-      public void onStatusChange(Job job, Status before, Status after) {
-        if (afterStatusChangedListener != null) {
-          afterStatusChangedListener.onStatusChanged(job, before, after);
-        }
-      }
-    };
+  @Override
+  public void onProgressUpdate(Paragraph paragraph, int progress) {
+  }
+
+  @Override
+  public void onStatusChange(Paragraph paragraph, Status before, Status after) {
+    if (afterStatusChangedListener != null) {
+      afterStatusChangedListener.onStatusChanged(paragraph, before, after);
+    }
   }
 
+
   private interface StatusChangedListener {
     void onStatusChanged(Job job, Status before, Status after);
   }