You are viewing a plain text version of this content. The canonical link for it is here.
Posted to oak-commits@jackrabbit.apache.org by mr...@apache.org on 2017/02/23 09:04:03 UTC

svn commit: r1784093 - in /jackrabbit/oak/trunk: oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/ oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/ oak-run/src/main/java/org/apache/jackrabbit/oak/plugins/document/

Author: mreutegg
Date: Thu Feb 23 09:04:03 2017
New Revision: 1784093

URL: http://svn.apache.org/viewvc?rev=1784093&view=rev
Log:
OAK-5761: Move commit value resolution to DocumentNodeStore

Remove NodeDocument.isCommitted() and NodeDocument.getCommitRevision()

Modified:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/Collision.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/Commit.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/LastRevRecoveryAgent.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/NodeDocument.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/SplitOperations.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/CollisionTest.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/CollisionWithSplitTest.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStoreTest.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentSplitTest.java
    jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStoreHelper.java

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/Collision.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/Collision.java?rev=1784093&r1=1784092&r2=1784093&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/Collision.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/Collision.java Thu Feb 23 09:04:03 2017
@@ -27,6 +27,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.jackrabbit.oak.plugins.document.util.Utils.isCommitted;
 import static org.apache.jackrabbit.oak.plugins.document.util.Utils.isPropertyName;
 
 /**
@@ -51,15 +52,18 @@ class Collision {
     private final Revision theirRev;
     private final UpdateOp ourOp;
     private final Revision ourRev;
+    private final RevisionContext context;
 
     Collision(@Nonnull NodeDocument document,
               @Nonnull Revision theirRev,
               @Nonnull UpdateOp ourOp,
-              @Nonnull Revision ourRev) {
+              @Nonnull Revision ourRev,
+              @Nonnull RevisionContext context) {
         this.document = checkNotNull(document);
         this.theirRev = checkNotNull(theirRev);
         this.ourOp = checkNotNull(ourOp);
         this.ourRev = checkNotNull(ourRev);
+        this.context = checkNotNull(context);
     }
 
     /**
@@ -76,14 +80,14 @@ class Collision {
     @Nonnull
     Revision mark(DocumentStore store) throws DocumentStoreException {
         // first try to mark their revision
-        if (markCommitRoot(document, theirRev, ourRev, store)) {
+        if (markCommitRoot(document, theirRev, ourRev, store, context)) {
             return theirRev;
         }
         // their commit wins, we have to mark ourRev
         NodeDocument newDoc = Collection.NODES.newDocument(store);
         document.deepCopy(newDoc);
         UpdateUtils.applyChanges(newDoc, ourOp);
-        if (!markCommitRoot(newDoc, ourRev, theirRev, store)) {
+        if (!markCommitRoot(newDoc, ourRev, theirRev, store, context)) {
             throw new IllegalStateException("Unable to annotate our revision "
                     + "with collision marker. Our revision: " + ourRev
                     + ", document:\n" + newDoc.format());
@@ -132,18 +136,20 @@ class Collision {
      *            marker.
      * @param other the revision which detected the collision.
      * @param store the document store.
+     * @param context the revision context.
      * @return <code>true</code> if the commit for the given revision was marked
      *         successfully; <code>false</code> otherwise.
      */
     private static boolean markCommitRoot(@Nonnull NodeDocument document,
                                           @Nonnull Revision revision,
                                           @Nonnull Revision other,
-                                          @Nonnull DocumentStore store) {
+                                          @Nonnull DocumentStore store,
+                                          @Nonnull RevisionContext context) {
         String p = document.getPath();
         String commitRootPath;
         // first check if we can mark the commit with the given revision
         if (document.containsRevision(revision)) {
-            if (document.isCommitted(revision)) {
+            if (isCommitted(context.getCommitValue(revision, document))) {
                 // already committed
                 return false;
             }
@@ -161,7 +167,7 @@ class Collision {
         UpdateOp op = new UpdateOp(Utils.getIdFromPath(commitRootPath), false);
         NodeDocument commitRoot = store.find(Collection.NODES, op.getId());
         // check commit status of revision
-        if (commitRoot.isCommitted(revision)) {
+        if (isCommitted(context.getCommitValue(revision, commitRoot))) {
             return false;
         }
         // check if there is already a collision marker
@@ -188,7 +194,7 @@ class Collision {
         } else {
             // check again if revision is still not committed
             // See OAK-3882
-            if (commitRoot.isCommitted(revision)) {
+            if (isCommitted(context.getCommitValue(revision, commitRoot))) {
                 // meanwhile the change was committed and
                 // already moved to a previous document
                 // -> remove collision marker again

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/Commit.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/Commit.java?rev=1784093&r1=1784092&r2=1784093&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/Commit.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/Commit.java Thu Feb 23 09:04:03 2017
@@ -502,7 +502,7 @@ public class Commit {
                 // TODO: unify above conflict detection and isConflicting()
                 boolean allowConflictingDeleteChange = allowConcurrentAddRemove(before, op);
                 for (Revision r : collisions) {
-                    Collision c = new Collision(before, r, op, revision);
+                    Collision c = new Collision(before, r, op, revision, nodeStore);
                     if (c.isConflicting() && !allowConflictingDeleteChange) {
                         // mark collisions on commit root
                         if (c.mark(store).equals(revision)) {

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/LastRevRecoveryAgent.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/LastRevRecoveryAgent.java?rev=1784093&r1=1784092&r2=1784093&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/LastRevRecoveryAgent.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/LastRevRecoveryAgent.java Thu Feb 23 09:04:03 2017
@@ -25,6 +25,8 @@ import static java.util.Collections.sing
 import static org.apache.jackrabbit.oak.plugins.document.Collection.JOURNAL;
 import static org.apache.jackrabbit.oak.plugins.document.Collection.NODES;
 import static org.apache.jackrabbit.oak.plugins.document.util.Utils.PROPERTY_OR_DELETED;
+import static org.apache.jackrabbit.oak.plugins.document.util.Utils.isCommitted;
+import static org.apache.jackrabbit.oak.plugins.document.util.Utils.resolveCommitRevision;
 
 import java.util.Iterator;
 import java.util.Map;
@@ -399,8 +401,9 @@ public class LastRevRecoveryAgent {
             // collect committed changes of this cluster node
             for (Map.Entry<Revision, String> entry : filterKeys(valueMap, cp).entrySet()) {
                 Revision rev = entry.getKey();
-                if (doc.isCommitted(rev)) {
-                    lastModified = Utils.max(lastModified, doc.getCommitRevision(rev));
+                String cv = nodeStore.getCommitValue(rev, doc);
+                if (isCommitted(cv)) {
+                    lastModified = Utils.max(lastModified, resolveCommitRevision(rev, cv));
                     break;
                 }
             }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/NodeDocument.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/NodeDocument.java?rev=1784093&r1=1784092&r2=1784093&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/NodeDocument.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/NodeDocument.java Thu Feb 23 09:04:03 2017
@@ -528,55 +528,11 @@ public final class NodeDocument extends
     }
 
     /**
-     * Returns <code>true</code> if the given <code>revision</code> is marked
-     * committed.
-     *
-     * @param revision the revision.
-     * @return <code>true</code> if committed; <code>false</code> otherwise.
-     */
-    public boolean isCommitted(@Nonnull Revision revision) {
-        NodeDocument commitRootDoc = getCommitRoot(checkNotNull(revision));
-        if (commitRootDoc == null) {
-            return false;
-        }
-        String value = commitRootDoc.getLocalRevisions().get(revision);
-        if (value != null) {
-            return Utils.isCommitted(value);
-        }
-        // check previous docs
-        for (NodeDocument prev : commitRootDoc.getPreviousDocs(REVISIONS, revision)) {
-            if (prev.containsRevision(revision)) {
-                return prev.isCommitted(revision);
-            }
-        }
-        return false;
-    }
-
-    /**
-     * Returns the commit revision for the change with the given revision.
-     *
-     * @param revision the revision of a change.
-     * @return the commit revision of the change or {@code null} if the change
-     *          is not committed or unknown.
-     */
-    @CheckForNull
-    public Revision getCommitRevision(@Nonnull Revision revision) {
-        NodeDocument commitRoot = getCommitRoot(checkNotNull(revision));
-        if (commitRoot == null) {
-            return null;
-        }
-        String value = commitRoot.getCommitValue(revision);
-        if (Utils.isCommitted(value)) {
-            return Utils.resolveCommitRevision(revision, value);
-        }
-        return null;
-    }
-
-    /**
      * Returns <code>true</code> if this document contains an entry for the
      * given <code>revision</code> in the {@link #REVISIONS} map. Please note
      * that an entry in the {@link #REVISIONS} map does not necessarily mean
-     * the the revision is committed. Use {@link #isCommitted(Revision)} to get
+     * the the revision is committed.
+     * Use {@link RevisionContext#getCommitValue(Revision, NodeDocument)} to get
      * the commit state of a revision.
      *
      * @param revision the revision to check.
@@ -836,10 +792,10 @@ public final class NodeDocument extends
                     // 5) changeRev is not on a branch, 'r' is committed and
                     //    newer than baseRev -> newestRev
 
-                    NodeDocument commitRoot = getCommitRoot(r);
                     Revision commitRevision = null;
-                    if (commitRoot != null) {
-                        commitRevision = commitRoot.getCommitRevision(r);
+                    String cv = context.getCommitValue(r, this);
+                    if (Utils.isCommitted(cv)) {
+                        commitRevision = resolveCommitRevision(r, cv);
                     }
                     if (commitRevision != null // committed but not yet visible
                             && head.isRevisionNewer(commitRevision)) {
@@ -909,11 +865,11 @@ public final class NodeDocument extends
      * @return <code>true</code> if the revision is valid; <code>false</code>
      *         otherwise.
      */
-    boolean isValidRevision(@Nonnull RevisionContext context,
-                            @Nonnull Revision rev,
-                            @Nullable String commitValue,
-                            @Nonnull RevisionVector readRevision,
-                            @Nonnull Map<Revision, String> validRevisions) {
+    private boolean isValidRevision(@Nonnull RevisionContext context,
+                                    @Nonnull Revision rev,
+                                    @Nullable String commitValue,
+                                    @Nonnull RevisionVector readRevision,
+                                    @Nonnull Map<Revision, String> validRevisions) {
         if (validRevisions.containsKey(rev)) {
             return true;
         }
@@ -921,7 +877,7 @@ public final class NodeDocument extends
         if (doc == null) {
             return false;
         }
-        if (doc.isCommitted(context, rev, commitValue, readRevision)) {
+        if (doc.isVisible(context, rev, commitValue, readRevision)) {
             validRevisions.put(rev, commitValue);
             return true;
         }
@@ -973,7 +929,7 @@ public final class NodeDocument extends
             // check if there may be more recent values in a previous document
             if (value != null
                     && !getPreviousRanges().isEmpty()
-                    && !isMostRecentCommitted(local, value.revision)) {
+                    && !isMostRecentCommitted(local, value.revision, nodeStore)) {
                 // not reading the most recent value, we may need to
                 // consider previous documents as well
                 for (Revision prev : getPreviousRanges().keySet()) {
@@ -1720,6 +1676,14 @@ public final class NodeDocument extends
         return getLocalMap(STALE_PREV);
     }
 
+    /**
+     * Resolves the commit value for the change with the given revision on this
+     * document. If necessary, this method will lookup the commit value on the
+     * referenced commit root document.
+     *
+     * @param revision the revision of a change on this document.
+     * @return the commit value associated with the change.
+     */
     @CheckForNull
     String resolveCommitValue(Revision revision) {
         NodeDocument commitRoot = getCommitRoot(revision);
@@ -1942,11 +1906,13 @@ public final class NodeDocument extends
      *
      * @param valueMap the value map sorted most recent first.
      * @param revision a committed revision.
+     * @param context the revision context.
      * @return if {@code revision} is the most recent committed revision in the
      *          {@code valueMap}.
      */
     private boolean isMostRecentCommitted(SortedMap<Revision, String> valueMap,
-                                          Revision revision) {
+                                          Revision revision,
+                                          RevisionContext context) {
         if (valueMap.isEmpty()) {
             return true;
         }
@@ -1957,8 +1923,9 @@ public final class NodeDocument extends
         }
         // need to check commit status
         for (Revision r : valueMap.keySet()) {
-            Revision c = getCommitRevision(r);
-            if (c != null) {
+            String cv = context.getCommitValue(r, this);
+            if (Utils.isCommitted(cv)) {
+                Revision c = resolveCommitRevision(r, cv);
                 return c.compareRevisionTimeThenClusterId(revision) <= 0;
             }
         }
@@ -2046,21 +2013,23 @@ public final class NodeDocument extends
      * Returns <code>true</code> if the given revision
      * {@link Utils#isCommitted(String)} in the revisions map (including
      * revisions split off to previous documents) and is visible from the
-     * <code>readRevision</code>.
+     * <code>readRevision</code>. This includes branch commits if the read
+     * revision is on the same branch and is equal or newer than the revision
+     * to check.
      *
      * @param revision  the revision to check.
      * @param commitValue the commit value of the revision to check or
      *                    <code>null</code> if unknown.
      * @param readRevision the read revision.
-     * @return <code>true</code> if the revision is committed, otherwise
+     * @return <code>true</code> if the revision is visible, otherwise
      *         <code>false</code>.
      */
-    private boolean isCommitted(@Nonnull RevisionContext context,
-                                @Nonnull Revision revision,
-                                @Nullable String commitValue,
-                                @Nonnull RevisionVector readRevision) {
+    private boolean isVisible(@Nonnull RevisionContext context,
+                              @Nonnull Revision revision,
+                              @Nullable String commitValue,
+                              @Nonnull RevisionVector readRevision) {
         if (commitValue == null) {
-            commitValue = getCommitValue(revision);
+            commitValue = context.getCommitValue(revision, this);
         }
         if (commitValue == null) {
             return false;
@@ -2075,7 +2044,8 @@ public final class NodeDocument extends
                 return !readRevision.isRevisionNewer(revision);
             } else {
                 // on same merged branch?
-                if (commitValue.equals(getCommitValue(readRevision.getBranchRevision().asTrunkRevision()))) {
+                Revision tr = readRevision.getBranchRevision().asTrunkRevision();
+                if (commitValue.equals(context.getCommitValue(tr, this))) {
                     // compare unresolved revision
                     return !readRevision.isRevisionNewer(revision);
                 }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/SplitOperations.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/SplitOperations.java?rev=1784093&r1=1784092&r2=1784093&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/SplitOperations.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/SplitOperations.java Thu Feb 23 09:04:03 2017
@@ -59,6 +59,7 @@ import static org.apache.jackrabbit.oak.
 import static org.apache.jackrabbit.oak.plugins.document.NodeDocument.setPrevious;
 import static org.apache.jackrabbit.oak.plugins.document.util.Utils.PROPERTY_OR_DELETED;
 import static org.apache.jackrabbit.oak.plugins.document.util.Utils.getPreviousIdFor;
+import static org.apache.jackrabbit.oak.plugins.document.util.Utils.isCommitted;
 
 /**
  * Utility class to create document split operations.
@@ -254,7 +255,7 @@ class SplitOperations {
                     // only consider local changes
                     continue;
                 }
-                if (doc.isCommitted(entry.getKey())
+                if (isCommitted(context.getCommitValue(entry.getKey(), doc))
                         && !mostRecentRevs.contains(entry.getKey())) {
                     // this is a commit root for changes in other documents
                     revisions.put(entry.getKey(), entry.getValue());
@@ -277,7 +278,7 @@ class SplitOperations {
                     && !changes.contains(r)) {
                 // OAK-2528: _commitRoot entry without associated change
                 // consider all but most recent as garbage (OAK-3333, OAK-4050)
-                if (mostRecent && doc.isCommitted(r)) {
+                if (mostRecent && isCommitted(context.getCommitValue(r, doc))) {
                     mostRecent = false;
                 } else if (isGarbage(r)) {
                     addGarbage(r, COMMIT_ROOT);
@@ -438,7 +439,7 @@ class SplitOperations {
                     continue;
                 }
                 changes.add(rev);
-                if (doc.isCommitted(rev)) {
+                if (isCommitted(context.getCommitValue(rev, doc))) {
                     splitMap.put(rev, entry.getValue());
                 } else if (isGarbage(rev)) {
                     addGarbage(rev, property);

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/CollisionTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/CollisionTest.java?rev=1784093&r1=1784092&r2=1784093&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/CollisionTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/CollisionTest.java Thu Feb 23 09:04:03 2017
@@ -99,12 +99,12 @@ public class CollisionTest {
         Revision c = ns.newRevision();
         UpdateOp op = new UpdateOp(id, true);
         NodeDocument.setDeleted(op, c, false);
-        Collision col = new Collision(doc, r1, op, c);
+        Collision col = new Collision(doc, r1, op, c, ns);
         assertTrue(col.isConflicting());
         // concurrent change
         op = new UpdateOp(id, false);
         op.setMapEntry("p", c, "b");
-        col = new Collision(doc, r1, op, c);
+        col = new Collision(doc, r1, op, c, ns);
         assertTrue(col.isConflicting());
 
         b = ns.getRoot().builder();
@@ -119,17 +119,17 @@ public class CollisionTest {
         op = new UpdateOp(id, false);
         op.setDelete(true);
         NodeDocument.setDeleted(op, c, true);
-        col = new Collision(doc, r2, op, c);
+        col = new Collision(doc, r2, op, c, ns);
         assertTrue(col.isConflicting());
         // concurrent conflicting property set
         op = new UpdateOp(id, false);
         op.setMapEntry("p", c, "c");
-        col = new Collision(doc, r2, op, c);
+        col = new Collision(doc, r2, op, c, ns);
         assertTrue(col.isConflicting());
         // concurrent non-conflicting property set
         op = new UpdateOp(id, false);
         op.setMapEntry("q", c, "a");
-        col = new Collision(doc, r2, op, c);
+        col = new Collision(doc, r2, op, c, ns);
         assertFalse(col.isConflicting());
 
         b = ns.getRoot().builder();
@@ -144,12 +144,12 @@ public class CollisionTest {
         op = new UpdateOp(id, false);
         op.setDelete(true);
         NodeDocument.setDeleted(op, c, true);
-        col = new Collision(doc, r3, op, c);
+        col = new Collision(doc, r3, op, c, ns);
         assertTrue(col.isConflicting());
         // concurrent conflicting property set
         op = new UpdateOp(id, false);
         op.setMapEntry("p", c, "d");
-        col = new Collision(doc, r3, op, c);
+        col = new Collision(doc, r3, op, c, ns);
         assertTrue(col.isConflicting());
     }
 

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/CollisionWithSplitTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/CollisionWithSplitTest.java?rev=1784093&r1=1784092&r2=1784093&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/CollisionWithSplitTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/CollisionWithSplitTest.java Thu Feb 23 09:04:03 2017
@@ -115,7 +115,7 @@ public class CollisionWithSplitTest exte
         // committed revision on ns2
         doc = ns2.getDocumentStore().find(NODES, id);
         assertTrue(doc.getLocalCommitRoot().containsKey(conflictRev));
-        Collision c = new Collision(doc, conflictRev, op, ourRev);
+        Collision c = new Collision(doc, conflictRev, op, ourRev, ns2);
         assertEquals("Collision must match our revision (" + ourRev + "). " +
                 "The conflict revision " + conflictRev + " is already committed.",
                 ourRev, c.mark(ns2.getDocumentStore()));

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStoreTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStoreTest.java?rev=1784093&r1=1784092&r2=1784093&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStoreTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStoreTest.java Thu Feb 23 09:04:03 2017
@@ -24,6 +24,7 @@ import static org.apache.jackrabbit.oak.
 import static org.apache.jackrabbit.oak.plugins.document.NodeDocument.NUM_REVS_THRESHOLD;
 import static org.apache.jackrabbit.oak.plugins.document.NodeDocument.PREV_SPLIT_FACTOR;
 import static org.apache.jackrabbit.oak.plugins.document.NodeDocument.getModifiedInSecs;
+import static org.apache.jackrabbit.oak.plugins.document.util.Utils.isCommitted;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
@@ -1167,7 +1168,7 @@ public class DocumentNodeStoreTest {
         String parentPath = "/:hidden/parent";
         NodeDocument parentDoc = docStore.find(Collection.NODES, Utils.getIdFromPath(parentPath));
         assertFalse("parent node of unseen children must not get deleted",
-                isDocDeleted(parentDoc));
+                isDocDeleted(parentDoc, store1));
 
         //Test 2 - parent shouldn't be removable if order of operation is:
         //# N1 and N2 know about /:hidden
@@ -1185,7 +1186,7 @@ public class DocumentNodeStoreTest {
 
         parentDoc = docStore.find(Collection.NODES, Utils.getIdFromPath(parentPath));
         assertFalse("parent node of unseen children must not get deleted",
-                isDocDeleted(parentDoc));
+                isDocDeleted(parentDoc, store1));
 
         store1.runBackgroundOperations();
         store2.runBackgroundOperations();
@@ -1219,7 +1220,7 @@ public class DocumentNodeStoreTest {
         parentPath = "/:hidden/parent1";
         parentDoc = docStore.find(Collection.NODES, Utils.getIdFromPath(parentPath));
         assertFalse("parent node of unseen children must not get deleted",
-                isDocDeleted(parentDoc));
+                isDocDeleted(parentDoc, store1));
 
         //Test 4 - parent shouldn't be removable if order of operation is:
         //# N1 and N2 know about /:hidden/parent1
@@ -1236,7 +1237,7 @@ public class DocumentNodeStoreTest {
 
         parentDoc = docStore.find(Collection.NODES, Utils.getIdFromPath(parentPath));
         assertFalse("parent node of unseen children must not get deleted",
-                isDocDeleted(parentDoc));
+                isDocDeleted(parentDoc, store1));
     }
 
     @Test
@@ -3003,14 +3004,14 @@ public class DocumentNodeStoreTest {
      * @param doc the document to be tested
      * @return latest committed value of _deleted map
      */
-    private boolean isDocDeleted(NodeDocument doc) {
+    private boolean isDocDeleted(NodeDocument doc, RevisionContext context) {
         boolean latestDeleted = false;
         SortedMap<Revision, String> localDeleted =
                 Maps.newTreeMap(StableRevisionComparator.REVERSE);
         localDeleted.putAll(doc.getLocalDeleted());
 
         for (Map.Entry<Revision, String> entry : localDeleted.entrySet()) {
-            if (doc.isCommitted(entry.getKey())) {
+            if (isCommitted(context.getCommitValue(entry.getKey(), doc))) {
                 latestDeleted = Boolean.parseBoolean(entry.getValue());
                 break;
             }

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentSplitTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentSplitTest.java?rev=1784093&r1=1784092&r2=1784093&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentSplitTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentSplitTest.java Thu Feb 23 09:04:03 2017
@@ -60,6 +60,7 @@ import static org.apache.jackrabbit.oak.
 import static org.apache.jackrabbit.oak.plugins.document.TestUtils.NO_BINARY;
 import static org.apache.jackrabbit.oak.plugins.document.UpdateOp.Operation.Type.REMOVE_MAP_ENTRY;
 import static org.apache.jackrabbit.oak.plugins.document.UpdateOp.Operation.Type.SET_MAP_ENTRY;
+import static org.apache.jackrabbit.oak.plugins.document.util.Utils.isCommitted;
 import static org.apache.jackrabbit.oak.plugins.memory.BinaryPropertyState.binaryProperty;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -96,7 +97,7 @@ public class DocumentSplitTest extends B
         assertEquals(1, revs.size());
         for (Revision rev : revisions) {
             assertTrue(doc.containsRevision(rev));
-            assertTrue(doc.isCommitted(rev));
+            assertTrue(isCommitted(ns.getCommitValue(rev, doc)));
         }
         // check if document is still there
         assertNotNull(ns.getNode("/", RevisionVector.fromString(head)));
@@ -137,7 +138,7 @@ public class DocumentSplitTest extends B
         for (Revision rev : revisions) {
             assertTrue("document should contain revision (or have revision in commit root path):" + rev, doc.containsRevision(rev)
                     || doc.getCommitRootPath(rev) != null);
-            assertTrue(doc.isCommitted(rev));
+            assertTrue(isCommitted(ns.getCommitValue(rev, doc)));
         }
         DocumentNodeState node = ns.getNode("/foo", RevisionVector.fromString(head));
         // check status of node
@@ -151,6 +152,7 @@ public class DocumentSplitTest extends B
     @Test
     public void splitCommitRoot() throws Exception {
         DocumentStore store = mk.getDocumentStore();
+        DocumentNodeStore ns = mk.getNodeStore();
         mk.commit("/", "+\"foo\":{}+\"bar\":{}", null, null);
         NodeDocument doc = store.find(NODES, Utils.getIdFromPath("/foo"));
         assertNotNull(doc);
@@ -170,13 +172,14 @@ public class DocumentSplitTest extends B
         // the second _commitRoot entry for the most recent prop change
         assertEquals(2, commits.size());
         for (Revision rev : commitRoots) {
-            assertTrue(doc.isCommitted(rev));
+            assertTrue(isCommitted(ns.getCommitValue(rev, doc)));
         }
     }
 
     @Test
     public void splitPropertyRevisions() throws Exception {
         DocumentStore store = mk.getDocumentStore();
+        DocumentNodeStore ns = mk.getNodeStore();
         mk.commit("/", "+\"foo\":{}", null, null);
         NodeDocument doc = store.find(NODES, Utils.getIdFromPath("/foo"));
         assertNotNull(doc);
@@ -193,7 +196,7 @@ public class DocumentSplitTest extends B
         // one remaining in the local revisions map
         assertEquals(1, localRevs.size());
         for (Revision rev : revisions) {
-            assertTrue(doc.isCommitted(rev));
+            assertTrue(isCommitted(ns.getCommitValue(rev, doc)));
         }
         // all revisions in the prop map
         Map<Revision, String> valueMap = doc.getValueMap("prop");

Modified: jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStoreHelper.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStoreHelper.java?rev=1784093&r1=1784092&r2=1784093&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStoreHelper.java (original)
+++ jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStoreHelper.java Thu Feb 23 09:04:03 2017
@@ -48,6 +48,7 @@ import org.apache.jackrabbit.oak.plugins
 import org.apache.jackrabbit.oak.plugins.document.util.Utils;
 
 import static org.apache.jackrabbit.oak.plugins.document.mongo.MongoDocumentStoreHelper.convertFromDBObject;
+import static org.apache.jackrabbit.oak.plugins.document.util.Utils.isCommitted;
 
 /**
  * Helper class to access package private method of DocumentNodeStore and other
@@ -124,7 +125,7 @@ public class DocumentNodeStoreHelper {
                 blobSize += size(blobs);
                 if (foundValid) {
                     garbageSize += size(blobs);
-                } else if (doc.isCommitted(entry.getKey())) {
+                } else if (isCommitted(ns.getCommitValue(entry.getKey(), doc))) {
                     foundValid = true;
                 } else {
                     garbageSize += size(blobs);