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/14 15:22:05 UTC

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

Author: mreutegg
Date: Wed Aug 14 13:22:05 2013
New Revision: 1513869

URL: http://svn.apache.org/r1513869
Log:
OAK-926: MongoMK: split documents when they are too large
- Move generic ID to Document and introduce getId()

Modified:
    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/MongoDocumentStore.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoMK.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Node.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/UpdateOp.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/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=1513869&r1=1513868&r2=1513869&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 Wed Aug 14 13:22:05 2013
@@ -46,12 +46,12 @@ class Collision {
 
     private static final Logger LOG = LoggerFactory.getLogger(Collision.class);
 
-    private final Map<String, Object> document;
+    private final Document document;
     private final String theirRev;
     private final UpdateOp ourOp;
     private final String ourRev;
 
-    Collision(@Nonnull Map<String, Object> document,
+    Collision(@Nonnull Document document,
               @Nonnull Revision theirRev,
               @Nonnull UpdateOp ourOp,
               @Nonnull Revision ourRev) {
@@ -75,7 +75,7 @@ class Collision {
             return;
         }
         // their commit wins, we have to mark ourRev
-        Map<String, Object> newDoc = Utils.newMap();
+        Document newDoc = Utils.newDocument();
         Utils.deepCopyMap(document, newDoc);
         MemoryDocumentStore.applyChanges(newDoc, ourOp);
         if (!markCommitRoot(newDoc, ourRev, store)) {
@@ -96,10 +96,10 @@ class Collision {
      * @return <code>true</code> if the commit for the given revision was marked
      *         successfully; <code>false</code> otherwise.
      */
-    private static boolean markCommitRoot(@Nonnull Map<String, Object> document,
+    private static boolean markCommitRoot(@Nonnull Document document,
                                           @Nonnull String revision,
                                           @Nonnull DocumentStore store) {
-        String p = Utils.getPathFromId((String) document.get(UpdateOp.ID));
+        String p = Utils.getPathFromId(document.getId());
         String commitRootPath = null;
         // first check if we can mark the commit with the given revision
         @SuppressWarnings("unchecked")

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=1513869&r1=1513868&r2=1513869&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 Wed Aug 14 13:22:05 2013
@@ -283,10 +283,10 @@ public class Commit {
      * @param op the operation
      */
     public void createOrUpdateNode(DocumentStore store, UpdateOp op) {
-        Map<String, Object> map = store.createOrUpdate(Collection.NODES, op);
+        Document doc = store.createOrUpdate(Collection.NODES, op);
         if (baseRevision != null) {
             final AtomicReference<List<Revision>> collisions = new AtomicReference<List<Revision>>();
-            Revision newestRev = mk.getNewestRevision(map, revision,
+            Revision newestRev = mk.getNewestRevision(doc, revision,
                     new CollisionHandler() {
                 @Override
                 void concurrentModification(Revision other) {
@@ -308,7 +308,7 @@ public class Commit {
                             op.path + " was already added in revision\n" + 
                             newestRev;
                 } else if (mk.isRevisionNewer(newestRev, baseRevision)
-                        && (op.isDelete || isConflicting(map, op))) {
+                        && (op.isDelete || isConflicting(doc, op))) {
                     conflictMessage = "The node " + 
                             op.path + " was changed in revision\n" + newestRev +
                             ", which was applied after the base revision\n" + 
@@ -317,24 +317,24 @@ public class Commit {
             }
             if (conflictMessage != null) {
                 conflictMessage += ", before\n" + revision + 
-                        "; document:\n" + Utils.formatDocument(map) + 
+                        "; document:\n" + Utils.formatDocument(doc) +
                         ",\nrevision order:\n" + mk.getRevisionComparator();
                 throw new MicroKernelException(conflictMessage);
             }
             // if we get here the modification was successful
             // -> check for collisions and conflict (concurrent updates
             // on a node are possible if property updates do not overlap)
-            if (collisions.get() != null && isConflicting(map, op)) {
+            if (collisions.get() != null && isConflicting(doc, op)) {
                 for (Revision r : collisions.get()) {
                     // mark collisions on commit root
-                    new Collision(map, r, op, revision).mark(store);
+                    new Collision(doc, r, op, revision).mark(store);
                 }
             }
         }
 
-        int size = Utils.estimateMemoryUsage(map);
+        int size = Utils.estimateMemoryUsage(doc);
         if (size > MAX_DOCUMENT_SIZE) {
-            UpdateOp[] split = splitDocument(map);
+            UpdateOp[] split = splitDocument(doc);
             
             // TODO check if the new main document is actually smaller;
             // otherwise, splitting doesn't make sense
@@ -411,10 +411,10 @@ public class Commit {
         return false;
     }
 
-    private UpdateOp[] splitDocument(Map<String, Object> map) {
-        String id = (String) map.get(UpdateOp.ID);
+    private UpdateOp[] splitDocument(Document doc) {
+        String id = doc.getId();
         String path = Utils.getPathFromId(id);
-        Long previous = (Long) map.get(UpdateOp.PREVIOUS);
+        Long previous = (Long) doc.get(UpdateOp.PREVIOUS);
         if (previous == null) {
             previous = 0L;
         } else {
@@ -425,9 +425,9 @@ public class Commit {
         UpdateOp main = new UpdateOp(path, id, false);
         setModified(main, revision);
         main.set(UpdateOp.PREVIOUS, previous);
-        for (Entry<String, Object> e : map.entrySet()) {
+        for (Entry<String, Object> e : doc.entrySet()) {
             String key = e.getKey();
-            if (key.equals(UpdateOp.ID)) {
+            if (key.equals(Document.ID)) {
                 // ok
             } else if (key.equals(UpdateOp.MODIFIED)) {
                 // ok

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=1513869&r1=1513868&r2=1513869&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 Wed Aug 14 13:22:05 2013
@@ -18,6 +18,8 @@ package org.apache.jackrabbit.oak.plugin
 
 import java.util.TreeMap;
 
+import javax.annotation.CheckForNull;
+
 /**
  * A document corresponds to a node stored in the MongoMK. A document contains
  * all the revisions of a node stored in the {@link DocumentStore}.
@@ -26,4 +28,17 @@ public class Document extends TreeMap<St
 
     private static final long serialVersionUID = -2428664083360273697L;
 
+    /**
+     * The node id, which contains the depth of the path
+     * (0 for root, 1 for children of the root), and then the path.
+     */
+    static final String ID = "_id";
+
+    /**
+     * @return the id of this document or <code>null</code> if none is set.
+     */
+    @CheckForNull
+    public String getId() {
+        return (String) get(ID);
+    }
 }

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=1513869&r1=1513868&r2=1513869&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 Wed Aug 14 13:22:05 2013
@@ -194,7 +194,7 @@ public class MongoDocumentStore implemen
                                 int limit) {
         log("query", fromKey, toKey, limit);
         DBCollection dbCollection = getDBCollection(collection);
-        QueryBuilder queryBuilder = QueryBuilder.start(UpdateOp.ID);
+        QueryBuilder queryBuilder = QueryBuilder.start(Document.ID);
         queryBuilder.greaterThanEquals(fromKey);
         queryBuilder.lessThan(toKey);
         if (indexedProperty != null) {
@@ -210,8 +210,7 @@ public class MongoDocumentStore implemen
                 DBObject o = cursor.next();
                 Document doc = convertFromDBObject(o);
                 if (collection == Collection.NODES) {
-                    String key = (String) doc.get(UpdateOp.ID);
-                    nodesCache.put(key, new CachedDocument(doc));
+                    nodesCache.put(doc.getId(), new CachedDocument(doc));
                 }
                 list.add(doc);
             }
@@ -253,7 +252,7 @@ public class MongoDocumentStore implemen
 
         for (Entry<String, Operation> entry : updateOp.changes.entrySet()) {
             String k = entry.getKey();
-            if (k.equals(UpdateOp.ID)) {
+            if (k.equals(Document.ID)) {
                 // avoid exception "Mod on _id not allowed"
                 continue;
             }
@@ -400,8 +399,7 @@ public class MongoDocumentStore implemen
                 }
                 if (collection == Collection.NODES) {
                     for (Document doc : docs) {
-                        String id = (String) doc.get(UpdateOp.ID);
-                        nodesCache.put(id, new CachedDocument(doc));
+                        nodesCache.put(doc.getId(), new CachedDocument(doc));
                     }
                 }
                 return true;
@@ -442,7 +440,7 @@ public class MongoDocumentStore implemen
     }
 
     private static QueryBuilder getByKeyQuery(String key) {
-        return QueryBuilder.start(UpdateOp.ID).is(key);
+        return QueryBuilder.start(Document.ID).is(key);
     }
     
     @Override

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoMK.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoMK.java?rev=1513869&r1=1513868&r2=1513869&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoMK.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoMK.java Wed Aug 14 13:22:05 2013
@@ -504,10 +504,10 @@ public class MongoMK implements MicroKer
     }
 
     /**
-     * Checks if the revision is valid for the given node map. A revision is
-     * considered valid if the given node map is the root of the commit, or the
-     * commit root has the revision set. This method may read further nodes to
-     * perform this check.
+     * Checks if the revision is valid for the given document. A revision is
+     * considered valid if the given document is the root of the commit, or the
+     * commit root has the revision set. This method may read further documents
+     * to perform this check.
      * This method also takes pending branches into consideration.
      * The <code>readRevision</code> identifies the read revision used by the
      * client, which may be a branch revision logged in {@link #branches}.
@@ -516,7 +516,7 @@ public class MongoMK implements MicroKer
      *
      * @param rev     revision to check.
      * @param readRevision the read revision of the client.
-     * @param nodeMap the node to check.
+     * @param doc the document to check.
      * @param validRevisions set of revisions already checked against
      *                       <code>readRevision</code> and considered valid.
      * @return <code>true</code> if the revision is valid; <code>false</code>
@@ -524,13 +524,13 @@ public class MongoMK implements MicroKer
      */
     boolean isValidRevision(@Nonnull Revision rev,
                             @Nonnull Revision readRevision,
-                            @Nonnull Map<String, Object> nodeMap,
+                            @Nonnull Document doc,
                             @Nonnull Set<Revision> validRevisions) {
         if (validRevisions.contains(rev)) {
             return true;
         }
         @SuppressWarnings("unchecked")
-        Map<String, String> revisions = (Map<String, String>) nodeMap.get(UpdateOp.REVISIONS);
+        Map<String, String> revisions = (Map<String, String>) doc.get(UpdateOp.REVISIONS);
         if (isCommitted(rev, readRevision, revisions)) {
             validRevisions.add(rev);
             return true;
@@ -541,12 +541,12 @@ public class MongoMK implements MicroKer
         }
         // check commit root
         @SuppressWarnings("unchecked")
-        Map<String, Integer> commitRoot = (Map<String, Integer>) nodeMap.get(UpdateOp.COMMIT_ROOT);
+        Map<String, Integer> commitRoot = (Map<String, Integer>) doc.get(UpdateOp.COMMIT_ROOT);
         String commitRootPath = null;
         if (commitRoot != null) {
             Integer depth = commitRoot.get(rev.toString());
             if (depth != null) {
-                String p = Utils.getPathFromId((String) nodeMap.get(UpdateOp.ID));
+                String p = Utils.getPathFromId(doc.getId());
                 commitRootPath = PathUtils.getAncestorPath(p, PathUtils.getDepth(p) - depth);
             }
         }
@@ -554,18 +554,18 @@ public class MongoMK implements MicroKer
             // shouldn't happen, either node is commit root for a revision
             // or has a reference to the commit root
             LOG.warn("Node {} does not have commit root reference for revision {}",
-                    nodeMap.get(UpdateOp.ID), rev);
-            LOG.warn(nodeMap.toString());
+                    doc.getId(), rev);
+            LOG.warn(doc.toString());
             return false;
         }
         // get root of commit
-        nodeMap = store.find(DocumentStore.Collection.NODES, 
+        doc = store.find(DocumentStore.Collection.NODES,
                 Utils.getIdFromPath(commitRootPath));
-        if (nodeMap == null) {
+        if (doc == null) {
             return false;
         }
         @SuppressWarnings("unchecked")
-        Map<String, String> rootRevisions = (Map<String, String>) nodeMap.get(UpdateOp.REVISIONS);
+        Map<String, String> rootRevisions = (Map<String, String>) doc.get(UpdateOp.REVISIONS);
         if (isCommitted(rev, readRevision, rootRevisions)) {
             validRevisions.add(rev);
             return true;
@@ -673,8 +673,7 @@ public class MongoMK implements MicroKer
                 continue;
             }
             // TODO put the whole node in the cache
-            String id = doc.get(UpdateOp.ID).toString();
-            String p = Utils.getPathFromId(id);
+            String p = Utils.getPathFromId(doc.getId());
             c.children.add(p);
         }
         return c;
@@ -682,21 +681,21 @@ public class MongoMK implements MicroKer
 
     private Node readNode(String path, Revision readRevision) {
         String id = Utils.getIdFromPath(path);
-        Map<String, Object> map = store.find(DocumentStore.Collection.NODES, id);
-        if (map == null) {
+        Document doc = store.find(DocumentStore.Collection.NODES, id);
+        if (doc == null) {
             return null;
         }
-        Revision min = getLiveRevision(map, readRevision);
+        Revision min = getLiveRevision(doc, readRevision);
         if (min == null) {
             // deleted
             return null;
         }
         Node n = new Node(path, readRevision);
-        for (String key : map.keySet()) {
+        for (String key : doc.keySet()) {
             if (!Utils.isPropertyName(key)) {
                 continue;
             }
-            Object v = map.get(key);
+            Object v = doc.get(key);
             @SuppressWarnings("unchecked")
             Map<String, String> valueMap = (Map<String, String>) v;
             if (valueMap != null) {
@@ -716,7 +715,7 @@ public class MongoMK implements MicroKer
         Revision lastRevision = null;
         Map<Integer, Revision> lastRevs = new HashMap<Integer, Revision>();
         @SuppressWarnings("unchecked")
-        Map<String, String> valueMap = (Map<String, String>) map.get(UpdateOp.LAST_REV);
+        Map<String, String> valueMap = (Map<String, String>) doc.get(UpdateOp.LAST_REV);
         if (valueMap != null) {
             for (String clusterId : valueMap.keySet()) {
                 lastRevs.put(Integer.parseInt(clusterId),
@@ -920,7 +919,7 @@ public class MongoMK implements MicroKer
         List<Document> list = store.query(DocumentStore.Collection.NODES, fromKey, toKey,
                 UpdateOp.MODIFIED, minValue, Integer.MAX_VALUE);
         for (Document doc : list) {
-            String id = doc.get(UpdateOp.ID).toString();
+            String id = doc.getId();
             String p = Utils.getPathFromId(id);
             Node fromNode = getNode(p, fromRev);
             Node toNode = getNode(p, toRev);
@@ -1227,31 +1226,31 @@ public class MongoMK implements MicroKer
      * Get the earliest (oldest) revision where the node was alive at or before
      * the provided revision, if the node was alive at the given revision.
      * 
-     * @param nodeMap the node map
+     * @param doc the document
      * @param maxRev the maximum revision to return
      * @return the earliest revision, or null if the node is deleted at the
      *         given revision
      */
-    private Revision getLiveRevision(Map<String, Object> nodeMap,
+    private Revision getLiveRevision(Document doc,
                                      Revision maxRev) {
-        return getLiveRevision(nodeMap, maxRev, new HashSet<Revision>());
+        return getLiveRevision(doc, maxRev, new HashSet<Revision>());
     }
 
     /**
      * Get the earliest (oldest) revision where the node was alive at or before
      * the provided revision, if the node was alive at the given revision.
      * 
-     * @param nodeMap the node map
+     * @param doc the document
      * @param maxRev the maximum revision to return
      * @param validRevisions the set of revisions already checked against maxRev
      *            and considered valid.
      * @return the earliest revision, or null if the node is deleted at the
      *         given revision
      */
-    private Revision getLiveRevision(Map<String, Object> nodeMap,
+    private Revision getLiveRevision(Document doc,
             Revision maxRev, Set<Revision> validRevisions) {
         @SuppressWarnings("unchecked")
-        Map<String, String> valueMap = (Map<String, String>) nodeMap
+        Map<String, String> valueMap = (Map<String, String>) doc
                 .get(UpdateOp.DELETED);
         if (valueMap == null) {
             return null;
@@ -1271,7 +1270,7 @@ public class MongoMK implements MicroKer
             }
             Revision propRev = Revision.fromString(r);
             if (isRevisionNewer(propRev, maxRev)
-                    || !isValidRevision(propRev, maxRev, nodeMap, validRevisions)) {
+                    || !isValidRevision(propRev, maxRev, doc, validRevisions)) {
                 continue;
             }
             if (deletedRev == null || isRevisionNewer(propRev, deletedRev)) {
@@ -1293,7 +1292,7 @@ public class MongoMK implements MicroKer
                 continue;
             }
             if (isRevisionNewer(propRev, maxRev)
-                    || !isValidRevision(propRev, maxRev, nodeMap, validRevisions)) {
+                    || !isValidRevision(propRev, maxRev, doc, validRevisions)) {
                 continue;
             }
             if (liveRev == null || isRevisionNewer(liveRev, propRev)) {
@@ -1306,26 +1305,26 @@ public class MongoMK implements MicroKer
     /**
      * Get the revision of the latest change made to this node.
      * 
-     * @param nodeMap the document
+     * @param doc the document
      * @param changeRev the revision of the current change
      * @param handler the conflict handler, which is called for concurrent changes
      *                preceding <code>before</code>.
      * @return the revision, or null if deleted
      */
     @SuppressWarnings("unchecked")
-    @Nullable Revision getNewestRevision(Map<String, Object> nodeMap,
+    @Nullable Revision getNewestRevision(Document doc,
                                          Revision changeRev, CollisionHandler handler) {
-        if (nodeMap == null) {
+        if (doc == null) {
             return null;
         }
         SortedSet<String> revisions = new TreeSet<String>(Collections.reverseOrder());
-        if (nodeMap.containsKey(UpdateOp.REVISIONS)) {
-            revisions.addAll(((Map<String, String>) nodeMap.get(UpdateOp.REVISIONS)).keySet());
+        if (doc.containsKey(UpdateOp.REVISIONS)) {
+            revisions.addAll(((Map<String, String>) doc.get(UpdateOp.REVISIONS)).keySet());
         }
-        if (nodeMap.containsKey(UpdateOp.COMMIT_ROOT)) {
-            revisions.addAll(((Map<String, Integer>) nodeMap.get(UpdateOp.COMMIT_ROOT)).keySet());
+        if (doc.containsKey(UpdateOp.COMMIT_ROOT)) {
+            revisions.addAll(((Map<String, Integer>) doc.get(UpdateOp.COMMIT_ROOT)).keySet());
         }
-        Map<String, String> deletedMap = (Map<String, String>) nodeMap
+        Map<String, String> deletedMap = (Map<String, String>) doc
                 .get(UpdateOp.DELETED);
         if (deletedMap != null) {
             revisions.addAll(deletedMap.keySet());
@@ -1343,7 +1342,7 @@ public class MongoMK implements MicroKer
             if (newestRev == null || isRevisionNewer(propRev, newestRev)) {
                 if (!propRev.equals(changeRev)) {
                     if (!isValidRevision(
-                            propRev, changeRev, nodeMap, new HashSet<Revision>())) {
+                            propRev, changeRev, doc, new HashSet<Revision>())) {
                         handler.concurrentModification(propRev);
                     } else {
                         newestRev = propRev;

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Node.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Node.java?rev=1513869&r1=1513868&r2=1513869&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Node.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/Node.java Wed Aug 14 13:22:05 2013
@@ -76,7 +76,7 @@ public class Node implements CacheValue 
     UpdateOp asOperation(boolean isNew) {
         String id = Utils.getIdFromPath(path);
         UpdateOp op = new UpdateOp(path, id, isNew);
-        op.set(UpdateOp.ID, id);
+        op.set(Document.ID, id);
         Commit.setModified(op, rev);
         op.setMapEntry(UpdateOp.DELETED, rev.toString(), "false");
         for (String p : properties.keySet()) {

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/UpdateOp.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/UpdateOp.java?rev=1513869&r1=1513868&r2=1513869&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/UpdateOp.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/UpdateOp.java Wed Aug 14 13:22:05 2013
@@ -24,13 +24,7 @@ import java.util.TreeMap;
  * A MongoDB "update" operation for one node.
  */
 public class UpdateOp {
-    
-    /**
-     * The node id, which contains the depth of the path
-     * (0 for root, 1 for children of the root), and then the path.
-     */
-    static final String ID = "_id";
-    
+
     /**
      * The last revision. Key: machine id, value: revision.
      */

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=1513869&r1=1513868&r2=1513869&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 Wed Aug 14 13:22:05 2013
@@ -108,7 +108,7 @@ public class MongoDocumentStoreTest {
         for (int i = 0; i < nUpdates; i++) {
             String path = "/node" + i;
             UpdateOp updateOp = new UpdateOp(path, path, true);
-            updateOp.set(UpdateOp.ID, "/node" + i);
+            updateOp.set(Document.ID, "/node" + i);
             updateOp.setMapEntry("property1", "key1", "value1");
             updateOp.increment("property2", 1);
             updateOp.set("property3", "value3");