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/10/01 14:11:01 UTC

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

Author: mreutegg
Date: Tue Oct  1 12:11:01 2013
New Revision: 1528065

URL: http://svn.apache.org/r1528065
Log:
OAK-1044: Reduce traffic between MongoMK and MongoDB
- Introduce multi document update in DocumentStore and use it in MongoMK.backgroundWrite()

Modified:
    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/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/MongoMK.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/UpdateOp.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/LoggingDocumentStoreWrapper.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/TimingDocumentStoreWrapper.java

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=1528065&r1=1528064&r2=1528065&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 Oct  1 12:11:01 2013
@@ -205,7 +205,7 @@ public class Commit {
                 // branch commits only become visible on merge
                 NodeDocument.setLastRev(op, revision);
             }
-            if (op.isNew) {
+            if (op.isNew()) {
                 NodeDocument.setDeleted(op, revision, false);
             }
             if (op == commitRoot) {
@@ -219,7 +219,7 @@ public class Commit {
                 }
             }
         }
-        if (changedNodes.size() == 0 && commitRoot.isNew) {
+        if (changedNodes.size() == 0 && commitRoot.isNew()) {
             // no updates and root of commit is also new. that is,
             // it is the root of a subtree added in a commit.
             // so we try to add the root like all other nodes
@@ -256,7 +256,7 @@ public class Commit {
             // with added nodes (the commit root might be written twice,
             // first to check if there was a conflict, and only then to commit
             // the revision, with the revision property set)
-            if (changedNodes.size() > 0 || !commitRoot.isNew) {
+            if (changedNodes.size() > 0 || !commitRoot.isNew()) {
                 NodeDocument.setRevision(commitRoot, revision, commitValue);
                 opLog.add(commitRoot);
                 createOrUpdateNode(store, commitRoot);
@@ -307,17 +307,17 @@ public class Commit {
             }
             String conflictMessage = null;
             if (newestRev == null) {
-                if (op.isDelete || !op.isNew) {
+                if (op.isDelete() || !op.isNew()) {
                     conflictMessage = "The node " + 
                             op.getId() + " does not exist or is already deleted";
                 }
             } else {
-                if (op.isNew) {
+                if (op.isNew()) {
                     conflictMessage = "The node " + 
                             op.getId() + " was already added in revision\n" +
                             newestRev;
                 } else if (mk.isRevisionNewer(newestRev, baseRevision)
-                        && (op.isDelete || isConflicting(doc, op))) {
+                        && (op.isDelete() || isConflicting(doc, op))) {
                     conflictMessage = "The node " + 
                             op.getId() + " was changed in revision\n" + newestRev +
                             ", which was applied after the base revision\n" + 
@@ -401,9 +401,9 @@ public class Commit {
                 }
             }
             UpdateOp op = operations.get(path);
-            boolean isNew = op != null && op.isNew;
+            boolean isNew = op != null && op.isNew();
             boolean isWritten = op != null;
-            boolean isDelete = op != null && op.isDelete;
+            boolean isDelete = op != null && op.isDelete();
             mk.applyChanges(revision, path, 
                     isNew, isDelete, isWritten, isBranchCommit,
                     added, removed);

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=1528065&r1=1528064&r2=1528065&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 Oct  1 12:11:01 2013
@@ -112,6 +112,19 @@ public interface DocumentStore {
      * @return true if this worked (if none of the documents already existed)
      */
     <T extends Document> boolean create(Collection<T> collection, List<UpdateOp> updateOps);
+
+    /**
+     * Update documents with the given keys. Only existing documents are
+     * updated.
+     *
+     * @param <T> the document type.
+     * @param collection the collection.
+     * @param keys the keys of the documents to update.
+     * @param updateOp the update operation to apply to each of the documents.
+     */
+    <T extends Document> void update(Collection<T> collection,
+                                     List<String> keys,
+                                     UpdateOp updateOp);
     
     /**
      * Create or update a document. For MongoDb, this is using "findAndModify" with

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=1528065&r1=1528064&r2=1528065&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 Oct  1 12:11:01 2013
@@ -179,7 +179,7 @@ public class MemoryDocumentStore impleme
 
             T doc = collection.newDocument(this);
             if (oldDoc == null) {
-                if (!update.isNew) {
+                if (!update.isNew()) {
                     throw new MicroKernelException("Document does not exist: " + update.id);
                 }
             } else {
@@ -200,7 +200,7 @@ public class MemoryDocumentStore impleme
 
     private static boolean checkConditions(Document doc,
                                            UpdateOp update) {
-        for (Map.Entry<Key, Operation> change : update.changes.entrySet()) {
+        for (Map.Entry<Key, Operation> change : update.getChanges().entrySet()) {
             Operation op = change.getValue();
             if (op.type == Operation.Type.CONTAINS_MAP_ENTRY) {
                 Key k = change.getKey();
@@ -246,7 +246,7 @@ public class MemoryDocumentStore impleme
     public static void applyChanges(@Nonnull Document doc,
                                     @Nonnull UpdateOp update,
                                     @Nonnull Comparator<Revision> comparator) {
-        for (Entry<Key, Operation> e : checkNotNull(update).changes.entrySet()) {
+        for (Entry<Key, Operation> e : checkNotNull(update).getChanges().entrySet()) {
             Key k = e.getKey();
             Operation op = e.getValue();
             switch (op.type) {
@@ -312,6 +312,25 @@ public class MemoryDocumentStore impleme
     }
 
     @Override
+    public <T extends Document> void update(Collection<T> collection,
+                                            List<String> keys,
+                                            UpdateOp updateOp) {
+        Lock lock = rwLock.writeLock();
+        lock.lock();
+        try {
+            ConcurrentSkipListMap<String, T> map = getMap(collection);
+            for (String key : keys) {
+                if (!map.containsKey(key)) {
+                    continue;
+                }
+                internalCreateOrUpdate(collection, new UpdateOp(key, updateOp), true);
+            }
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    @Override
     public String toString() {
         StringBuilder buff = new StringBuilder();
         buff.append("Nodes:\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=1528065&r1=1528064&r2=1528065&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 Oct  1 12:11:01 2013
@@ -269,66 +269,19 @@ public class MongoDocumentStore implemen
                                                  boolean upsert,
                                                  boolean checkConditions) {
         DBCollection dbCollection = getDBCollection(collection);
-
-        BasicDBObject setUpdates = new BasicDBObject();
-        BasicDBObject incUpdates = new BasicDBObject();
-        BasicDBObject unsetUpdates = new BasicDBObject();
-
-        // always increment modCount
-        updateOp.increment(Document.MOD_COUNT, 1);
-
-        // other updates
-        for (Entry<Key, Operation> entry : updateOp.changes.entrySet()) {
-            Key k = entry.getKey();
-            if (k.getName().equals(Document.ID)) {
-                // avoid exception "Mod on _id not allowed"
-                continue;
-            }
-            Operation op = entry.getValue();
-            switch (op.type) {
-                case SET: {
-                    setUpdates.append(k.toString(), op.value);
-                    break;
-                }
-                case INCREMENT: {
-                    incUpdates.append(k.toString(), op.value);
-                    break;
-                }
-                case SET_MAP_ENTRY: {
-                    setUpdates.append(k.toString(), op.value);
-                    break;
-                }
-                case REMOVE_MAP_ENTRY: {
-                    unsetUpdates.append(k.toString(), "1");
-                    break;
-                }
-            }
-        }
-
-        BasicDBObject update = new BasicDBObject();
-        if (!setUpdates.isEmpty()) {
-            update.append("$set", setUpdates);
-        }
-        if (!incUpdates.isEmpty()) {
-            update.append("$inc", incUpdates);
-        }
-        if (!unsetUpdates.isEmpty()) {
-            update.append("$unset", unsetUpdates);
-        }
-
-        // dbCollection.update(query, update, true /*upsert*/, false /*multi*/,
-        //         WriteConcern.SAFE);
-        // return null;
+        DBObject update = createUpdate(updateOp);
 
         // get modCount of cached document
         Number modCount = null;
-        @SuppressWarnings("unchecked")
-        T cachedDoc = (T) nodesCache.getIfPresent(updateOp.getId());
-        if (cachedDoc != null) {
-            modCount = cachedDoc.getModCount();
+        T cachedDoc = null;
+        if (collection == Collection.NODES) {
+            //noinspection unchecked
+            cachedDoc = (T) nodesCache.getIfPresent(updateOp.getId());
+            if (cachedDoc != null) {
+                modCount = cachedDoc.getModCount();
+            }
         }
 
-
         long start = start();
         try {
             // perform a conditional update with limited result
@@ -345,7 +298,6 @@ public class MongoDocumentStore implemen
                         false /*upsert*/);
                 if (oldNode != null) {
                     // success, update cached document
-                    // FIXME: ensure consistent cache update
                     applyToCache(collection, cachedDoc, updateOp);
                     // return previously cached document
                     return cachedDoc;
@@ -403,7 +355,7 @@ public class MongoDocumentStore implemen
             T target = collection.newDocument(this);
             MemoryDocumentStore.applyChanges(target, update, comparator);
             docs.add(target);
-            for (Entry<Key, Operation> entry : update.changes.entrySet()) {
+            for (Entry<Key, Operation> entry : update.getChanges().entrySet()) {
                 Key k = entry.getKey();
                 Operation op = entry.getValue();
                 switch (op.type) {
@@ -456,6 +408,39 @@ public class MongoDocumentStore implemen
         }        
     }
 
+    @Override
+    public <T extends Document> void update(Collection<T> collection,
+                                            List<String> keys,
+                                            UpdateOp updateOp) {
+        DBCollection dbCollection = getDBCollection(collection);
+        QueryBuilder query = QueryBuilder.start(Document.ID).in(keys);
+        DBObject update = createUpdate(updateOp);
+        long start = start();
+        try {
+            try {
+
+                WriteResult writeResult = dbCollection.updateMulti(query.get(), update);
+                if (writeResult.getError() != null) {
+                    throw new MicroKernelException("Update failed: " + writeResult.getError());
+                }
+                if (collection == Collection.NODES) {
+                    // update cache
+                    for (String key : keys) {
+                        @SuppressWarnings("unchecked")
+                        T doc = (T) nodesCache.getIfPresent(key);
+                        if (doc != null) {
+                            applyToCache(collection, doc, new UpdateOp(key, updateOp));
+                        }
+                    }
+                }
+            } catch (MongoException e) {
+                throw new MicroKernelException(e);
+            }
+        } finally {
+            end("update", start);
+        }
+    }
+
     @CheckForNull
     private <T extends Document> T convertFromDBObject(@Nonnull Collection<T> collection,
                                                        @Nullable DBObject n) {
@@ -531,6 +516,17 @@ public class MongoDocumentStore implemen
     }
 
 
+    /**
+     * Applies an update to the nodes cache.
+     * <p>
+     * FIXME: ensure consistent cache update.
+     *
+     * @param <T> the document type.
+     * @param collection the document collection.
+     * @param oldDoc the old document or <code>null</code> if the update is for
+     *               a new document (insert).
+     * @param updateOp the update operation.
+     */
     private <T extends Document> void applyToCache(@Nonnull Collection<T> collection,
                                                    @Nullable T oldDoc,
                                                    @Nonnull UpdateOp updateOp) {
@@ -553,7 +549,7 @@ public class MongoDocumentStore implemen
                                               boolean checkConditions) {
         QueryBuilder query = getByKeyQuery(updateOp.id);
 
-        for (Entry<Key, Operation> entry : updateOp.changes.entrySet()) {
+        for (Entry<Key, Operation> entry : updateOp.getChanges().entrySet()) {
             Key k = entry.getKey();
             Operation op = entry.getValue();
             switch (op.type) {
@@ -568,4 +564,60 @@ public class MongoDocumentStore implemen
         return query;
     }
 
+    /**
+     * Creates a MongoDB update object from the given UpdateOp.
+     *
+     * @param updateOp the update op.
+     * @return the DBObject.
+     */
+    @Nonnull
+    private DBObject createUpdate(UpdateOp updateOp) {
+        BasicDBObject setUpdates = new BasicDBObject();
+        BasicDBObject incUpdates = new BasicDBObject();
+        BasicDBObject unsetUpdates = new BasicDBObject();
+
+        // always increment modCount
+        updateOp.increment(Document.MOD_COUNT, 1);
+
+        // other updates
+        for (Entry<Key, Operation> entry : updateOp.getChanges().entrySet()) {
+            Key k = entry.getKey();
+            if (k.getName().equals(Document.ID)) {
+                // avoid exception "Mod on _id not allowed"
+                continue;
+            }
+            Operation op = entry.getValue();
+            switch (op.type) {
+                case SET: {
+                    setUpdates.append(k.toString(), op.value);
+                    break;
+                }
+                case INCREMENT: {
+                    incUpdates.append(k.toString(), op.value);
+                    break;
+                }
+                case SET_MAP_ENTRY: {
+                    setUpdates.append(k.toString(), op.value);
+                    break;
+                }
+                case REMOVE_MAP_ENTRY: {
+                    unsetUpdates.append(k.toString(), "1");
+                    break;
+                }
+            }
+        }
+
+        BasicDBObject update = new BasicDBObject();
+        if (!setUpdates.isEmpty()) {
+            update.append("$set", setUpdates);
+        }
+        if (!incUpdates.isEmpty()) {
+            update.append("$inc", incUpdates);
+        }
+        if (!unsetUpdates.isEmpty()) {
+            update.append("$unset", unsetUpdates);
+        }
+
+        return update;
+    }
 }
\ No newline at end of file

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=1528065&r1=1528064&r2=1528065&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 Tue Oct  1 12:11:01 2013
@@ -451,7 +451,11 @@ public class MongoMK implements MicroKer
         });
         
         long now = Revision.getCurrentTimestamp();
-        for (String p : paths) {
+        UpdateOp updateOp = null;
+        Revision lastRev = null;
+        List<String> ids = new ArrayList<String>();
+        for (int i = 0; i < paths.size(); i++) {
+            String p = paths.get(i);
             Revision r = unsavedLastRevisions.get(p);
             if (r == null) {
                 continue;
@@ -462,10 +466,29 @@ public class MongoMK implements MicroKer
             if (Revision.getTimestampDifference(now, r.getTimestamp()) < asyncDelay) {
                 continue;
             }
-            Commit commit = new Commit(this, null, r);
-            commit.touchNode(p);
-            store.createOrUpdate(Collection.NODES, commit.getUpdateOperationForNode(p));
-            unsavedLastRevisions.remove(p);
+            int size = ids.size();
+            if (updateOp == null) {
+                // create UpdateOp
+                Commit commit = new Commit(this, null, r);
+                commit.touchNode(p);
+                updateOp = commit.getUpdateOperationForNode(p);
+                lastRev = r;
+                ids.add(Utils.getIdFromPath(p));
+            } else if (r.equals(lastRev)) {
+                // use multi update when possible
+                ids.add(Utils.getIdFromPath(p));
+            }
+            // update if this is the last path or
+            // revision is not equal to last revision
+            if (i + 1 >= paths.size() || size == ids.size()) {
+                store.update(Collection.NODES, ids, updateOp);
+                for (String id : ids) {
+                    unsavedLastRevisions.remove(Utils.getPathFromId(id));
+                }
+                ids.clear();
+                updateOp = null;
+                lastRev = null;
+            }
         }
     }
 

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=1528065&r1=1528064&r2=1528065&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 Oct  1 12:11:01 2013
@@ -509,7 +509,7 @@ public class NodeDocument extends Docume
             }
         }
 
-        for (Map.Entry<Key, Operation> entry : op.changes.entrySet()) {
+        for (Map.Entry<Key, Operation> entry : op.getChanges().entrySet()) {
             if (entry.getValue().type != Operation.Type.SET_MAP_ENTRY) {
                 continue;
             }

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=1528065&r1=1528064&r2=1528065&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 Tue Oct  1 12:11:01 2013
@@ -29,14 +29,14 @@ import static com.google.common.base.Pre
 /**
  * A MongoDB "update" operation for one document.
  */
-public class UpdateOp {
+public final class UpdateOp {
 
     final String id;
     
-    final boolean isNew;
-    boolean isDelete;
+    private boolean isNew;
+    private boolean isDelete;
     
-    final Map<Key, Operation> changes = new HashMap<Key, Operation>();
+    private Map<Key, Operation> changes = new HashMap<Key, Operation>();
     
     /**
      * Create an update operation for the document with the given id. The commit
@@ -49,6 +49,21 @@ public class UpdateOp {
         this.id = id;
         this.isNew = isNew;
     }
+
+    /**
+     * Creates an update operation for the document with the given id. The
+     * changes are shared with the other update operation.
+     *
+     * @param id the primary key.
+     * @param other the other update operation.
+     */
+    UpdateOp(String id, UpdateOp other) {
+        this.id = id;
+        this.changes = other.changes;
+        this.isNew = other.isNew;
+        this.isDelete = other.isDelete;
+
+    }
     
     public String getId() {
         return id;
@@ -61,6 +76,14 @@ public class UpdateOp {
     void setDelete(boolean isDelete) {
         this.isDelete = isDelete;
     }
+
+    boolean isDelete() {
+        return isDelete;
+    }
+
+    Map<Key, Operation> getChanges() {
+        return changes;
+    }
     
     /**
      * Add a new or update an existing map entry.

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/LoggingDocumentStoreWrapper.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/LoggingDocumentStoreWrapper.java?rev=1528065&r1=1528064&r2=1528065&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/LoggingDocumentStoreWrapper.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/util/LoggingDocumentStoreWrapper.java Tue Oct  1 12:11:01 2013
@@ -150,6 +150,25 @@ public class LoggingDocumentStoreWrapper
         }
     }
 
+    @Override
+    public <T extends Document> void update(final Collection<T> collection,
+                                            final List<String> keys,
+                                            final UpdateOp updateOp) {
+        try {
+            logMethod("update", collection, keys, updateOp);
+            logResult(new Callable<Void>() {
+                @Override
+                public Void call() throws Exception {
+                    store.update(collection, keys, updateOp);
+                    return null;
+                }
+            });
+        } catch (Exception e) {
+            logException(e);
+            throw convert(e);
+        }
+    }
+
     @Nonnull
     @Override
     public <T extends Document> T createOrUpdate(final Collection<T> collection,

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=1528065&r1=1528064&r2=1528065&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 Oct  1 12:11:01 2013
@@ -191,6 +191,22 @@ public class TimingDocumentStoreWrapper 
     }
 
     @Override
+    public <T extends Document> void update(Collection<T> collection,
+                                            List<String> keys,
+                                            UpdateOp updateOp) {
+        try {
+            long start = now();
+            base.update(collection, keys, updateOp);
+            updateAndLogTimes("update", start, 0, 0);
+            if (logCommonCall()) {
+                logCommonCall(start, "update " + collection);
+            }
+        } catch (Exception e) {
+            throw convert(e);
+        }
+    }
+
+    @Override
     @CheckForNull
     public <T extends Document> T createOrUpdate(Collection<T> collection, UpdateOp update)
             throws MicroKernelException {