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 2016/07/14 07:39:02 UTC

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

Author: mreutegg
Date: Thu Jul 14 07:39:01 2016
New Revision: 1752596

URL: http://svn.apache.org/viewvc?rev=1752596&view=rev
Log:
OAK-4528: diff calculation in DocumentNodeStore should try to re-use journal info on diff cache miss

Added:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalDiffLoader.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JsopNodeStateDiffer.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/JournalDiffLoaderTest.java   (with props)
Modified:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStore.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalEntry.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalGarbageCollector.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/UnmergedBranches.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/JournalEntryTest.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/JournalGCTest.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/SimpleTest.java

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStore.java?rev=1752596&r1=1752595&r2=1752596&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStore.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStore.java Thu Jul 14 07:39:01 2016
@@ -160,6 +160,12 @@ public final class DocumentNodeStore
             Long.getLong("oak.recoveryWaitTimeoutMS", 60000);
 
     /**
+     * Feature flag to disable the journal diff mechanism. See OAK-4528.
+     */
+    private boolean disableJournalDiff =
+            Boolean.getBoolean("oak.disableJournalDiff");
+
+    /**
      * The document store (might be used by multiple node stores).
      */
     protected final DocumentStore store;
@@ -825,6 +831,16 @@ public final class DocumentNodeStore
         return diffCache.getStats();
     }
 
+    /**
+     * Returns the journal entry that will be stored in the journal with the
+     * next background updated.
+     *
+     * @return the current journal entry.
+     */
+    JournalEntry getCurrentJournalEntry() {
+        return changes;
+    }
+
     void invalidateDocChildrenCache() {
         docChildrenCache.invalidateAll();
     }
@@ -1584,22 +1600,22 @@ public final class DocumentNodeStore
      */
     @Override
     public boolean compare(@Nonnull final AbstractDocumentNodeState node,
-                    @Nonnull final AbstractDocumentNodeState base,
-                    @Nonnull NodeStateDiff diff) {
+                           @Nonnull final AbstractDocumentNodeState base,
+                           @Nonnull NodeStateDiff diff) {
         if (!AbstractNodeState.comparePropertiesAgainstBaseState(node, base, diff)) {
             return false;
         }
         if (node.hasNoChildren() && base.hasNoChildren()) {
             return true;
         }
-        return dispatch(diffCache.getChanges(base.getRootRevision(),
+        return new JsopNodeStateDiffer(diffCache.getChanges(base.getRootRevision(),
                 node.getRootRevision(), node.getPath(),
                 new DiffCache.Loader() {
                     @Override
                     public String call() {
                         return diffImpl(base, node);
                     }
-                }), node, base, diff);
+                })).withoutPropertyChanges().compare(node, base, diff);
     }
 
     /**
@@ -2273,47 +2289,6 @@ public final class DocumentNodeStore
         }
     }
 
-    private boolean dispatch(@Nonnull final String jsonDiff,
-                             @Nonnull final AbstractDocumentNodeState node,
-                             @Nonnull final AbstractDocumentNodeState base,
-                             @Nonnull final NodeStateDiff diff) {
-        return DiffCache.parseJsopDiff(jsonDiff, new DiffCache.Diff() {
-            @Override
-            public boolean childNodeAdded(String name) {
-                return diff.childNodeAdded(name,
-                        node.getChildNode(name));
-            }
-
-            @Override
-            public boolean childNodeChanged(String name) {
-                boolean continueComparison = true;
-                NodeState baseChild = base.getChildNode(name);
-                NodeState nodeChild = node.getChildNode(name);
-                if (baseChild.exists()) {
-                    if (nodeChild.exists()) {
-                        continueComparison = diff.childNodeChanged(name,
-                                baseChild, nodeChild);
-                    } else {
-                        continueComparison = diff.childNodeDeleted(name,
-                                baseChild);
-                    }
-                } else {
-                    if (nodeChild.exists()) {
-                        continueComparison = diff.childNodeAdded(name,
-                                nodeChild);
-                    }
-                }
-                return continueComparison;
-            }
-
-            @Override
-            public boolean childNodeDeleted(String name) {
-                return diff.childNodeDeleted(name,
-                        base.getChildNode(name));
-            }
-        });
-    }
-
     /**
      * Search for presence of child node as denoted by path in the children cache of parent
      *
@@ -2356,44 +2331,54 @@ public final class DocumentNodeStore
 
     private String diffImpl(AbstractDocumentNodeState from, AbstractDocumentNodeState to)
             throws DocumentStoreException {
-        JsopWriter w = new JsopStream();
-        // TODO this does not work well for large child node lists
-        // use a document store index instead
         int max = MANY_CHILDREN_THRESHOLD;
 
         final boolean debug = LOG.isDebugEnabled();
         final long start = debug ? now() : 0;
+        long getChildrenDoneIn = start;
 
-        DocumentNodeState.Children fromChildren, toChildren;
-        fromChildren = getChildren(from, null, max);
-        toChildren = getChildren(to, null, max);
-
-        final long getChildrenDoneIn = debug ? now() : 0;
-
+        String diff;
         String diffAlgo;
         RevisionVector fromRev = from.getLastRevision();
         RevisionVector toRev = to.getLastRevision();
-        if (!fromChildren.hasMore && !toChildren.hasMore) {
-            diffAlgo = "diffFewChildren";
-            diffFewChildren(w, from.getPath(), fromChildren,
-                    fromRev, toChildren, toRev);
+        long minTimestamp = Utils.getMinTimestampForDiff(
+                fromRev, toRev, getMinExternalRevisions());
+
+        // use journal if possible
+        Revision tailRev = journalGarbageCollector.getTailRevision();
+        if (!disableJournalDiff
+                && tailRev.getTimestamp() < minTimestamp) {
+            diffAlgo = "diffJournalChildren";
+            diff = new JournalDiffLoader(from, to, this).call();
         } else {
-            if (FAST_DIFF) {
-                diffAlgo = "diffManyChildren";
-                fromRev = from.getRootRevision();
-                toRev = to.getRootRevision();
-                diffManyChildren(w, from.getPath(), fromRev, toRev);
-            } else {
-                diffAlgo = "diffAllChildren";
-                max = Integer.MAX_VALUE;
-                fromChildren = getChildren(from, null, max);
-                toChildren = getChildren(to, null, max);
+            DocumentNodeState.Children fromChildren, toChildren;
+            fromChildren = getChildren(from, null, max);
+            toChildren = getChildren(to, null, max);
+            getChildrenDoneIn = debug ? now() : 0;
+
+            JsopWriter w = new JsopStream();
+            if (!fromChildren.hasMore && !toChildren.hasMore) {
+                diffAlgo = "diffFewChildren";
                 diffFewChildren(w, from.getPath(), fromChildren,
                         fromRev, toChildren, toRev);
+            } else {
+                if (FAST_DIFF) {
+                    diffAlgo = "diffManyChildren";
+                    fromRev = from.getRootRevision();
+                    toRev = to.getRootRevision();
+                    diffManyChildren(w, from.getPath(), fromRev, toRev);
+                } else {
+                    diffAlgo = "diffAllChildren";
+                    max = Integer.MAX_VALUE;
+                    fromChildren = getChildren(from, null, max);
+                    toChildren = getChildren(to, null, max);
+                    diffFewChildren(w, from.getPath(), fromChildren,
+                            fromRev, toChildren, toRev);
+                }
             }
+            diff = w.toString();
         }
 
-        String diff = w.toString();
         if (debug) {
             long end = now();
             LOG.debug("Diff performed via '{}' at [{}] between revisions [{}] => [{}] took {} ms ({} ms), diff '{}', external '{}",

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalDiffLoader.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalDiffLoader.java?rev=1752596&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalDiffLoader.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalDiffLoader.java Thu Jul 14 07:39:01 2016
@@ -0,0 +1,227 @@
+/*
+ * 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.jackrabbit.oak.plugins.document;
+
+import java.io.IOException;
+import java.util.Map;
+
+import javax.annotation.CheckForNull;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import org.apache.jackrabbit.oak.cache.CacheStats;
+import org.apache.jackrabbit.oak.commons.sort.StringSort;
+import org.apache.jackrabbit.oak.plugins.document.util.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.jackrabbit.oak.plugins.document.JournalEntry.asId;
+import static org.apache.jackrabbit.oak.plugins.document.JournalEntry.fillExternalChanges;
+
+/**
+ * A DiffCache loader reading from journal entries.
+ */
+class JournalDiffLoader implements DiffCache.Loader {
+
+    private static final Logger LOG = LoggerFactory.getLogger(JournalDiffLoader.class);
+
+    private final AbstractDocumentNodeState base;
+
+    private final AbstractDocumentNodeState node;
+
+    private final DocumentNodeStore ns;
+
+    JournalDiffLoader(@Nonnull AbstractDocumentNodeState base,
+                      @Nonnull AbstractDocumentNodeState node,
+                      @Nonnull DocumentNodeStore ns) {
+        this.base = base;
+        this.node = node;
+        this.ns = ns;
+    }
+
+    @Override
+    public String call() {
+        RevisionVector afterRev = node.getRootRevision();
+        RevisionVector beforeRev = base.getRootRevision();
+
+        JournalEntry localPending = ns.getCurrentJournalEntry();
+        DocumentStore store = ns.getDocumentStore();
+        NodeDocument root = Utils.getRootDocument(store);
+        Map<Integer, Revision> lastRevs = root.getLastRev();
+        int clusterId = ns.getClusterId();
+        Revision localLastRev = lastRevs.get(clusterId);
+        if (localLastRev == null) {
+            throw new IllegalStateException("Root document does not have a " +
+                    "lastRev entry for local clusterId " + clusterId);
+        }
+
+        StringSort changes = JournalEntry.newSorter();
+        try {
+            readTrunkChanges(beforeRev, afterRev, localPending, localLastRev, changes);
+
+            readBranchChanges(beforeRev, changes);
+            readBranchChanges(afterRev, changes);
+
+            changes.sort();
+            DiffCache df = ns.getDiffCache();
+            WrappedDiffCache wrappedCache = new WrappedDiffCache(node.getPath(), df);
+            JournalEntry.applyTo(changes, wrappedCache, beforeRev, afterRev);
+
+            return wrappedCache.changes;
+        } catch (IOException e) {
+            throw DocumentStoreException.convert(e);
+        } finally {
+            Utils.closeIfCloseable(changes);
+        }
+    }
+
+    private void readBranchChanges(RevisionVector rv,
+                                   StringSort changes) throws IOException {
+        if (!rv.isBranch() || ns.isDisableBranches()) {
+            return;
+        }
+        Branch b = ns.getBranches().getBranch(rv);
+        if (b == null) {
+            if (!ns.getBranches().isBranchBase(rv)) {
+                missingBranch(rv);
+            }
+            return;
+        }
+        DocumentStore store = ns.getDocumentStore();
+        for (Revision br : b.getCommits()) {
+            Branch.BranchCommit bc = b.getCommit(br);
+            if (!bc.isRebase()) {
+                JournalEntry entry = store.find(Collection.JOURNAL, asId(br));
+                if (entry != null) {
+                    entry.addTo(changes);
+                } else {
+                    LOG.warn("Missing journal entry for {}", asId(br));
+                }
+            }
+        }
+    }
+
+    private void readTrunkChanges(RevisionVector beforeRev,
+                                  RevisionVector afterRev,
+                                  JournalEntry localPending,
+                                  Revision localLastRev,
+                                  StringSort changes) throws IOException {
+        if (ns.isDisableBranches()) {
+            beforeRev = beforeRev.asTrunkRevision();
+            afterRev = afterRev.asTrunkRevision();
+        } else {
+            beforeRev = getBaseRevision(beforeRev);
+            afterRev = getBaseRevision(afterRev);
+        }
+        if (beforeRev.equals(afterRev)) {
+            return;
+        }
+
+        int clusterId = ns.getClusterId();
+        RevisionVector max = beforeRev.pmax(afterRev);
+        RevisionVector min = beforeRev.pmin(afterRev);
+        for (Revision to : max) {
+            Revision from = min.getRevision(to.getClusterId());
+            if (from == null) {
+                // there is no min revision with this clusterId
+                // use revision with a timestamp of zero
+                from = new Revision(0, 0, to.getClusterId());
+            }
+            fillExternalChanges(changes, from, to, ns.getDocumentStore());
+        }
+        // do we need to include changes from pending local changes?
+        if (!max.isRevisionNewer(localLastRev)
+                && !localLastRev.equals(max.getRevision(clusterId))) {
+            // journal does not contain all local changes
+            localPending.addTo(changes);
+        }
+    }
+
+    @Nonnull
+    private RevisionVector getBaseRevision(RevisionVector rv) {
+        if (!rv.isBranch()) {
+            return rv;
+        }
+        Branch b = ns.getBranches().getBranch(rv);
+        if (b != null) {
+            rv = b.getBase(rv.getBranchRevision());
+        } else if (ns.getBranches().isBranchBase(rv)) {
+            rv = rv.asTrunkRevision();
+        } else {
+            missingBranch(rv);
+        }
+        return rv;
+    }
+
+    private static void missingBranch(RevisionVector rv) {
+        throw new IllegalStateException("Missing branch for revision " + rv);
+    }
+
+    private static class WrappedDiffCache extends DiffCache {
+
+        private final String path;
+        private String changes = "";
+        private final DiffCache cache;
+
+        WrappedDiffCache(String path, DiffCache cache) {
+            this.path = path;
+            this.cache = cache;
+        }
+
+        @CheckForNull
+        String getChanges() {
+            return changes;
+        }
+
+        @Override
+        String getChanges(@Nonnull RevisionVector from,
+                          @Nonnull RevisionVector to,
+                          @Nonnull String path,
+                          @Nullable Loader loader) {
+            return cache.getChanges(from, to, path, loader);
+        }
+
+        @Nonnull
+        @Override
+        Entry newEntry(@Nonnull RevisionVector from,
+                       @Nonnull RevisionVector to,
+                       boolean local) {
+            final Entry entry = cache.newEntry(from, to, local);
+            return new Entry() {
+                @Override
+                public void append(@Nonnull String path,
+                                   @Nonnull String changes) {
+                    entry.append(path, changes);
+                    if (path.equals(WrappedDiffCache.this.path)) {
+                        WrappedDiffCache.this.changes = changes;
+                    }
+                }
+
+                @Override
+                public boolean done() {
+                    return entry.done();
+                }
+            };
+        }
+
+        @Nonnull
+        @Override
+        Iterable<CacheStats> getStats() {
+            return cache.getStats();
+        }
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalDiffLoader.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalEntry.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalEntry.java?rev=1752596&r1=1752595&r2=1752596&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalEntry.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalEntry.java Thu Jul 14 07:39:01 2016
@@ -168,7 +168,9 @@ public final class JournalEntry extends
     /**
      * Reads all external changes between the two given revisions (with the same
      * clusterId) from the journal and appends the paths therein to the provided
-     * sorter.
+     * sorter. If there is no exact match of a journal entry for the given
+     * {@code to} revision, this method will fill external changes from the
+     * next higher journal entry that contains the revision.
      *
      * @param sorter the StringSort to which all externally changed paths
      *               between the provided revisions will be added
@@ -184,6 +186,10 @@ public final class JournalEntry extends
             throws IOException {
         checkArgument(checkNotNull(from).getClusterId() == checkNotNull(to).getClusterId());
 
+        if (from.compareRevisionTime(to) >= 0) {
+            return;
+        }
+
         // to is inclusive, but DocumentStore.query() toKey is exclusive
         final String inclusiveToId = asId(to);
         to = new Revision(to.getTimestamp(), to.getCounter() + 1,
@@ -196,6 +202,8 @@ public final class JournalEntry extends
         // limit, then loop and do subsequent queries
         final String toId = asId(to);
         String fromId = asId(from);
+        int numEntries = 0;
+        JournalEntry lastEntry = null;
         while (true) {
             if (fromId.equals(inclusiveToId)) {
                 // avoid query if from and to are off by just 1 counter (which
@@ -205,6 +213,10 @@ public final class JournalEntry extends
                 break;
             }
             List<JournalEntry> partialResult = store.query(JOURNAL, fromId, toId, READ_CHUNK_SIZE);
+            numEntries += partialResult.size();
+            if (!partialResult.isEmpty()) {
+                lastEntry = partialResult.get(partialResult.size() - 1);
+            }
 
             for (JournalEntry d : partialResult) {
                 d.addTo(sorter);
@@ -217,6 +229,16 @@ public final class JournalEntry extends
             // include the from which we'd otherwise double-process)
             fromId = partialResult.get(partialResult.size() - 1).getId();
         }
+        // check if last processed journal entry covers toId, otherwise
+        // read next document. also read next journal entry when none
+        // were read so far
+        if (numEntries == 0
+                || (lastEntry != null && !lastEntry.getId().equals(inclusiveToId))) {
+            String maxId = asId(new Revision(Long.MAX_VALUE, 0, to.getClusterId()));
+            for (JournalEntry d : store.query(JOURNAL, inclusiveToId, maxId, 1)) {
+                d.addTo(sorter);
+            }
+        }
     }
 
     long getRevisionTimestamp() {

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalGarbageCollector.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalGarbageCollector.java?rev=1752596&r1=1752595&r2=1752596&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalGarbageCollector.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JournalGarbageCollector.java Thu Jul 14 07:39:01 2016
@@ -22,11 +22,14 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.jackrabbit.oak.plugins.document.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Stopwatch;
 
+import static org.apache.jackrabbit.oak.plugins.document.Collection.SETTINGS;
+
 /**
  * The JournalGarbageCollector can clean up JournalEntries that are older than a
  * particular age.
@@ -38,12 +41,27 @@ import com.google.common.base.Stopwatch;
  */
 public class JournalGarbageCollector {
 
+    private static final Logger log = LoggerFactory.getLogger(JournalGarbageCollector.class);
+
+    /**
+     * ID of the journalGC document in the settings collection.
+     */
+    private static final String JOURNAL_GC_ID = "journalGC";
+
+    /**
+     * Key name of the entry that contains the timestamp of the journal tail.
+     */
+    private static final String TAIL_TIMESTAMP = "tailTimestamp";
+
     private final DocumentNodeStore ns;
 
-    private static final Logger log = LoggerFactory.getLogger(JournalGarbageCollector.class);
+    private volatile long lastTailTimestampRefresh = Long.MIN_VALUE;
+
+    private Revision tailRevision;
 
     public JournalGarbageCollector(DocumentNodeStore nodeStore) {
         this.ns = nodeStore;
+        this.tailRevision = new Revision(0, 0, ns.getClusterId());
     }
 
     /**
@@ -89,6 +107,10 @@ public class JournalGarbageCollector {
         // will compete at deletion, which is not optimal
         // due to performance, but does not harm.
 
+        // update the tail timestamp in the journalGC document
+        // of the settings collection
+        updateTailTimestamp(gcOlderThan);
+
         // 1. get the list of cluster node ids
         final List<ClusterNodeInfoDocument> clusterNodeInfos = ClusterNodeInfoDocument.all(ds);
         int numDeleted = 0;
@@ -139,6 +161,36 @@ public class JournalGarbageCollector {
         return numDeleted;
     }
 
+    private void updateTailTimestamp(long gcOlderThan) {
+        UpdateOp op = new UpdateOp(JOURNAL_GC_ID, true);
+        op.max(TAIL_TIMESTAMP, gcOlderThan);
+        ns.getDocumentStore().createOrUpdate(SETTINGS, op);
+    }
+
+    public Revision getTailRevision() {
+        refreshTailRevisionIfNecessary();
+        return tailRevision;
+    }
+
+    private void refreshTailRevisionIfNecessary() {
+        // refresh once a minute
+        long now = ns.getClock().getTime();
+        if (lastTailTimestampRefresh + TimeUnit.MINUTES.toMillis(1) > now) {
+            return;
+        }
+        lastTailTimestampRefresh = now;
+
+        Document doc = ns.getDocumentStore().find(SETTINGS, JOURNAL_GC_ID);
+        if (doc == null) {
+            // no gc yet
+            return;
+        }
+        Long ts = Utils.asLong((Number) doc.get(TAIL_TIMESTAMP));
+        if (ts != null) {
+            tailRevision = Utils.max(tailRevision, new Revision(ts, 0, ns.getClusterId()));
+        }
+    }
+
     private List<String> asKeys(List<JournalEntry> deletionBatch) {
         final List<String> keys = new ArrayList<String>(deletionBatch.size());
         for (JournalEntry e : deletionBatch) {

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JsopNodeStateDiffer.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JsopNodeStateDiffer.java?rev=1752596&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JsopNodeStateDiffer.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JsopNodeStateDiffer.java Thu Jul 14 07:39:01 2016
@@ -0,0 +1,103 @@
+/*
+ * 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.jackrabbit.oak.plugins.document;
+
+import javax.annotation.Nonnull;
+
+import org.apache.jackrabbit.oak.spi.state.AbstractNodeState;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+import org.apache.jackrabbit.oak.spi.state.NodeStateDiff;
+
+/**
+ * A {@link NodeStateDiffer} implementation backed by a JSOP String.
+ */
+class JsopNodeStateDiffer implements NodeStateDiffer {
+
+    private final String jsonDiff;
+    private boolean withoutPropertyChanges = false;
+
+    JsopNodeStateDiffer(String diff) {
+        this.jsonDiff = diff == null ? "" : diff;
+    }
+
+    JsopNodeStateDiffer withoutPropertyChanges() {
+        withoutPropertyChanges = true;
+        return this;
+    }
+
+    @Override
+    public boolean compare(@Nonnull final AbstractDocumentNodeState node,
+                           @Nonnull final AbstractDocumentNodeState base,
+                           @Nonnull final NodeStateDiff diff) {
+        if (!withoutPropertyChanges) {
+            if (!AbstractNodeState.comparePropertiesAgainstBaseState(node, base, diff)) {
+                return false;
+            }
+        }
+
+        return DiffCache.parseJsopDiff(jsonDiff, new DiffCache.Diff() {
+            @Override
+            public boolean childNodeAdded(String name) {
+                return diff.childNodeAdded(name,
+                        node.getChildNode(name));
+            }
+
+            @Override
+            public boolean childNodeChanged(String name) {
+                boolean continueComparison = true;
+                NodeState baseChild = base.getChildNode(name);
+                NodeState nodeChild = node.getChildNode(name);
+                if (baseChild.exists()) {
+                    if (nodeChild.exists()) {
+                        continueComparison = compareExisting(
+                                baseChild, nodeChild, name, diff);
+                    } else {
+                        continueComparison = diff.childNodeDeleted(name,
+                                baseChild);
+                    }
+                } else {
+                    if (nodeChild.exists()) {
+                        continueComparison = diff.childNodeAdded(name,
+                                nodeChild);
+                    }
+                }
+                return continueComparison;
+            }
+
+            @Override
+            public boolean childNodeDeleted(String name) {
+                return diff.childNodeDeleted(name,
+                        base.getChildNode(name));
+            }
+        });
+    }
+
+    private static boolean compareExisting(NodeState baseChild,
+                                           NodeState nodeChild,
+                                           String name,
+                                           NodeStateDiff diff) {
+        if (baseChild instanceof AbstractDocumentNodeState
+                && nodeChild instanceof AbstractDocumentNodeState) {
+            AbstractDocumentNodeState beforeChild = (AbstractDocumentNodeState) baseChild;
+            AbstractDocumentNodeState afterChild = (AbstractDocumentNodeState) nodeChild;
+            if (beforeChild.getLastRevision().equals(afterChild.getLastRevision())) {
+                return true;
+            }
+        }
+        return diff.childNodeChanged(name, baseChild, nodeChild);
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/JsopNodeStateDiffer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/UnmergedBranches.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/UnmergedBranches.java?rev=1752596&r1=1752595&r2=1752596&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/UnmergedBranches.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/document/UnmergedBranches.java Thu Jul 14 07:39:01 2016
@@ -138,6 +138,26 @@ class UnmergedBranches {
     }
 
     /**
+     * Returns {@code true} if the given revision is the base of an unmerged
+     * branch.
+     *
+     * @param r the base revision of a branch.
+     * @return {@code true} if such a branch exists, {@code false} otherwise.
+     */
+    boolean isBranchBase(@Nonnull RevisionVector r) {
+        if (!r.isBranch()) {
+            return false;
+        }
+        RevisionVector base = r.asTrunkRevision();
+        for (Branch b : branches) {
+            if (b.getBase().equals(base)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    /**
      * Returns the branch commit with the given revision or {@code null} if
      * it doesn't exists.
      *

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=1752596&r1=1752595&r2=1752596&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 Jul 14 07:39:01 2016
@@ -2167,6 +2167,10 @@ public class DocumentNodeStoreTest {
     // OAK-1970
     @Test
     public void diffMany() throws Exception {
+        // make sure diffMany is used and not the new
+        // journal diff introduced with OAK-4528
+        System.setProperty("oak.disableJournalDiff", "true");
+
         Clock clock = new Clock.Virtual();
         clock.waitUntil(System.currentTimeMillis());
         Revision.setClock(clock);
@@ -2229,6 +2233,8 @@ public class DocumentNodeStoreTest {
         // startValue must be based on the revision of the before state
         // and not when '/test' was last modified
         assertEquals(beforeModified, (long) startValues.get(0));
+
+        System.clearProperty("oak.disableJournalDiff");
     }
 
     // OAK-2620

Added: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/JournalDiffLoaderTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/JournalDiffLoaderTest.java?rev=1752596&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/JournalDiffLoaderTest.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/JournalDiffLoaderTest.java Thu Jul 14 07:39:01 2016
@@ -0,0 +1,196 @@
+/*
+ * 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.jackrabbit.oak.plugins.document;
+
+import java.util.Set;
+
+import org.apache.jackrabbit.oak.api.CommitFailedException;
+import org.apache.jackrabbit.oak.plugins.document.memory.MemoryDocumentStore;
+import org.apache.jackrabbit.oak.spi.commit.CommitInfo;
+import org.apache.jackrabbit.oak.spi.commit.EmptyHook;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeStore;
+import org.junit.Rule;
+import org.junit.Test;
+
+import static com.google.common.collect.Sets.newHashSet;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class JournalDiffLoaderTest {
+
+    @Rule
+    public DocumentMKBuilderProvider builderProvider = new DocumentMKBuilderProvider();
+
+    @Test
+    public void fromCurrentJournalEntry() throws Exception {
+        DocumentNodeStore ns = builderProvider.newBuilder()
+                .setAsyncDelay(0).getNodeStore();
+        DocumentNodeState s1 = ns.getRoot();
+        NodeBuilder builder = ns.getRoot().builder();
+        builder.child("foo");
+        merge(ns, builder);
+        DocumentNodeState s2 = ns.getRoot();
+
+        assertEquals(newHashSet("foo"), changeChildNodes(ns, s1, s2));
+    }
+
+    @Test
+    public void fromSingleJournalEntry() throws Exception {
+        DocumentNodeStore ns = builderProvider.newBuilder()
+                .setAsyncDelay(0).getNodeStore();
+        DocumentNodeState s1 = ns.getRoot();
+        NodeBuilder builder = ns.getRoot().builder();
+        builder.child("foo");
+        merge(ns, builder);
+        DocumentNodeState s2 = ns.getRoot();
+        ns.runBackgroundOperations();
+
+        assertEquals(newHashSet("foo"), changeChildNodes(ns, s1, s2));
+    }
+
+    @Test
+    public void fromJournalAndCurrentEntry() throws Exception {
+        DocumentNodeStore ns = builderProvider.newBuilder()
+                .setAsyncDelay(0).getNodeStore();
+        DocumentNodeState s1 = ns.getRoot();
+        NodeBuilder builder = ns.getRoot().builder();
+        builder.child("foo");
+        merge(ns, builder);
+
+        ns.runBackgroundOperations();
+
+        builder = ns.getRoot().builder();
+        builder.child("bar");
+        merge(ns, builder);
+        DocumentNodeState s2 = ns.getRoot();
+
+        assertEquals(newHashSet("foo", "bar"), changeChildNodes(ns, s1, s2));
+    }
+
+    @Test
+    public void fromMultipleJournalEntries() throws Exception {
+        DocumentNodeStore ns = builderProvider.newBuilder()
+                .setAsyncDelay(0).getNodeStore();
+        DocumentNodeState s1 = ns.getRoot();
+        NodeBuilder builder = ns.getRoot().builder();
+        builder.child("foo");
+        merge(ns, builder);
+        ns.runBackgroundOperations();
+
+        builder = ns.getRoot().builder();
+        builder.child("bar");
+        merge(ns, builder);
+        ns.runBackgroundOperations();
+
+        builder = ns.getRoot().builder();
+        builder.child("baz");
+        merge(ns, builder);
+        ns.runBackgroundOperations();
+
+        DocumentNodeState s2 = ns.getRoot();
+
+        assertEquals(newHashSet("foo", "bar", "baz"), changeChildNodes(ns, s1, s2));
+    }
+
+    @Test
+    public void fromPartialJournalEntry() throws Exception {
+        DocumentNodeStore ns = builderProvider.newBuilder()
+                .setAsyncDelay(0).getNodeStore();
+        DocumentNodeState s1 = ns.getRoot();
+        NodeBuilder builder = ns.getRoot().builder();
+        builder.child("foo");
+        merge(ns, builder);
+        ns.runBackgroundOperations();
+
+        builder = ns.getRoot().builder();
+        builder.child("bar");
+        merge(ns, builder);
+
+        DocumentNodeState s2 = ns.getRoot();
+
+        builder = ns.getRoot().builder();
+        builder.child("baz");
+        merge(ns, builder);
+        ns.runBackgroundOperations();
+
+        // will also report 'baz' because that change is also
+        // present in the second journal entry
+        assertEquals(newHashSet("foo", "bar", "baz"), changeChildNodes(ns, s1, s2));
+    }
+
+    @Test
+    public void fromExternalChange() throws Exception {
+        DocumentStore store = new MemoryDocumentStore();
+        DocumentNodeStore ns1 = builderProvider.newBuilder().setClusterId(1)
+                .setDocumentStore(store).setAsyncDelay(0).getNodeStore();
+        DocumentNodeStore ns2 = builderProvider.newBuilder().setClusterId(2)
+                .setDocumentStore(store).setAsyncDelay(0).getNodeStore();
+
+        DocumentNodeState s1 = ns1.getRoot();
+        NodeBuilder builder = ns1.getRoot().builder();
+        builder.child("foo");
+        merge(ns1, builder);
+
+        builder = ns2.getRoot().builder();
+        builder.child("bar");
+        merge(ns2, builder);
+        ns2.runBackgroundOperations();
+
+        // create journal entry for ns1 and pick up changes from ns2
+        ns1.runBackgroundOperations();
+
+        builder = ns1.getRoot().builder();
+        builder.child("baz");
+        merge(ns1, builder);
+
+        DocumentNodeState s2 = ns1.getRoot();
+        assertEquals(newHashSet("foo", "bar", "baz"), changeChildNodes(ns1, s1, s2));
+    }
+
+    private static Set<String> changeChildNodes(DocumentNodeStore store,
+                                                AbstractDocumentNodeState before,
+                                                AbstractDocumentNodeState after) {
+        String diff = new JournalDiffLoader(before, after, store).call();
+        final Set<String> changes = newHashSet();
+        DiffCache.parseJsopDiff(diff, new DiffCache.Diff() {
+            @Override
+            public boolean childNodeAdded(String name) {
+                fail();
+                return true;
+            }
+
+            @Override
+            public boolean childNodeChanged(String name) {
+                changes.add(name);
+                return true;
+            }
+
+            @Override
+            public boolean childNodeDeleted(String name) {
+                fail();
+                return true;
+            }
+        });
+        return changes;
+    }
+
+    private static void merge(NodeStore store, NodeBuilder builder)
+            throws CommitFailedException {
+        store.merge(builder, EmptyHook.INSTANCE, CommitInfo.EMPTY);
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/JournalDiffLoaderTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/JournalEntryTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/JournalEntryTest.java?rev=1752596&r1=1752595&r2=1752596&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/JournalEntryTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/JournalEntryTest.java Thu Jul 14 07:39:01 2016
@@ -136,6 +136,76 @@ public class JournalEntryTest {
     }
 
     @Test
+    public void fillExternalChanges2() throws Exception {
+        Revision r1 = new Revision(1, 0, 1);
+        Revision r2 = new Revision(2, 0, 1);
+        Revision r3 = new Revision(3, 0, 1);
+        Revision r4 = new Revision(4, 0, 1);
+        DocumentStore store = new MemoryDocumentStore();
+        JournalEntry entry = JOURNAL.newDocument(store);
+        entry.modified("/");
+        entry.modified("/foo");
+        UpdateOp op = entry.asUpdateOp(r2);
+        assertTrue(store.create(JOURNAL, Collections.singletonList(op)));
+
+        entry = JOURNAL.newDocument(store);
+        entry.modified("/");
+        entry.modified("/bar");
+        op = entry.asUpdateOp(r4);
+        assertTrue(store.create(JOURNAL, Collections.singletonList(op)));
+
+        StringSort sort = JournalEntry.newSorter();
+        JournalEntry.fillExternalChanges(sort, r1, r1, store);
+        assertEquals(0, sort.getSize());
+        sort.close();
+
+        sort = JournalEntry.newSorter();
+        JournalEntry.fillExternalChanges(sort, r1, r2, store);
+        assertEquals(Sets.newHashSet("/", "/foo"), Sets.newHashSet(sort));
+        sort.close();
+
+        sort = JournalEntry.newSorter();
+        JournalEntry.fillExternalChanges(sort, r1, r3, store);
+        assertEquals(Sets.newHashSet("/", "/foo", "/bar"), Sets.newHashSet(sort));
+        sort.close();
+
+        sort = JournalEntry.newSorter();
+        JournalEntry.fillExternalChanges(sort, r1, r4, store);
+        assertEquals(Sets.newHashSet("/", "/foo", "/bar"), Sets.newHashSet(sort));
+        sort.close();
+
+        sort = JournalEntry.newSorter();
+        JournalEntry.fillExternalChanges(sort, r2, r2, store);
+        assertEquals(0, sort.getSize());
+        sort.close();
+
+        sort = JournalEntry.newSorter();
+        JournalEntry.fillExternalChanges(sort, r2, r3, store);
+        assertEquals(Sets.newHashSet("/", "/bar"), Sets.newHashSet(sort));
+        sort.close();
+
+        sort = JournalEntry.newSorter();
+        JournalEntry.fillExternalChanges(sort, r2, r4, store);
+        assertEquals(Sets.newHashSet("/", "/bar"), Sets.newHashSet(sort));
+        sort.close();
+
+        sort = JournalEntry.newSorter();
+        JournalEntry.fillExternalChanges(sort, r3, r3, store);
+        assertEquals(0, sort.getSize());
+        sort.close();
+
+        sort = JournalEntry.newSorter();
+        JournalEntry.fillExternalChanges(sort, r3, r4, store);
+        assertEquals(Sets.newHashSet("/", "/bar"), Sets.newHashSet(sort));
+        sort.close();
+
+        sort = JournalEntry.newSorter();
+        JournalEntry.fillExternalChanges(sort, r4, r4, store);
+        assertEquals(0, sort.getSize());
+        sort.close();
+    }
+
+    @Test
     public void getRevisionTimestamp() throws Exception {
         DocumentStore store = new MemoryDocumentStore();
         JournalEntry entry = JOURNAL.newDocument(store);

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/JournalGCTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/JournalGCTest.java?rev=1752596&r1=1752595&r2=1752596&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/JournalGCTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/JournalGCTest.java Thu Jul 14 07:39:01 2016
@@ -28,6 +28,7 @@ import org.junit.Rule;
 import org.junit.Test;
 
 import static org.apache.jackrabbit.oak.plugins.document.Collection.JOURNAL;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 
@@ -74,4 +75,41 @@ public class JournalGCTest {
         entry = ns.getDocumentStore().find(JOURNAL, JournalEntry.asId(head));
         assertNull(entry);
     }
+
+    @Test
+    public void getTailRevision() throws Exception {
+        Clock c = new Clock.Virtual();
+        c.waitUntil(System.currentTimeMillis());
+        DocumentNodeStore ns = builderProvider.newBuilder()
+                .clock(c).setAsyncDelay(0).getNodeStore();
+
+        JournalGarbageCollector jgc = ns.getJournalGarbageCollector();
+        assertEquals(new Revision(0, 0, ns.getClusterId()), jgc.getTailRevision());
+
+        NodeBuilder builder = ns.getRoot().builder();
+        builder.child("foo");
+        ns.merge(builder, EmptyHook.INSTANCE, CommitInfo.EMPTY);
+        ns.runBackgroundOperations();
+
+        assertEquals(0, jgc.gc(1, 10, TimeUnit.HOURS));
+
+        // current time, but without the increment done by getTime()
+        long now = c.getTime() - 1;
+        Revision tail = new Revision(now - TimeUnit.HOURS.toMillis(1), 0, ns.getClusterId());
+
+        c.waitUntil(c.getTime() + TimeUnit.MINUTES.toMillis(1));
+        assertEquals(tail, jgc.getTailRevision());
+
+        c.waitUntil(c.getTime() + TimeUnit.HOURS.toMillis(1));
+
+        // must collect all journal entries. the first created when
+        // DocumentNodeStore was initialized and the second created
+        // by the background update
+        assertEquals(2, jgc.gc(1, 10, TimeUnit.HOURS));
+
+        // current time, but without the increment done by getTime()
+        now = c.getTime() - 1;
+        tail = new Revision(now - TimeUnit.HOURS.toMillis(1), 0, ns.getClusterId());
+        assertEquals(tail, jgc.getTailRevision());
+    }
 }

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/SimpleTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/SimpleTest.java?rev=1752596&r1=1752595&r2=1752596&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/SimpleTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/SimpleTest.java Thu Jul 14 07:39:01 2016
@@ -16,10 +16,13 @@
  */
 package org.apache.jackrabbit.oak.plugins.document;
 
+import static org.hamcrest.CoreMatchers.anyOf;
+import static org.hamcrest.CoreMatchers.equalTo;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -162,7 +165,7 @@ public class SimpleTest {
         String diff23 = mk.diff(rev2, rev3, "/", 0).trim();
         assertEquals("+\"/t3\":{}", diff23);
         String diff13 = mk.diff(rev1, rev3, "/", 0).trim();
-        assertEquals("+\"/t2\":{}+\"/t3\":{}", diff13);
+        assertThat(diff13, anyOf(equalTo("+\"/t2\":{}+\"/t3\":{}"), equalTo("+\"/t3\":{}+\"/t2\":{}")));
         String diff34 = mk.diff(rev3, rev4, "/", 0).trim();
         assertEquals("^\"/t3\":{}", diff34);
     }