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/04/05 10:46:47 UTC

svn commit: r1737801 - 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: Tue Apr  5 08:46:47 2016
New Revision: 1737801

URL: http://svn.apache.org/viewvc?rev=1737801&view=rev
Log:
OAK-2392: [DocumentMK] Garbage Collect older revisions of binary properties in main document

Modified:
    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/DocumentNodeStore.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/CollisionWithSplitTest.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentSplitTest.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/NodeDocumentTest.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/PreviousDocCacheTest.java

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=1737801&r1=1737800&r2=1737801&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 Tue Apr  5 08:46:47 2016
@@ -447,7 +447,10 @@ public class Commit {
     }
 
     private void checkSplitCandidate(@Nullable NodeDocument doc) {
-        if (doc != null && doc.getMemory() > SPLIT_CANDIDATE_THRESHOLD) {
+        if (doc == null) {
+            return;
+        }
+        if (doc.getMemory() > SPLIT_CANDIDATE_THRESHOLD || doc.hasBinary()) {
             nodeStore.addSplitCandidate(doc.getId());
         }
     }

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=1737801&r1=1737800&r2=1737801&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 Tue Apr  5 08:46:47 2016
@@ -64,6 +64,7 @@ import java.util.concurrent.locks.Reentr
 import javax.annotation.CheckForNull;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
+import javax.jcr.PropertyType;
 import javax.management.NotCompliantMBeanException;
 
 import com.google.common.base.Function;
@@ -360,6 +361,23 @@ public final class DocumentNodeStore
         }
     };
 
+    /**
+     * A predicate, which takes a String and returns {@code true} if the String
+     * is a serialized binary value of a {@link DocumentPropertyState}. The
+     * apply method will throw an IllegalArgumentException if the String is
+     * malformed.
+     */
+    private final Predicate<String> isBinary = new Predicate<String>() {
+        @Override
+        public boolean apply(@Nullable String input) {
+            if (input == null) {
+                return false;
+            }
+            return new DocumentPropertyState(DocumentNodeStore.this,
+                    "p", input).getType().tag() == PropertyType.BINARY;
+        }
+    };
+
     private final Clock clock;
 
     private final Checkpoints checkpoints;
@@ -1972,7 +1990,7 @@ public final class DocumentNodeStore
             if (doc == null) {
                 continue;
             }
-            for (UpdateOp op : doc.split(this, head)) {
+            for (UpdateOp op : doc.split(this, head, isBinary)) {
                 NodeDocument before = null;
                 if (!op.isNew() ||
                         !store.create(Collection.NODES, Collections.singletonList(op))) {

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=1737801&r1=1737800&r2=1737801&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 Tue Apr  5 08:46:47 2016
@@ -1183,12 +1183,17 @@ public final class NodeDocument extends
      * @param context the revision context.
      * @param head    the head revision before this document was retrieved from
      *                the document store.
+     * @param isBinaryValue a predicate that returns {@code true} if the given
+     *                      String value is considered a binary; {@code false}
+     *                      otherwise.
      * @return the split operations.
      */
     @Nonnull
     public Iterable<UpdateOp> split(@Nonnull RevisionContext context,
-                                    @Nonnull RevisionVector head) {
-        return SplitOperations.forDocument(this, context, head, NUM_REVS_THRESHOLD);
+                                    @Nonnull RevisionVector head,
+                                    @Nonnull Predicate<String> isBinaryValue) {
+        return SplitOperations.forDocument(this, context, head,
+                isBinaryValue, NUM_REVS_THRESHOLD);
     }
 
     /**

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=1737801&r1=1737800&r2=1737801&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 Tue Apr  5 08:46:47 2016
@@ -35,11 +35,13 @@ import org.apache.jackrabbit.oak.plugins
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Predicate;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
 import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.collect.Iterables.any;
 import static com.google.common.collect.Sets.filter;
 import static org.apache.jackrabbit.oak.plugins.document.NodeDocument.COMMIT_ROOT;
 import static org.apache.jackrabbit.oak.plugins.document.NodeDocument.DOC_SIZE_THRESHOLD;
@@ -69,10 +71,12 @@ class SplitOperations {
     private final String id;
     private final Revision headRevision;
     private final RevisionContext context;
+    private final Predicate<String> isBinaryValue;
     private final int numRevsThreshold;
     private Revision high;
     private Revision low;
     private int numValues;
+    private boolean hasBinary;
     private Map<String, NavigableMap<Revision, String>> committedChanges;
     private Set<Revision> changes;
     private Map<String, Set<Revision>> garbage;
@@ -85,9 +89,11 @@ class SplitOperations {
     private SplitOperations(@Nonnull NodeDocument doc,
                             @Nonnull RevisionContext context,
                             @Nonnull RevisionVector headRevision,
+                            @Nonnull Predicate<String> isBinaryValue,
                             int numRevsThreshold) {
         this.doc = checkNotNull(doc);
         this.context = checkNotNull(context);
+        this.isBinaryValue = checkNotNull(isBinaryValue);
         this.path = doc.getPath();
         this.id = doc.getId();
         this.headRevision = checkNotNull(headRevision).getRevision(context.getClusterId());
@@ -106,6 +112,9 @@ class SplitOperations {
      * @param context the revision context.
      * @param headRevision the head revision before the document was retrieved
      *                     from the document store.
+     * @param isBinaryValue a predicate that returns {@code true} if the given
+     *                      String value is considered a binary; {@code false}
+     *                      otherwise.
      * @param numRevsThreshold only split off at least this number of revisions.
      * @return list of update operations. An empty list indicates the document
      *          does not require a split.
@@ -116,12 +125,14 @@ class SplitOperations {
     static List<UpdateOp> forDocument(@Nonnull NodeDocument doc,
                                       @Nonnull RevisionContext context,
                                       @Nonnull RevisionVector headRevision,
+                                      @Nonnull Predicate<String> isBinaryValue,
                                       int numRevsThreshold) {
         if (doc.isSplitDocument()) {
             throw new IllegalArgumentException(
                     "Not a main document: " + doc.getId());
         }
-        return new SplitOperations(doc, context, headRevision, numRevsThreshold).create();
+        return new SplitOperations(doc, context, headRevision,
+                isBinaryValue, numRevsThreshold).create();
 
     }
 
@@ -173,7 +184,8 @@ class SplitOperations {
         return doc.getLocalRevisions().size() + doc.getLocalCommitRoot().size() > numRevsThreshold
                 || doc.getMemory() >= DOC_SIZE_THRESHOLD
                 || previous.size() >= PREV_SPLIT_FACTOR
-                || !doc.getStalePrev().isEmpty();
+                || !doc.getStalePrev().isEmpty()
+                || doc.hasBinary();
     }
 
     /**
@@ -188,6 +200,7 @@ class SplitOperations {
                 Revision r = splitMap.lastKey();
                 splitMap.remove(r);
                 splitRevs.addAll(splitMap.keySet());
+                hasBinary |= hasBinaryProperty(splitMap.values());
                 mostRecentRevs.add(r);
             }
             if (splitMap.isEmpty()) {
@@ -200,6 +213,10 @@ class SplitOperations {
         }
     }
 
+    private boolean hasBinaryProperty(Iterable<String> values) {
+        return doc.hasBinary() && any(values, isBinaryValue);
+    }
+
     /**
      * Collect _revisions and _commitRoot entries that can be moved to a
      * previous document.
@@ -310,7 +327,8 @@ class SplitOperations {
         // check if we have enough data to split off
         if (high != null && low != null
                 && (numValues >= numRevsThreshold
-                || doc.getMemory() > DOC_SIZE_THRESHOLD)) {
+                || doc.getMemory() > DOC_SIZE_THRESHOLD
+                || hasBinary)) {
             // enough changes to split off
             // move to another document
             main = new UpdateOp(id, false);
@@ -342,8 +360,10 @@ class SplitOperations {
             UpdateUtils.applyChanges(oldDoc, old);
             setSplitDocProps(doc, oldDoc, old, high);
             // only split if enough of the data can be moved to old document
+            // or there are binaries to split off
             if (oldDoc.getMemory() > doc.getMemory() * SPLIT_RATIO
-                    || numValues >= numRevsThreshold) {
+                    || numValues >= numRevsThreshold
+                    || hasBinary) {
                 splitOps.add(old);
             } else {
                 main = null;

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=1737801&r1=1737800&r2=1737801&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 Tue Apr  5 08:46:47 2016
@@ -19,6 +19,7 @@ package org.apache.jackrabbit.oak.plugin
 import java.util.Collections;
 import java.util.List;
 
+import com.google.common.base.Predicates;
 import com.mongodb.DB;
 
 import org.apache.jackrabbit.oak.api.CommitFailedException;
@@ -31,6 +32,7 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import static com.google.common.base.Predicates.alwaysFalse;
 import static org.apache.jackrabbit.oak.plugins.document.Collection.NODES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -91,8 +93,8 @@ public class CollisionWithSplitTest exte
         // run document split on ns1
         DocumentStore store = ns1.getDocumentStore();
         NodeDocument doc = Utils.getRootDocument(store);
-        List<UpdateOp> ops = SplitOperations.forDocument(doc,
-                ns1, ns1.getHeadRevision(), NUM_NODES);
+        List<UpdateOp> ops = SplitOperations.forDocument(doc, ns1,
+                ns1.getHeadRevision(), Predicates.<String>alwaysFalse(), NUM_NODES);
         assertFalse(ops.isEmpty());
         for (UpdateOp op : ops) {
             if (!op.isNew() ||

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=1737801&r1=1737800&r2=1737801&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 Tue Apr  5 08:46:47 2016
@@ -28,10 +28,12 @@ import java.util.TreeSet;
 import javax.annotation.Nonnull;
 
 import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterators;
 
 import org.apache.jackrabbit.oak.api.CommitFailedException;
+import org.apache.jackrabbit.oak.api.PropertyState;
 import org.apache.jackrabbit.oak.commons.PathUtils;
 import org.apache.jackrabbit.oak.plugins.document.UpdateOp.Key;
 import org.apache.jackrabbit.oak.plugins.document.UpdateOp.Operation;
@@ -49,6 +51,7 @@ import com.google.common.collect.Iterabl
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
+import static com.google.common.collect.ImmutableList.copyOf;
 import static org.apache.jackrabbit.oak.plugins.document.Collection.NODES;
 import static org.apache.jackrabbit.oak.plugins.document.MongoBlobGCTest.randomStream;
 import static org.apache.jackrabbit.oak.plugins.document.NodeDocument.NUM_REVS_THRESHOLD;
@@ -56,6 +59,7 @@ import static org.apache.jackrabbit.oak.
 import static org.apache.jackrabbit.oak.plugins.document.NodeDocument.SplitDocType;
 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.memory.BinaryPropertyState.binaryProperty;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -535,8 +539,9 @@ public class DocumentSplitTest extends B
 
         doc = store.find(NODES, id);
         assertNotNull(doc);
-        List<UpdateOp> splitOps = Lists.newArrayList(
-                doc.split(mk.getNodeStore(), mk.getNodeStore().getHeadRevision()));
+        List<UpdateOp> splitOps = Lists.newArrayList(doc.split(
+                mk.getNodeStore(), mk.getNodeStore().getHeadRevision(),
+                Predicates.<String>alwaysFalse()));
         assertEquals(2, splitOps.size());
         // first update op is for the new intermediate doc
         op = splitOps.get(0);
@@ -591,8 +596,9 @@ public class DocumentSplitTest extends B
 
         // must split document and create a previous document starting at
         // the second most recent revision
-        List<UpdateOp> splitOps = Lists.newArrayList(
-                doc.split(mk.getNodeStore(), mk.getNodeStore().getHeadRevision()));
+        List<UpdateOp> splitOps = Lists.newArrayList(doc.split(
+                mk.getNodeStore(), mk.getNodeStore().getHeadRevision(),
+                Predicates.<String>alwaysFalse()));
         assertEquals(2, splitOps.size());
         String prevId = Utils.getPreviousIdFor("/test", revs.get(revs.size() - 2), 0);
         assertEquals(prevId, splitOps.get(0).getId());
@@ -669,7 +675,8 @@ public class DocumentSplitTest extends B
         doc.put(NodeDocument.ID, Utils.getIdFromPath("/test"));
         doc.put(NodeDocument.SD_TYPE, NodeDocument.SplitDocType.DEFAULT.type);
         RevisionVector head = mk.getNodeStore().getHeadRevision();
-        SplitOperations.forDocument(doc, DummyRevisionContext.INSTANCE, head, NUM_REVS_THRESHOLD);
+        SplitOperations.forDocument(doc, DummyRevisionContext.INSTANCE, head,
+                Predicates.<String>alwaysFalse(), NUM_REVS_THRESHOLD);
     }
 
     @Test
@@ -851,7 +858,8 @@ public class DocumentSplitTest extends B
             for (String id : ns.getSplitCandidates()) {
                 RevisionVector head = ns.getHeadRevision();
                 NodeDocument doc = store.find(NODES, id);
-                List<UpdateOp> ops = SplitOperations.forDocument(doc, rc, head, NUM_REVS_THRESHOLD);
+                List<UpdateOp> ops = SplitOperations.forDocument(doc, rc, head,
+                        Predicates.<String>alwaysFalse(), NUM_REVS_THRESHOLD);
                 Set<Revision> removed = Sets.newHashSet();
                 Set<Revision> added = Sets.newHashSet();
                 for (UpdateOp op : ops) {
@@ -889,6 +897,30 @@ public class DocumentSplitTest extends B
         }
     }
 
+    @Test
+    public void splitDocumentWithBinary() throws Exception {
+        DocumentStore store = mk.getDocumentStore();
+        DocumentNodeStore ns = mk.getNodeStore();
+        NodeBuilder builder = ns.getRoot().builder();
+        builder.child("foo");
+        merge(ns, builder);
+
+        PropertyState binary = binaryProperty("p", "value".getBytes());
+
+        for (int i = 0; i < 10; i++) {
+            builder = ns.getRoot().builder();
+            builder.child("foo").setProperty(binary);
+            merge(ns, builder);
+            ns.runBackgroundOperations();
+        }
+
+        NodeDocument foo = store.find(NODES, Utils.getIdFromPath("/foo"));
+        assertNotNull(foo);
+        List<NodeDocument> prevDocs = copyOf(foo.getAllPreviousDocs());
+        // all but most recent value are moved to individual previous docs
+        assertEquals(9, prevDocs.size());
+    }
+
     private static class TestRevisionContext implements RevisionContext {
 
         private final RevisionContext rc;

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/NodeDocumentTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/NodeDocumentTest.java?rev=1737801&r1=1737800&r2=1737801&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/NodeDocumentTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/NodeDocumentTest.java Tue Apr  5 08:46:47 2016
@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
+import com.google.common.base.Predicates;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
@@ -68,7 +69,7 @@ public class NodeDocumentTest {
         }
         UpdateUtils.applyChanges(doc, op);
         RevisionVector head = DummyRevisionContext.INSTANCE.getHeadRevision();
-        doc.split(DummyRevisionContext.INSTANCE, head);
+        doc.split(DummyRevisionContext.INSTANCE, head, Predicates.<String>alwaysFalse());
     }
 
     @Test
@@ -201,7 +202,8 @@ public class NodeDocumentTest {
             if (r.nextFloat() < 0.2) {
                 RevisionVector head = ns.getHeadRevision();
                 for (UpdateOp op : SplitOperations.forDocument(
-                        getRootDocument(store), ns, head, 2)) {
+                        getRootDocument(store), ns, head,
+                        Predicates.<String>alwaysFalse(), 2)) {
                     store.createOrUpdate(NODES, op);
                 }
             }
@@ -317,7 +319,7 @@ public class NodeDocumentTest {
                 NodeDocument doc = ns.getDocumentStore().find(
                         NODES, Utils.getIdFromPath("/test"));
                 for (UpdateOp op : SplitOperations.forDocument(
-                        doc, ns, head, 2)) {
+                        doc, ns, head, Predicates.<String>alwaysFalse(), 2)) {
                     store.createOrUpdate(NODES, op);
                 }
             }
@@ -498,7 +500,8 @@ public class NodeDocumentTest {
             if (random.nextDouble() < 0.2) {
                 RevisionVector head = ns.getHeadRevision();
                 for (UpdateOp op : SplitOperations.forDocument(
-                        getRootDocument(store), ns, head, 2)) {
+                        getRootDocument(store), ns, head,
+                        Predicates.<String>alwaysFalse(), 2)) {
                     store.createOrUpdate(NODES, op);
                 }
             }
@@ -567,7 +570,8 @@ public class NodeDocumentTest {
             if (Math.random() < 0.2) {
                 RevisionVector head = ns.getHeadRevision();
                 for (UpdateOp op : SplitOperations.forDocument(
-                        getRootDocument(store), ns, head, 2)) {
+                        getRootDocument(store), ns, head,
+                        Predicates.<String>alwaysFalse(), 2)) {
                     store.createOrUpdate(NODES, op);
                 }
             }

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/PreviousDocCacheTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/PreviousDocCacheTest.java?rev=1737801&r1=1737800&r2=1737801&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/PreviousDocCacheTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/document/PreviousDocCacheTest.java Tue Apr  5 08:46:47 2016
@@ -1,122 +1,124 @@
-/*
- * 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 com.google.common.collect.Iterators;
-import org.apache.jackrabbit.oak.api.CommitFailedException;
-import org.apache.jackrabbit.oak.cache.CacheStats;
-import org.apache.jackrabbit.oak.plugins.document.util.Utils;
-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.Test;
-
-import java.util.Collections;
-import java.util.List;
-
-import static org.apache.jackrabbit.oak.plugins.document.Collection.NODES;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-
-public class PreviousDocCacheTest extends AbstractMongoConnectionTest {
-
-    @Test
-    public void cacheTestPrevDocs() throws Exception {
-        DocumentNodeStore ns = mk.getNodeStore();
-        DocumentStore docStore = ns.getDocumentStore();
-
-        final int SPLIT_THRESHOLD = 10;
-        NodeBuilder b;
-
-        //Set property 110 times. Split at each 10. This should lead to 11 leaf prev docs and 1 intermediate prev doc.
-        for (int j = 0; j <= SPLIT_THRESHOLD; j++) {
-            for (int i = 0; i < SPLIT_THRESHOLD; i++) {
-                b = ns.getRoot().builder();
-                b.setProperty("foo", "node-" + j + "-" + i);
-                merge(ns, b);
-            }
-            splitDocs(ns, SPLIT_THRESHOLD);
-        }
-
-        CacheStats nodesCache = null;
-        CacheStats prevDocsCache = null;
-        for (CacheStats cacheStats : docStore.getCacheStats()) {
-            if ("Document-Documents".equals(cacheStats.getName())) {
-                nodesCache = cacheStats;
-            } else if ("Document-PrevDocuments".equals(cacheStats.getName())) {
-                prevDocsCache = cacheStats;
-            }
-        }
-        assertNotNull("Nodes cache must not be null", nodesCache);
-        assertNotNull("Prev docs cache must not be null", prevDocsCache);
-
-        validateFullyLoadedCache(docStore, SPLIT_THRESHOLD, nodesCache, prevDocsCache);
-
-        docStore.invalidateCache();
-        assertEquals("No entries expected in nodes cache", 0, nodesCache.getElementCount());
-        assertEquals("No entries expected in prev docs cache", 0, prevDocsCache.getElementCount());
-
-        NodeDocument doc = docStore.find(NODES, "0:/");
-        assertEquals("Only main doc entry expected in nodes cache", 1, nodesCache.getElementCount());
-        assertEquals("No entries expected in prev docs cache", 0, prevDocsCache.getElementCount());
-
-        Iterators.size(doc.getAllPreviousDocs());
-        validateFullyLoadedCache(docStore, SPLIT_THRESHOLD, nodesCache, prevDocsCache);
-    }
-
-    private void validateFullyLoadedCache(DocumentStore docStore, int splitThreshold, CacheStats nodesCache, CacheStats prevDocsCache) {
-        assertEquals("Nodes cache must have 2 elements - '/' and intermediate split doc",
-                2, nodesCache.getElementCount());
-        assertEquals("Unexpected number of leaf prev docs", splitThreshold + 1, prevDocsCache.getElementCount());
-
-        resetStats(nodesCache, prevDocsCache);
-        NodeDocument doc = docStore.getIfCached(NODES, "0:/");
-        assertEquals("Root doc must be available in nodes cache", 1, nodesCache.getHitCount());
-        assertEquals("Prev docs must not be read", 0, prevDocsCache.getHitCount());
-
-        Iterators.size(doc.getAllPreviousDocs());
-        assertEquals("Nodes cache should not have a miss", 0, nodesCache.getMissCount());
-        assertEquals("Prev docs cache should not have a miss", 0, prevDocsCache.getMissCount());
-    }
-
-    private void resetStats(CacheStats ... cacheStatses) {
-        for (CacheStats cacheStats : cacheStatses) {
-            cacheStats.resetStats();
-        }
-    }
-
-    private void splitDocs(DocumentNodeStore ns, int splitDocLimit) {
-        DocumentStore store = ns.getDocumentStore();
-        NodeDocument doc = Utils.getRootDocument(store);
-        List<UpdateOp> ops = SplitOperations.forDocument(doc,
-                ns, ns.getHeadRevision(), splitDocLimit/2);
-        assertFalse(ops.isEmpty());
-        for (UpdateOp op : ops) {
-            if (!op.isNew() ||
-                    !store.create(NODES, Collections.singletonList(op))) {
-                store.createOrUpdate(NODES, op);
-            }
-        }
-    }
-
-    private static void merge(NodeStore store, NodeBuilder builder)
-            throws CommitFailedException {
-        store.merge(builder, EmptyHook.INSTANCE, CommitInfo.EMPTY);
-    }
-}
+/*
+ * 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 com.google.common.base.Predicates;
+import com.google.common.collect.Iterators;
+import org.apache.jackrabbit.oak.api.CommitFailedException;
+import org.apache.jackrabbit.oak.cache.CacheStats;
+import org.apache.jackrabbit.oak.plugins.document.util.Utils;
+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.Test;
+
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.jackrabbit.oak.plugins.document.Collection.NODES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+
+public class PreviousDocCacheTest extends AbstractMongoConnectionTest {
+
+    @Test
+    public void cacheTestPrevDocs() throws Exception {
+        DocumentNodeStore ns = mk.getNodeStore();
+        DocumentStore docStore = ns.getDocumentStore();
+
+        final int SPLIT_THRESHOLD = 10;
+        NodeBuilder b;
+
+        //Set property 110 times. Split at each 10. This should lead to 11 leaf prev docs and 1 intermediate prev doc.
+        for (int j = 0; j <= SPLIT_THRESHOLD; j++) {
+            for (int i = 0; i < SPLIT_THRESHOLD; i++) {
+                b = ns.getRoot().builder();
+                b.setProperty("foo", "node-" + j + "-" + i);
+                merge(ns, b);
+            }
+            splitDocs(ns, SPLIT_THRESHOLD);
+        }
+
+        CacheStats nodesCache = null;
+        CacheStats prevDocsCache = null;
+        for (CacheStats cacheStats : docStore.getCacheStats()) {
+            if ("Document-Documents".equals(cacheStats.getName())) {
+                nodesCache = cacheStats;
+            } else if ("Document-PrevDocuments".equals(cacheStats.getName())) {
+                prevDocsCache = cacheStats;
+            }
+        }
+        assertNotNull("Nodes cache must not be null", nodesCache);
+        assertNotNull("Prev docs cache must not be null", prevDocsCache);
+
+        validateFullyLoadedCache(docStore, SPLIT_THRESHOLD, nodesCache, prevDocsCache);
+
+        docStore.invalidateCache();
+        assertEquals("No entries expected in nodes cache", 0, nodesCache.getElementCount());
+        assertEquals("No entries expected in prev docs cache", 0, prevDocsCache.getElementCount());
+
+        NodeDocument doc = docStore.find(NODES, "0:/");
+        assertEquals("Only main doc entry expected in nodes cache", 1, nodesCache.getElementCount());
+        assertEquals("No entries expected in prev docs cache", 0, prevDocsCache.getElementCount());
+
+        Iterators.size(doc.getAllPreviousDocs());
+        validateFullyLoadedCache(docStore, SPLIT_THRESHOLD, nodesCache, prevDocsCache);
+    }
+
+    private void validateFullyLoadedCache(DocumentStore docStore, int splitThreshold, CacheStats nodesCache, CacheStats prevDocsCache) {
+        assertEquals("Nodes cache must have 2 elements - '/' and intermediate split doc",
+                2, nodesCache.getElementCount());
+        assertEquals("Unexpected number of leaf prev docs", splitThreshold + 1, prevDocsCache.getElementCount());
+
+        resetStats(nodesCache, prevDocsCache);
+        NodeDocument doc = docStore.getIfCached(NODES, "0:/");
+        assertEquals("Root doc must be available in nodes cache", 1, nodesCache.getHitCount());
+        assertEquals("Prev docs must not be read", 0, prevDocsCache.getHitCount());
+
+        Iterators.size(doc.getAllPreviousDocs());
+        assertEquals("Nodes cache should not have a miss", 0, nodesCache.getMissCount());
+        assertEquals("Prev docs cache should not have a miss", 0, prevDocsCache.getMissCount());
+    }
+
+    private void resetStats(CacheStats ... cacheStatses) {
+        for (CacheStats cacheStats : cacheStatses) {
+            cacheStats.resetStats();
+        }
+    }
+
+    private void splitDocs(DocumentNodeStore ns, int splitDocLimit) {
+        DocumentStore store = ns.getDocumentStore();
+        NodeDocument doc = Utils.getRootDocument(store);
+        List<UpdateOp> ops = SplitOperations.forDocument(doc,
+                ns, ns.getHeadRevision(), Predicates.<String>alwaysFalse(),
+                splitDocLimit/2);
+        assertFalse(ops.isEmpty());
+        for (UpdateOp op : ops) {
+            if (!op.isNew() ||
+                    !store.create(NODES, Collections.singletonList(op))) {
+                store.createOrUpdate(NODES, op);
+            }
+        }
+    }
+
+    private static void merge(NodeStore store, NodeBuilder builder)
+            throws CommitFailedException {
+        store.merge(builder, EmptyHook.INSTANCE, CommitInfo.EMPTY);
+    }
+}