You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zeppelin.apache.org by bz...@apache.org on 2016/06/17 01:34:37 UTC

zeppelin git commit: Update and refactor NotebookRepo versioning API

Repository: zeppelin
Updated Branches:
  refs/heads/master c806d4a4c -> ff197d062


Update and refactor NotebookRepo versioning API

### What is this PR for?
This is firstly to refactor API for versioning and keep everthing inside of one interface (NotebookRepo) instead of two different interfaces (NotebookRepoVersioned). Secondly, there're modifications to existing versioning api, with considerations of future complete implementation of versioning. Note that this PR doesn't implement all suggested interfaces, but lays foundation for their implementation.

### What type of PR is it?
Improvement && Refactoring

### Todos
* [x] - move versioning api (get, history) from NotebookRepoVersioned to NotebookRepo
* [x] - refactor and naming changes
* [x] - modify checkpoint api (add return value) and modify NotebookRepoSync to deal with it

### What is the Jira issue?

### How should this be tested?
Basically it doesn't add new functionality, so the only requirement is for tests to pass.

### Screenshots (if appropriate)

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

Author: Khalid Huseynov <kh...@nflabs.com>

Closes #1007 from khalidhuseynov/repo/versioning-api-update and squashes the following commits:

f900058 [Khalid Huseynov] Rev -> Revision
17eee08 [Khalid Huseynov] fix checkstyle
9140b16 [Khalid Huseynov] fix tests
ea46851 [Khalid Huseynov] apply changes to NotebookRepoSync
e82d8a9 [Khalid Huseynov] propagate changes to all repos
b68dd26 [Khalid Huseynov] move and update versioning api


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

Branch: refs/heads/master
Commit: ff197d06227001e671e7fa06deca7a8a64813fef
Parents: c806d4a
Author: Khalid Huseynov <kh...@nflabs.com>
Authored: Tue Jun 14 11:18:22 2016 -0700
Committer: Alexander Bezzubov <bz...@apache.org>
Committed: Fri Jun 17 10:34:25 2016 +0900

----------------------------------------------------------------------
 .../notebook/repo/AzureNotebookRepo.java        | 15 ++++-
 .../zeppelin/notebook/repo/GitNotebookRepo.java | 20 ++++---
 .../zeppelin/notebook/repo/NotebookRepo.java    | 43 +++++++++++++-
 .../notebook/repo/NotebookRepoSync.java         | 30 ++++++++--
 .../notebook/repo/NotebookRepoVersioned.java    | 60 --------------------
 .../zeppelin/notebook/repo/S3NotebookRepo.java  | 18 +++++-
 .../zeppelin/notebook/repo/VFSNotebookRepo.java | 18 +++++-
 .../repo/zeppelinhub/ZeppelinHubRepo.java       | 17 +++++-
 .../notebook/repo/GitNotebookRepoTest.java      | 12 ++--
 .../notebook/repo/NotebookRepoSyncTest.java     |  4 +-
 10 files changed, 145 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/zeppelin/blob/ff197d06/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/AzureNotebookRepo.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/AzureNotebookRepo.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/AzureNotebookRepo.java
index 14c56ec..3a3bffd 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/AzureNotebookRepo.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/AzureNotebookRepo.java
@@ -205,8 +205,21 @@ public class AzureNotebookRepo implements NotebookRepo {
   }
 
   @Override
-  public void checkpoint(String noteId, String checkPointName) throws IOException {
+  public Revision checkpoint(String noteId, String checkpointMsg) throws IOException {
     // no-op
     LOG.info("Checkpoint feature isn't supported in {}", this.getClass().toString());
+    return null;
+  }
+
+  @Override
+  public Note get(String noteId, Revision rev) throws IOException {
+    // Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<Revision> revisionHistory(String noteId) {
+    // Auto-generated method stub
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/ff197d06/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/GitNotebookRepo.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/GitNotebookRepo.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/GitNotebookRepo.java
index 85a534e..2ab7c60 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/GitNotebookRepo.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/GitNotebookRepo.java
@@ -46,7 +46,7 @@ import com.google.common.collect.Lists;
  *
  *   TODO(bzz): add default .gitignore
  */
-public class GitNotebookRepo extends VFSNotebookRepo implements NotebookRepoVersioned {
+public class GitNotebookRepo extends VFSNotebookRepo {
   private static final Logger LOG = LoggerFactory.getLogger(GitNotebookRepo.class);
 
   private String localPath;
@@ -71,41 +71,44 @@ public class GitNotebookRepo extends VFSNotebookRepo implements NotebookRepoVers
 
   /* implemented as git add+commit
    * @param pattern is the noteId
-   * @param commitMessage is a commit message (checkpoint name)
+   * @param commitMessage is a commit message (checkpoint message)
    * (non-Javadoc)
    * @see org.apache.zeppelin.notebook.repo.VFSNotebookRepo#checkpoint(String, String)
    */
   @Override
-  public void checkpoint(String pattern, String commitMessage) {
+  public Revision checkpoint(String pattern, String commitMessage) {
+    Revision revision = null;
     try {
       List<DiffEntry> gitDiff = git.diff().call();
       if (!gitDiff.isEmpty()) {
         LOG.debug("Changes found for pattern '{}': {}", pattern, gitDiff);
         DirCache added = git.add().addFilepattern(pattern).call();
         LOG.debug("{} changes are about to be commited", added.getEntryCount());
-        git.commit().setMessage(commitMessage).call();
+        RevCommit commit = git.commit().setMessage(commitMessage).call();
+        revision = new Revision(commit.getName(), commit.getShortMessage(), commit.getCommitTime());
       } else {
         LOG.debug("No changes found {}", pattern);
       }
     } catch (GitAPIException e) {
       LOG.error("Failed to add+comit {} to Git", pattern, e);
     }
+    return revision;
   }
 
   @Override
-  public Note get(String noteId, String rev) throws IOException {
+  public Note get(String noteId, Revision rev) throws IOException {
     //TODO(bzz): something like 'git checkout rev', that will not change-the-world though
     return super.get(noteId);
   }
 
   @Override
-  public List<Rev> history(String noteId) {
-    List<Rev> history = Lists.newArrayList();
+  public List<Revision> revisionHistory(String noteId) {
+    List<Revision> history = Lists.newArrayList();
     LOG.debug("Listing history for {}:", noteId);
     try {
       Iterable<RevCommit> logs = git.log().addPath(noteId).call();
       for (RevCommit log: logs) {
-        history.add(new Rev(log.getName(), log.getCommitTime()));
+        history.add(new Revision(log.getName(), log.getShortMessage(), log.getCommitTime()));
         LOG.debug(" - ({},{},{})", log.getName(), log.getCommitTime(), log.getFullMessage());
       }
     } catch (NoHeadException e) {
@@ -131,5 +134,4 @@ public class GitNotebookRepo extends VFSNotebookRepo implements NotebookRepoVers
     this.git = git;
   }
 
-
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/ff197d06/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/NotebookRepo.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/NotebookRepo.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/NotebookRepo.java
index 7608892..855b7ad 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/NotebookRepo.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/NotebookRepo.java
@@ -39,7 +39,46 @@ public interface NotebookRepo {
   @ZeppelinApi public void close();
 
   /**
-   * chekpoint (versioning) for notebooks (optional)
+   * Versioning API
    */
-  @ZeppelinApi public void checkpoint(String noteId, String checkPointName) throws IOException;
+  /**
+   * chekpoint (set revision) for notebook.
+   * @param noteId Id of the Notebook
+   * @param checkpointMsg message description of the checkpoint
+   * @return Rev
+   * @throws IOException
+   */
+  @ZeppelinApi public Revision checkpoint(String noteId, String checkpointMsg) throws IOException;
+
+  /**
+   * Get particular revision of the Notebook.
+   * 
+   * @param noteId Id of the Notebook
+   * @param rev revision of the Notebook
+   * @return a Notebook
+   * @throws IOException
+   */
+  @ZeppelinApi public Note get(String noteId, Revision rev) throws IOException;
+
+  /**
+   * List of revisions of the given Notebook.
+   * 
+   * @param noteId id of the Notebook
+   * @return list of revisions
+   */
+  @ZeppelinApi public List<Revision> revisionHistory(String noteId);
+
+  /**
+   * Represents the 'Revision' a point in life of the notebook
+   */
+  static class Revision {
+    public Revision(String revId, String message, int time) {
+      this.revId = revId;
+      this.message = message;
+      this.time = time;
+    }
+    public String revId;
+    public String message;
+    public int time;
+  }
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/ff197d06/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/NotebookRepoSync.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/NotebookRepoSync.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/NotebookRepoSync.java
index 88399d0..389c6fd 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/NotebookRepoSync.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/NotebookRepoSync.java
@@ -354,13 +354,16 @@ public class NotebookRepoSync implements NotebookRepo {
 
   //checkpoint to all available storages
   @Override
-  public void checkpoint(String noteId, String checkPointName) throws IOException {
+  public Revision checkpoint(String noteId, String checkpointMsg) throws IOException {
     int repoCount = getRepoCount();
+    int repoBound = Math.min(repoCount, getMaxRepoNum());
     int errorCount = 0;
     String errorMessage = "";
-    for (int i = 0; i < Math.min(repoCount, getMaxRepoNum()); i++) {
+    List<Revision> allRepoCheckpoints = new ArrayList<Revision>();
+    Revision rev = null;
+    for (int i = 0; i < repoBound; i++) {
       try {
-        getRepo(i).checkpoint(noteId, checkPointName);
+        allRepoCheckpoints.add(getRepo(i).checkpoint(noteId, checkpointMsg));
       } catch (IOException e) {
         LOG.warn("Couldn't checkpoint in {} storage with index {} for note {}", 
           getRepo(i).getClass().toString(), i, noteId);
@@ -370,9 +373,28 @@ public class NotebookRepoSync implements NotebookRepo {
       }
     }
     // throw exception if failed to commit for all initialized repos
-    if (errorCount == Math.min(repoCount, getMaxRepoNum())) {
+    if (errorCount == repoBound) {
       throw new IOException(errorMessage);
     }
+    if (allRepoCheckpoints.size() > 0) {
+      rev = allRepoCheckpoints.get(0);
+      // if failed to checkpoint on first storage, then return result on second
+      if (allRepoCheckpoints.size() > 1 && rev == null) {
+        rev = allRepoCheckpoints.get(1);
+      }
+    }
+    return rev;
+  }
 
+  @Override
+  public Note get(String noteId, Revision rev) throws IOException {
+    // Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<Revision> revisionHistory(String noteId) {
+    // Auto-generated method stub
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/ff197d06/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/NotebookRepoVersioned.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/NotebookRepoVersioned.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/NotebookRepoVersioned.java
deleted file mode 100644
index 4615afd..0000000
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/NotebookRepoVersioned.java
+++ /dev/null
@@ -1,60 +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.repo;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.zeppelin.notebook.Note;
-
-/**
- * Notebook repository w/ versions
- */
-public interface NotebookRepoVersioned extends NotebookRepo {
-
-  /**
-   * Get particular revision of the Notebooks
-   *
-   * @param noteId Id of the Notebook
-   * @param rev revision of the Notebook
-   * @return a Notebook
-   * @throws IOException
-   */
-  public Note get(String noteId, String rev) throws IOException;
-
-  /**
-   * List of revisions of the given Notebook
-   *
-   * @param noteId id of the Notebook
-   * @return list of revisions
-   */
-  public List<Rev> history(String noteId);
-
-  /**
-   * Represents the 'Revision' a point in life of the notebook
-   */
-  static class Rev {
-    public Rev(String name, int time) {
-      this.name = name;
-      this.time = time;
-    }
-    String name;
-    int time;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/ff197d06/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/S3NotebookRepo.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/S3NotebookRepo.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/S3NotebookRepo.java
index 451d483..c760667 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/S3NotebookRepo.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/S3NotebookRepo.java
@@ -249,8 +249,20 @@ public class S3NotebookRepo implements NotebookRepo {
   }
 
   @Override
-  public void checkpoint(String noteId, String checkPointName) throws IOException {
-    // no-op
-    LOG.info("Checkpoint feature isn't supported in {}", this.getClass().toString());
+  public Revision checkpoint(String noteId, String checkpointMsg) throws IOException {
+    // Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public Note get(String noteId, Revision rev) throws IOException {
+    // Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<Revision> revisionHistory(String noteId) {
+    // Auto-generated method stub
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/ff197d06/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/VFSNotebookRepo.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/VFSNotebookRepo.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/VFSNotebookRepo.java
index 8c0cf2c..06cf25f 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/VFSNotebookRepo.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/VFSNotebookRepo.java
@@ -253,9 +253,21 @@ public class VFSNotebookRepo implements NotebookRepo {
   }
 
   @Override
-  public void checkpoint(String noteId, String checkPointName) throws IOException {
-    // no-op
-    logger.info("Checkpoint feature isn't supported in {}", this.getClass().toString());
+  public Revision checkpoint(String noteId, String checkpointMsg) throws IOException {
+    // Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public Note get(String noteId, Revision rev) throws IOException {
+    // Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<Revision> revisionHistory(String noteId) {
+    // Auto-generated method stub
+    return null;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/ff197d06/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/zeppelinhub/ZeppelinHubRepo.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/zeppelinhub/ZeppelinHubRepo.java b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/zeppelinhub/ZeppelinHubRepo.java
index 3d344b4..2c249c9 100644
--- a/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/zeppelinhub/ZeppelinHubRepo.java
+++ b/zeppelin-zengine/src/main/java/org/apache/zeppelin/notebook/repo/zeppelinhub/ZeppelinHubRepo.java
@@ -188,8 +188,21 @@ public class ZeppelinHubRepo implements NotebookRepo {
   }
 
   @Override
-  public void checkpoint(String noteId, String checkPointName) throws IOException {
-    
+  public Revision checkpoint(String noteId, String checkpointMsg) throws IOException {
+    // Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public Note get(String noteId, Revision rev) throws IOException {
+    // Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<Revision> revisionHistory(String noteId) {
+    // Auto-generated method stub
+    return null;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/ff197d06/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/GitNotebookRepoTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/GitNotebookRepoTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/GitNotebookRepoTest.java
index 879b1ad..fe020cb 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/GitNotebookRepoTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/GitNotebookRepoTest.java
@@ -32,7 +32,7 @@ import org.apache.zeppelin.interpreter.mock.MockInterpreter2;
 import org.apache.zeppelin.notebook.Note;
 import org.apache.zeppelin.notebook.NoteInfo;
 import org.apache.zeppelin.notebook.Paragraph;
-import org.apache.zeppelin.notebook.repo.NotebookRepoVersioned.Rev;
+import org.apache.zeppelin.notebook.repo.NotebookRepo.Revision;
 import org.eclipse.jgit.api.Git;
 import org.eclipse.jgit.api.errors.GitAPIException;
 import org.eclipse.jgit.diff.DiffEntry;
@@ -111,7 +111,7 @@ public class GitNotebookRepoTest {
     assertThat(notebookRepo.list()).isNotEmpty();
 
     //when
-    List<Rev> testNotebookHistory = notebookRepo.history(TEST_NOTE_ID);
+    List<Revision> testNotebookHistory = notebookRepo.revisionHistory(TEST_NOTE_ID);
 
     //then
     //no initial commit, empty history
@@ -124,11 +124,11 @@ public class GitNotebookRepoTest {
     notebookRepo = new GitNotebookRepo(conf);
     assertThat(notebookRepo.list()).isNotEmpty();
     assertThat(containsNote(notebookRepo.list(), TEST_NOTE_ID)).isTrue();
-    assertThat(notebookRepo.history(TEST_NOTE_ID)).isEmpty();
+    assertThat(notebookRepo.revisionHistory(TEST_NOTE_ID)).isEmpty();
 
     notebookRepo.checkpoint(TEST_NOTE_ID, "first commit");
-    List<Rev> notebookHistoryBefore = notebookRepo.history(TEST_NOTE_ID);
-    assertThat(notebookRepo.history(TEST_NOTE_ID)).isNotEmpty();
+    List<Revision> notebookHistoryBefore = notebookRepo.revisionHistory(TEST_NOTE_ID);
+    assertThat(notebookRepo.revisionHistory(TEST_NOTE_ID)).isNotEmpty();
     int initialCount = notebookHistoryBefore.size();
     
     // add changes to note
@@ -144,7 +144,7 @@ public class GitNotebookRepoTest {
     notebookRepo.checkpoint(TEST_NOTE_ID, "second commit");
     
     // see if commit is added
-    List<Rev> notebookHistoryAfter = notebookRepo.history(TEST_NOTE_ID);
+    List<Revision> notebookHistoryAfter = notebookRepo.revisionHistory(TEST_NOTE_ID);
     assertThat(notebookHistoryAfter.size()).isEqualTo(initialCount + 1);
   }
   

http://git-wip-us.apache.org/repos/asf/zeppelin/blob/ff197d06/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/NotebookRepoSyncTest.java
----------------------------------------------------------------------
diff --git a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/NotebookRepoSyncTest.java b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/NotebookRepoSyncTest.java
index 1699d68..138977e 100644
--- a/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/NotebookRepoSyncTest.java
+++ b/zeppelin-zengine/src/test/java/org/apache/zeppelin/notebook/repo/NotebookRepoSyncTest.java
@@ -244,7 +244,7 @@ public class NotebookRepoSyncTest implements JobListenerFactory {
     String noteId = vRepoSync.list().get(0).getId();
     // first checkpoint
     vRepoSync.checkpoint(noteId, "checkpoint message");
-    int vCount = gitRepo.history(noteId).size();
+    int vCount = gitRepo.revisionHistory(noteId).size();
     assertThat(vCount).isEqualTo(1);
     
     Paragraph p = note.addParagraph();
@@ -256,7 +256,7 @@ public class NotebookRepoSyncTest implements JobListenerFactory {
     // save and checkpoint again
     vRepoSync.save(note);
     vRepoSync.checkpoint(noteId, "checkpoint message 2");
-    assertThat(gitRepo.history(noteId).size()).isEqualTo(vCount + 1);
+    assertThat(gitRepo.revisionHistory(noteId).size()).isEqualTo(vCount + 1);
   }
   
   static void delete(File file){