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 2013/08/20 15:20:19 UTC

svn commit: r1515819 - in /jackrabbit/oak/trunk/oak-core/src: main/java/org/apache/jackrabbit/oak/plugins/mongomk/ main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/ test/java/org/apache/jackrabbit/oak/core/ test/java/org/apache/jackrabbit/oak/p...

Author: mreutegg
Date: Tue Aug 20 13:20:19 2013
New Revision: 1515819

URL: http://svn.apache.org/r1515819
Log:
OAK-926: MongoMK: split documents when they are too large
- Make Documents returned by DocumentStore immutable
- Document now implements CacheValue
- Document does not extend from TreeMap anymore
- Enable RootFuzzIT for MongoMK

Modified:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/ClusterNodeInfoDocument.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Collision.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Commit.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Document.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/DocumentStore.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MemoryDocumentStore.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoDocumentStore.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/NodeDocument.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/TimingDocumentStoreWrapper.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/Utils.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/core/RootFuzzIT.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoDocumentStoreTest.java

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/ClusterNodeInfoDocument.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/ClusterNodeInfoDocument.java?rev=1515819&r1=1515818&r2=1515819&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/ClusterNodeInfoDocument.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/ClusterNodeInfoDocument.java Tue Aug 20 13:20:19 2013
@@ -21,5 +21,4 @@ package org.apache.jackrabbit.oak.plugin
  */
 public class ClusterNodeInfoDocument extends Document {
 
-    private static final long serialVersionUID = 4392051250296984718L;
 }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Collision.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Collision.java?rev=1515819&r1=1515818&r2=1515819&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Collision.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Collision.java Tue Aug 20 13:20:19 2013
@@ -76,12 +76,12 @@ class Collision {
         }
         // their commit wins, we have to mark ourRev
         NodeDocument newDoc = Collection.NODES.newDocument();
-        Utils.deepCopyMap(document, newDoc);
+        document.deepCopy(newDoc);
         MemoryDocumentStore.applyChanges(newDoc, ourOp);
         if (!markCommitRoot(newDoc, ourRev, store)) {
             throw new MicroKernelException("Unable to annotate our revision "
                     + "with collision marker. Our revision: " + ourRev
-                    + ", document:\n" + Utils.formatDocument(newDoc));
+                    + ", document:\n" + newDoc.format());
         }
     }
 
@@ -146,9 +146,9 @@ class Collision {
     }
     
     private static void throwNoCommitRootException(@Nonnull String revision,
-                                                   @Nonnull Map<String, Object> document)
+                                                   @Nonnull Document document)
                                                            throws MicroKernelException {
         throw new MicroKernelException("No commit root for revision: "
-                + revision + ", document: " + Utils.formatDocument(document));
+                + revision + ", document: " + document.format());
     }
 }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Commit.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Commit.java?rev=1515819&r1=1515818&r2=1515819&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Commit.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Commit.java Tue Aug 20 13:20:19 2013
@@ -340,7 +340,7 @@ public class Commit {
             }
             if (conflictMessage != null) {
                 conflictMessage += ", before\n" + revision + 
-                        "; document:\n" + Utils.formatDocument(doc) +
+                        "; document:\n" + doc.format() +
                         ",\nrevision order:\n" + mk.getRevisionComparator();
                 throw new MicroKernelException(conflictMessage);
             }
@@ -355,8 +355,7 @@ public class Commit {
             }
         }
 
-        int size = Utils.estimateMemoryUsage(doc);
-        if (doc != null && size > MAX_DOCUMENT_SIZE) {
+        if (doc != null && doc.getMemory() > MAX_DOCUMENT_SIZE) {
             UpdateOp[] split = doc.splitDocument(mk, revision, mk.getSplitDocumentAgeMillis());
             
             // TODO check if the new main document is actually smaller;

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Document.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Document.java?rev=1515819&r1=1515818&r2=1515819&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Document.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Document.java Tue Aug 20 13:20:19 2013
@@ -16,17 +16,22 @@
  */
 package org.apache.jackrabbit.oak.plugins.mongomk;
 
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
 import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import javax.annotation.CheckForNull;
 
+import org.apache.jackrabbit.oak.cache.CacheValue;
+import org.apache.jackrabbit.oak.plugins.mongomk.util.Utils;
+
 /**
  * A document corresponds to a node stored in the MongoMK. A document contains
  * all the revisions of a node stored in the {@link DocumentStore}.
  */
-public class Document extends TreeMap<String, Object> {
-
-    private static final long serialVersionUID = -2428664083360273697L;
+public class Document implements CacheValue {
 
     /**
      * The node id, which contains the depth of the path
@@ -35,10 +40,100 @@ public class Document extends TreeMap<St
     static final String ID = "_id";
 
     /**
+     * The data of this document.
+     */
+    protected Map<String, Object> data = new TreeMap<String, Object>();
+
+    /**
+     * Whether this document is sealed (immutable data).
+     */
+    private AtomicBoolean sealed = new AtomicBoolean(false);
+
+    /**
      * @return the id of this document or <code>null</code> if none is set.
      */
     @CheckForNull
     public String getId() {
         return (String) get(ID);
     }
+
+    /**
+     * Gets the data for the given <code>key</code>.
+     *
+     * @param key the key.
+     * @return the data or <code>null</code>.
+     */
+    @CheckForNull
+    public Object get(String key) {
+        return data.get(key);
+    }
+
+    /**
+     * Sets the data for the given <code>key</code>.
+     *
+     * @param key the key.
+     * @param value the value to set.
+     * @return the previous value or <code>null</code> if there was none.
+     */
+    @CheckForNull
+    public Object put(String key, Object value) {
+        return data.put(key, value);
+    }
+
+    /**
+     * @return a Set view of the keys contained in this document.
+     */
+    public Set<String> keySet() {
+        return data.keySet();
+    }
+
+    /**
+     * Seals this document and turns it into an immutable object. Any attempt
+     * to modify this document afterwards will result in an
+     * {@link UnsupportedOperationException}.
+     */
+    public void seal() {
+        if (!sealed.getAndSet(true)) {
+            data = seal(data);
+        }
+    }
+
+    /**
+     * Performs a deep copy of the data within this document to the given target.
+     *
+     * @param target the target document.
+     */
+    public void deepCopy(Document target) {
+        Utils.deepCopyMap(data, target.data);
+    }
+
+    /**
+     * Formats this document for use in a log message.
+     *
+     * @return the formatted string
+     */
+    public String format() {
+        return data.toString().replaceAll(", _", ",\n_").replaceAll("}, ", "},\n");
+    }
+
+    //-----------------------------< CacheValue >-------------------------------
+
+    @Override
+    public int getMemory() {
+        return Utils.estimateMemoryUsage(this.data);
+    }
+
+    //------------------------------< internal >--------------------------------
+
+    private static Map<String, Object> seal(Map<String, Object> map) {
+        for (Map.Entry<String, Object> entry : map.entrySet()) {
+            Object value = entry.getValue();
+            if (value instanceof Map) {
+                @SuppressWarnings("unchecked")
+                Map<String, Object> childMap = (Map<String, Object>) value;
+                entry.setValue(seal(childMap));
+            }
+        }
+        return Collections.unmodifiableMap(map);
+    }
 }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/DocumentStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/DocumentStore.java?rev=1515819&r1=1515818&r2=1515819&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/DocumentStore.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/DocumentStore.java Tue Aug 20 13:20:19 2013
@@ -17,7 +17,6 @@
 package org.apache.jackrabbit.oak.plugins.mongomk;
 
 import java.util.List;
-import java.util.Map;
 
 import javax.annotation.CheckForNull;
 import javax.annotation.Nonnull;
@@ -32,8 +31,7 @@ public interface DocumentStore {
     /**
      * Get a document.
      * <p>
-     * The returned document is a clone (the caller can modify it without affecting
-     * the stored version).
+     * The returned document is immutable.
      * 
      * @param collection the collection
      * @param key the key
@@ -46,8 +44,7 @@ public interface DocumentStore {
      * Get a document, ignoring the cache if the cached entry is older than the
      * specified time.
      * <p>
-     * The returned document is a clone (the caller can modify it without affecting
-     * the stored version).
+     * The returned document is immutable.
      * 
      * @param collection the collection
      * @param key the key
@@ -59,7 +56,7 @@ public interface DocumentStore {
 
     /**
      * Get a list of documents where the key is greater than a start value and
-     * less than an end value.
+     * less than an end value. The returned documents are immutable.
      * 
      * @param collection the collection
      * @param fromKey the start value (excluding)
@@ -75,7 +72,7 @@ public interface DocumentStore {
     
     /**
      * Get a list of documents where the key is greater than a start value and
-     * less than an end value.
+     * less than an end value. The returned documents are immutable.
      * 
      * @param collection the collection
      * @param fromKey the start value (excluding)
@@ -112,7 +109,7 @@ public interface DocumentStore {
     
     /**
      * Create or update a document. For MongoDb, this is using "findAndModify" with
-     * the "upsert" flag (insert or update).
+     * the "upsert" flag (insert or update). The returned document is immutable.
      *
      * @param collection the collection
      * @param update the update operation
@@ -126,8 +123,8 @@ public interface DocumentStore {
     /**
      * Performs a conditional update (e.g. using
      * {@link UpdateOp.Operation.Type#CONTAINS_MAP_ENTRY} and only updates the
-     * document if the condition is <code>true</code>.
-     *
+     * document if the condition is <code>true</code>. The returned document is
+     * immutable.
      *
      * @param collection the collection
      * @param update the update operation with the condition

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MemoryDocumentStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MemoryDocumentStore.java?rev=1515819&r1=1515818&r2=1515819&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MemoryDocumentStore.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MemoryDocumentStore.java Tue Aug 20 13:20:19 2013
@@ -22,6 +22,9 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import javax.annotation.CheckForNull;
 import javax.annotation.Nonnull;
@@ -48,6 +51,8 @@ public class MemoryDocumentStore impleme
     private ConcurrentSkipListMap<String, Document> clusterNodes =
             new ConcurrentSkipListMap<String, Document>();
 
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+
     @Override
     public <T extends Document> T find(Collection<T> collection, String key, int maxCacheAge) {
         return find(collection, key);
@@ -55,16 +60,14 @@ public class MemoryDocumentStore impleme
     
     @Override
     public <T extends Document> T find(Collection<T> collection, String key) {
-        ConcurrentSkipListMap<String, T> map = getMap(collection);
-        Document doc = map.get(key);
-        if (doc == null) {
-            return null;
-        }
-        T copy = collection.newDocument();
-        synchronized (doc) {
-            Utils.deepCopyMap(doc, copy);
+        Lock lock = rwLock.readLock();
+        lock.lock();
+        try {
+            ConcurrentSkipListMap<String, T> map = getMap(collection);
+            return map.get(key);
+        } finally {
+            lock.unlock();
         }
-        return copy;
     }
     
     @Override
@@ -84,31 +87,52 @@ public class MemoryDocumentStore impleme
                                 String indexedProperty,
                                 long startValue,
                                 int limit) {
-        ConcurrentSkipListMap<String, T> map = getMap(collection);
-        ConcurrentNavigableMap<String, T> sub = map.subMap(fromKey, toKey);
-        ArrayList<T> list = new ArrayList<T>();
-        for (Document doc : sub.values()) {
-            if (indexedProperty != null) {
-                Long value = (Long) doc.get(indexedProperty);
-                if (value < startValue) {
-                    continue;
+        Lock lock = rwLock.readLock();
+        lock.lock();
+        try {
+            ConcurrentSkipListMap<String, T> map = getMap(collection);
+            ConcurrentNavigableMap<String, T> sub = map.subMap(fromKey, toKey);
+            ArrayList<T> list = new ArrayList<T>();
+            for (T doc : sub.values()) {
+                if (indexedProperty != null) {
+                    Long value = (Long) doc.get(indexedProperty);
+                    if (value < startValue) {
+                        continue;
+                    }
+                }
+                list.add(doc);
+                if (list.size() >= limit) {
+                    break;
                 }
             }
-            T copy = collection.newDocument();
-            synchronized (doc) {
-                Utils.deepCopyMap(doc, copy);
-            }
-            list.add(copy);
-            if (list.size() >= limit) {
-                break;
-            }
+            return list;
+        } finally {
+            lock.unlock();
         }
-        return list;
     }
 
     @Override
     public void remove(Collection collection, String path) {
-        getMap(collection).remove(path);
+        Lock lock = rwLock.writeLock();
+        lock.lock();
+        try {
+            getMap(collection).remove(path);
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    @CheckForNull
+    @Override
+    public <T extends Document> T createOrUpdate(Collection<T> collection, UpdateOp update)
+            throws MicroKernelException {
+        return internalCreateOrUpdate(collection, update, false);
+    }
+
+    @Override
+    public <T extends Document> T findAndUpdate(Collection<T> collection, UpdateOp update)
+            throws MicroKernelException {
+        return internalCreateOrUpdate(collection, update, true);
     }
 
     /**
@@ -134,64 +158,43 @@ public class MemoryDocumentStore impleme
                                                           UpdateOp update,
                                                           boolean checkConditions) {
         ConcurrentSkipListMap<String, T> map = getMap(collection);
-        T doc;
         T oldDoc;
 
-        // get the node if it's there
-        oldDoc = doc = map.get(update.key);
-
-        if (doc == null) {
-            if (!update.isNew) {
-                throw new MicroKernelException("Document does not exist: " + update.key);
-            }
-            // for a new node, add it (without synchronization)
-            doc = collection.newDocument();
-            oldDoc = map.putIfAbsent(update.key, doc);
-            if (oldDoc != null) {
-                // somebody else added it at the same time
-                doc = oldDoc;
+        Lock lock = rwLock.writeLock();
+        lock.lock();
+        try {
+            // get the node if it's there
+            oldDoc = map.get(update.key);
+
+            T doc = collection.newDocument();
+            if (oldDoc == null) {
+                if (!update.isNew) {
+                    throw new MicroKernelException("Document does not exist: " + update.key);
+                }
+            } else {
+                oldDoc.deepCopy(doc);
             }
-        }
-        synchronized (doc) {
             if (checkConditions && !checkConditions(doc, update)) {
                 return null;
             }
-            if (oldDoc != null) {
-                // clone the old node
-                // (document level operations are synchronized)
-                T oldDoc2 = collection.newDocument();
-                Utils.deepCopyMap(oldDoc, oldDoc2);
-                oldDoc = oldDoc2;
-            }
-            // to return the new document:
             // update the document
-            // (document level operations are synchronized)
             applyChanges(doc, update);
+            doc.seal();
+            map.put(update.key, doc);
+            return oldDoc;
+        } finally {
+            lock.unlock();
         }
-        return oldDoc;
-    }
-
-    @CheckForNull
-    @Override
-    public <T extends Document> T createOrUpdate(Collection<T> collection, UpdateOp update)
-            throws MicroKernelException {
-        return internalCreateOrUpdate(collection, update, false);
     }
 
-    @Override
-    public <T extends Document> T findAndUpdate(Collection<T> collection, UpdateOp update)
-            throws MicroKernelException {
-        return internalCreateOrUpdate(collection, update, true);
-    }
-
-    private static boolean checkConditions(Map<String, Object> target,
+    private static boolean checkConditions(Document doc,
                                            UpdateOp update) {
         for (Map.Entry<String, Operation> change : update.changes.entrySet()) {
             Operation op = change.getValue();
             if (op.type == Operation.Type.CONTAINS_MAP_ENTRY) {
                 String k = change.getKey();
                 String[] kv = k.split("\\.");
-                Object value = target.get(kv[0]);
+                Object value = doc.get(kv[0]);
                 if (value == null) {
                     if (Boolean.TRUE.equals(op.value)) {
                         return false;
@@ -219,44 +222,44 @@ public class MemoryDocumentStore impleme
 
 
     /**
-     * Apply the changes to the in-memory map.
+     * Apply the changes to the in-memory document.
      * 
-     * @param target the target map
+     * @param doc the target document.
      * @param update the changes to apply
      */
-    public static void applyChanges(Map<String, Object> target, UpdateOp update) {
+    public static void applyChanges(Document doc, UpdateOp update) {
         for (Entry<String, Operation> e : update.changes.entrySet()) {
             String k = e.getKey();
             Operation op = e.getValue();
             switch (op.type) {
             case SET: {
-                target.put(k, op.value);
+                doc.put(k, op.value);
                 break;
             }
             case INCREMENT: {
-                Object old = target.get(k);
+                Object old = doc.get(k);
                 Long x = (Long) op.value;
                 if (old == null) {
                     old = 0L;
                 }
-                target.put(k, ((Long) old) + x);
+                doc.put(k, ((Long) old) + x);
                 break;
             }
             case SET_MAP_ENTRY: {
                 String[] kv = splitInTwo(k, '.');
-                Object old = target.get(kv[0]);
+                Object old = doc.get(kv[0]);
                 @SuppressWarnings("unchecked")
                 Map<String, Object> m = (Map<String, Object>) old;
                 if (m == null) {
                     m = Utils.newMap();
-                    target.put(kv[0], m);
+                    doc.put(kv[0], m);
                 }
                 m.put(kv[1], op.value);
                 break;
             }
             case REMOVE_MAP_ENTRY: {
                 String[] kv = splitInTwo(k, '.');
-                Object old = target.get(kv[0]);
+                Object old = doc.get(kv[0]);
                 @SuppressWarnings("unchecked")
                 Map<String, Object> m = (Map<String, Object>) old;
                 if (m != null) {
@@ -266,12 +269,12 @@ public class MemoryDocumentStore impleme
             }
             case SET_MAP: {
                 String[] kv = splitInTwo(k, '.');
-                Object old = target.get(kv[0]);
+                Object old = doc.get(kv[0]);
                 @SuppressWarnings("unchecked")
                 Map<String, Object> m = (Map<String, Object>) old;
                 if (m == null) {
                     m = Utils.newMap();
-                    target.put(kv[0], m);
+                    doc.put(kv[0], m);
                 }
                 m.put(kv[1], op.value);
                 break;
@@ -292,17 +295,24 @@ public class MemoryDocumentStore impleme
     }
 
     @Override
-    public <T extends Document> boolean create(Collection<T> collection, List<UpdateOp> updateOps) {
-        ConcurrentSkipListMap<String, T> map = getMap(collection);
-        for (UpdateOp op : updateOps) {
-            if (map.containsKey(op.key)) {
-                return false;
+    public <T extends Document> boolean create(Collection<T> collection,
+                                               List<UpdateOp> updateOps) {
+        Lock lock = rwLock.writeLock();
+        lock.lock();
+        try {
+            ConcurrentSkipListMap<String, T> map = getMap(collection);
+            for (UpdateOp op : updateOps) {
+                if (map.containsKey(op.key)) {
+                    return false;
+                }
             }
+            for (UpdateOp op : updateOps) {
+                internalCreateOrUpdate(collection, op, false);
+            }
+            return true;
+        } finally {
+            lock.unlock();
         }
-        for (UpdateOp op : updateOps) {
-            createOrUpdate(collection, op);
-        }
-        return true;
     }
 
     @Override
@@ -311,9 +321,9 @@ public class MemoryDocumentStore impleme
         buff.append("Nodes:\n");
         for (String p : nodes.keySet()) {
             buff.append("Path: ").append(p).append('\n');
-            Map<String, Object> e = nodes.get(p);
-            for (String prop : e.keySet()) {
-                buff.append(prop).append('=').append(e.get(prop)).append('\n');
+            NodeDocument doc = nodes.get(p);
+            for (String prop : doc.keySet()) {
+                buff.append(prop).append('=').append(doc.get(prop)).append('\n');
             }
             buff.append("\n");
         }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoDocumentStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoDocumentStore.java?rev=1515819&r1=1515818&r2=1515819&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoDocumentStore.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoDocumentStore.java Tue Aug 20 13:20:19 2013
@@ -28,7 +28,6 @@ import javax.annotation.Nonnull;
 
 import org.apache.jackrabbit.mk.api.MicroKernelException;
 import org.apache.jackrabbit.oak.plugins.mongomk.UpdateOp.Operation;
-import org.apache.jackrabbit.oak.plugins.mongomk.util.Utils;
 import org.apache.jackrabbit.oak.cache.CacheStats;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -172,11 +171,13 @@ public class MongoDocumentStore implemen
         DBCollection dbCollection = getDBCollection(collection);
         long start = start();
         try {
-            DBObject doc = dbCollection.findOne(getByKeyQuery(key).get());
-            if (doc == null) {
+            DBObject obj = dbCollection.findOne(getByKeyQuery(key).get());
+            if (obj == null) {
                 return null;
             }
-            return convertFromDBObject(collection, doc);
+            T doc = convertFromDBObject(collection, obj);
+            doc.seal();
+            return doc;
         } finally {
             end("findUncached", start);
         }
@@ -216,6 +217,7 @@ public class MongoDocumentStore implemen
             for (int i = 0; i < limit && cursor.hasNext(); i++) {
                 DBObject o = cursor.next();
                 T doc = convertFromDBObject(collection, o);
+                doc.seal();
                 if (collection == Collection.NODES) {
                     nodesCache.put(doc.getId(), (NodeDocument) doc);
                 }
@@ -325,12 +327,13 @@ public class MongoDocumentStore implemen
             // cache the new document
             if (collection == Collection.NODES) {
                 T newDoc = collection.newDocument();
-                Utils.deepCopyMap(doc, newDoc);
+                doc.deepCopy(newDoc);
                 String key = updateOp.getKey();
                 MemoryDocumentStore.applyChanges(newDoc, updateOp);
+                newDoc.seal();
                 nodesCache.put(key, (NodeDocument) newDoc);
             }
-            
+            doc.seal();
             return doc;
         } catch (Exception e) {
             throw new MicroKernelException(e);
@@ -406,6 +409,7 @@ public class MongoDocumentStore implemen
                 }
                 if (collection == Collection.NODES) {
                     for (T doc : docs) {
+                        doc.seal();
                         nodesCache.put(doc.getId(), (NodeDocument) doc);
                     }
                 }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/NodeDocument.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/NodeDocument.java?rev=1515819&r1=1515818&r2=1515819&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/NodeDocument.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/NodeDocument.java Tue Aug 20 13:20:19 2013
@@ -42,12 +42,10 @@ import static com.google.common.base.Pre
 /**
  * A document storing data about a node.
  */
-public class NodeDocument extends Document implements CacheValue {
+public class NodeDocument extends Document {
 
     private static final Logger log = LoggerFactory.getLogger(NodeDocument.class);
 
-    private static final long serialVersionUID = 6713219541688419314L;
-
     /**
      * Marker document, which indicates the document does not exist.
      */
@@ -169,10 +167,10 @@ public class NodeDocument extends Docume
                                       Revision changeRev,
                                       CollisionHandler handler) {
         SortedSet<String> revisions = new TreeSet<String>(Collections.reverseOrder());
-        if (containsKey(REVISIONS)) {
+        if (data.containsKey(REVISIONS)) {
             revisions.addAll(((Map<String, String>) get(REVISIONS)).keySet());
         }
-        if (containsKey(COMMIT_ROOT)) {
+        if (data.containsKey(COMMIT_ROOT)) {
             revisions.addAll(((Map<String, Integer>) get(COMMIT_ROOT)).keySet());
         }
         Map<String, String> deletedMap = (Map<String, String>)get(DELETED);
@@ -455,7 +453,7 @@ public class NodeDocument extends Docume
         UpdateOp main = new UpdateOp(path, id, false);
         setModified(main, commitRevision);
         main.set(NodeDocument.PREVIOUS, previous);
-        for (Map.Entry<String, Object> e : entrySet()) {
+        for (Map.Entry<String, Object> e : data.entrySet()) {
             String key = e.getKey();
             if (key.equals(Document.ID)) {
                 // ok
@@ -507,13 +505,6 @@ public class NodeDocument extends Docume
         op.set(MODIFIED, Commit.getModified(revision.getTimestamp()));
     }
 
-    //-----------------------------< CacheValue >-------------------------------
-
-    @Override
-    public int getMemory() {
-        return Utils.estimateMemoryUsage(this);
-    }
-
     //----------------------------< internal >----------------------------------
 
     /**

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/TimingDocumentStoreWrapper.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/TimingDocumentStoreWrapper.java?rev=1515819&r1=1515818&r2=1515819&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/TimingDocumentStoreWrapper.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/TimingDocumentStoreWrapper.java Tue Aug 20 13:20:19 2013
@@ -80,7 +80,7 @@ public class TimingDocumentStoreWrapper 
         try {
             long start = now();
             T result = base.find(collection, key);
-            updateAndLogTimes("find", start, 0, size(result));
+            updateAndLogTimes("find", start, 0, result.getMemory());
             return result;
         } catch (Exception e) {
             throw convert(e);
@@ -93,7 +93,7 @@ public class TimingDocumentStoreWrapper 
         try {
             long start = now();
             T result = base.find(collection, key, maxCacheAge);
-            updateAndLogTimes("find2", start, 0, size(result));
+            updateAndLogTimes("find2", start, 0, result.getMemory());
             return result;
         } catch (Exception e) {
             throw convert(e);
@@ -164,7 +164,7 @@ public class TimingDocumentStoreWrapper 
         try {
             long start = now();
             T result = base.createOrUpdate(collection, update);
-            updateAndLogTimes("createOrUpdate", start, 0, size(result));
+            updateAndLogTimes("createOrUpdate", start, 0, result.getMemory());
             return result;
         } catch (Exception e) {
             throw convert(e);
@@ -178,7 +178,7 @@ public class TimingDocumentStoreWrapper 
         try {
             long start = now();
             T result = base.findAndUpdate(collection, update);
-            updateAndLogTimes("findAndUpdate", start, 0, size(result));
+            updateAndLogTimes("findAndUpdate", start, 0, result.getMemory());
             return result;
         } catch (Exception e) {
             throw convert(e);
@@ -243,14 +243,10 @@ public class TimingDocumentStoreWrapper 
         }
     }
     
-    private static <T extends Document> int size(T doc) {
-        return Utils.estimateMemoryUsage(doc);
-    }
-
     private static <T extends Document> int size(List<T> list) {
         int result = 0;
         for (T doc : list) {
-            result += size(doc);
+            result += doc.getMemory();
         }
         return result;
     }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/Utils.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/Utils.java?rev=1515819&r1=1515818&r2=1515819&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/Utils.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/Utils.java Tue Aug 20 13:20:19 2013
@@ -202,16 +202,6 @@ public class Utils {
     }
     
     /**
-     * Formats a MongoDB document for use in a log message.
-     * 
-     * @param document the MongoDB document.
-     * @return the formatted string
-     */
-    public static String formatDocument(Map<String, Object> document) {
-        return document.toString().replaceAll(", _", ",\n_").replaceAll("}, ", "},\n");
-    }
-
-    /**
      * Returns the lower key limit to retrieve the children of the given
      * <code>path</code>.
      *

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/core/RootFuzzIT.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/core/RootFuzzIT.java?rev=1515819&r1=1515818&r2=1515819&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/core/RootFuzzIT.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/core/RootFuzzIT.java Tue Aug 20 13:20:19 2013
@@ -126,9 +126,6 @@ public class RootFuzzIT {
 
     @Test
     public void fuzzTest() throws Exception {
-        // FIXME takes too long on MongoMK. See OAK-926
-        assumeTrue(fixture != NodeStoreFixture.MONGO_MK || EXECUTE_MONGO_MK);
-
         // FIXME fails on SegmentMK. See OAK-965
         assumeTrue(fixture != NodeStoreFixture.SEGMENT_MK || EXECUTE_SEGMENT_MK);
 

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoDocumentStoreTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoDocumentStoreTest.java?rev=1515819&r1=1515818&r2=1515819&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoDocumentStoreTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoDocumentStoreTest.java Tue Aug 20 13:20:19 2013
@@ -82,21 +82,21 @@ public class MongoDocumentStoreTest {
         updateOp.increment("property2", 1);
         updateOp.set("property3", "value3");
         docStore.createOrUpdate(Collection.NODES, updateOp);
-        Map<String, Object> obj = docStore.find(Collection.NODES, "/");
+        NodeDocument doc = docStore.find(Collection.NODES, "/");
 
-        Map<?, ?> property1 = (Map<?, ?>) obj.get("property1");
+        Map<?, ?> property1 = (Map<?, ?>) doc.get("property1");
         String value1 = (String) property1.get("key1");
         assertEquals("value1", value1);
 
-        Long value2 = (Long) obj.get("property2");
+        Long value2 = (Long) doc.get("property2");
         assertEquals(Long.valueOf(1), value2);
 
-        String value3 = (String) obj.get("property3");
+        String value3 = (String) doc.get("property3");
         assertEquals("value3", value3);
 
         docStore.remove(Collection.NODES, "/");
-        obj = docStore.find(Collection.NODES, "/");
-        assertTrue(obj == null);
+        doc = docStore.find(Collection.NODES, "/");
+        assertTrue(doc == null);
     }
 
     @Test
@@ -158,11 +158,11 @@ public class MongoDocumentStoreTest {
         op.set("prop", "value");
         op.containsMapEntry("map", "key", true);
         // update if key exists -> must succeed
-        Map<String, Object> doc = docStore.findAndUpdate(Collection.NODES, op);
+        NodeDocument doc = docStore.findAndUpdate(Collection.NODES, op);
         assertNotNull(doc);
 
         doc = docStore.find(Collection.NODES, "/node");
-        assertTrue(doc.containsKey("prop"));
+        assertNotNull(doc.get("prop"));
         assertEquals("value", doc.get("prop"));
 
         op = new UpdateOp("/node", "/node", false);
@@ -173,7 +173,7 @@ public class MongoDocumentStoreTest {
 
         // value must still be the same
         doc = docStore.find(Collection.NODES, "/node");
-        assertTrue(doc.containsKey("prop"));
+        assertNotNull(doc.get("prop"));
         assertEquals("value", doc.get("prop"));
     }