Author: thomasm
Date: Thu Apr 25 15:06:42 2013
New Revision: 1475812
URL: http://svn.apache.org/r1475812
Log:
OAK-619 MongoMK: cluster aware revision id comparison (WIP)
Added:
jackrabbit/oak/trunk/oak-mongomk/src/test/java/org/apache/jackrabbit/mongomk/RevisionTest.java
Modified:
jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/MemoryDocumentStore.java
jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/MongoDocumentStore.java
jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/MongoMK.java
jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/Revision.java
jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/UpdateOp.java
jackrabbit/oak/trunk/oak-mongomk/src/test/java/org/apache/jackrabbit/mongomk/SimpleTest.java
jackrabbit/oak/trunk/oak-mongomk/src/test/java/org/apache/jackrabbit/mongomk/blob/MongoMKWriteTest.java
Modified: jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/MemoryDocumentStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/MemoryDocumentStore.java?rev=1475812&r1=1475811&r2=1475812&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/MemoryDocumentStore.java
(original)
+++ jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/MemoryDocumentStore.java
Thu Apr 25 15:06:42 2013
@@ -175,7 +175,7 @@ public class MemoryDocumentStore impleme
}
} else {
if (value instanceof java.util.Collection) {
- java.util.Collection col = (java.util.Collection) value;
+ java.util.Collection<?> col = (java.util.Collection<?>)
value;
if (Boolean.TRUE.equals(op.value)) {
if (!col.contains(kv[1])) {
return false;
@@ -250,8 +250,10 @@ public class MemoryDocumentStore impleme
}
m.put(kv[1], op.value);
break;
-
}
+ case CONTAINS_MAP_ENTRY:
+ // no effect
+ break;
}
}
}
Modified: jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/MongoDocumentStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/MongoDocumentStore.java?rev=1475812&r1=1475811&r2=1475812&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/MongoDocumentStore.java
(original)
+++ jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/MongoDocumentStore.java
Thu Apr 25 15:06:42 2013
@@ -208,7 +208,7 @@ public class MongoDocumentStore implemen
}
@CheckForNull
- private Map<String, Object> findAndModifiy(Collection collection,
+ private Map<String, Object> findAndModify(Collection collection,
UpdateOp updateOp,
boolean upsert,
boolean checkConditions) {
@@ -307,7 +307,7 @@ public class MongoDocumentStore implemen
UpdateOp update)
throws MicroKernelException {
log("createOrUpdate", update);
- Map<String, Object> map = findAndModifiy(collection, update, true, false);
+ Map<String, Object> map = findAndModify(collection, update, true, false);
log("createOrUpdate returns ", map);
return map;
}
@@ -317,7 +317,7 @@ public class MongoDocumentStore implemen
UpdateOp update)
throws MicroKernelException {
log("findAndUpdate", update);
- Map<String, Object> map = findAndModifiy(collection, update, false, true);
+ Map<String, Object> map = findAndModify(collection, update, false, true);
log("findAndUpdate returns ", map);
return map;
}
@@ -350,9 +350,12 @@ public class MongoDocumentStore implemen
inserts[i].put(kv[0], value);
break;
}
- case REMOVE_MAP_ENTRY: {
+ case REMOVE_MAP_ENTRY:
// nothing to do for new entries
- }
+ break;
+ case CONTAINS_MAP_ENTRY:
+ // no effect
+ break;
}
}
}
Modified: jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/MongoMK.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/MongoMK.java?rev=1475812&r1=1475811&r2=1475812&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/MongoMK.java
(original)
+++ jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/MongoMK.java
Thu Apr 25 15:06:42 2013
@@ -46,6 +46,7 @@ import org.apache.jackrabbit.mk.json.Jso
import org.apache.jackrabbit.mk.json.JsopWriter;
import org.apache.jackrabbit.mongomk.DocumentStore.Collection;
import org.apache.jackrabbit.mongomk.Node.Children;
+import org.apache.jackrabbit.mongomk.Revision.RevisionComparator;
import org.apache.jackrabbit.mongomk.blob.MongoBlobStore;
import org.apache.jackrabbit.mongomk.util.Utils;
import org.apache.jackrabbit.oak.commons.PathUtils;
@@ -105,7 +106,7 @@ public class MongoMK implements MicroKer
private final BlobStore blobStore;
/**
- * The cluster node info.
+ * The cluster instance info.
*/
private final ClusterNodeInfo clusterNodeInfo;
@@ -129,7 +130,11 @@ public class MongoMK implements MicroKer
private final Cache<String, Node.Children> nodeChildrenCache;
/**
- * The unsaved last revisions.
+ * The unsaved last revisions. This contains the parents of all changed
+ * nodes, once those nodes are committed but the parent node itself wasn't
+ * committed yet. The parents are not immediately persisted as this would
+ * cause each commit to change all parents (including the root node), which
+ * would limit write scalability.
*
* Key: path, value: revision.
*/
@@ -137,7 +142,7 @@ public class MongoMK implements MicroKer
new ConcurrentHashMap<String, Revision>();
/**
- * The last known revision for each cluster node.
+ * The last known revision for each cluster instance.
*
* Key: the machine id, value: revision.
*/
@@ -161,6 +166,11 @@ public class MongoMK implements MicroKer
private final Map<Revision, Revision> branchCommits
= new ConcurrentHashMap<Revision, Revision>();
+ /**
+ * The comparator for revisions.
+ */
+ private final RevisionComparator revisionComparator = new RevisionComparator();
+
MongoMK(Builder builder) {
this.store = builder.getDocumentStore();
this.blobStore = builder.getBlobStore();
@@ -290,7 +300,7 @@ public class MongoMK implements MicroKer
Revision r = Revision.fromString(e.getValue());
Revision last = lastKnownRevision.get(machineId);
- if (last == null || last.compareRevisionTime(r) != 0) {
+ if (last == null || r.compareRevisionTime(last) > 0) {
// TODO invalidating the whole cache is not really needed,
// instead only those children that are cached could be checked
@@ -415,17 +425,24 @@ public class MongoMK implements MicroKer
}
if (x.getClusterId() == this.clusterId &&
requestRevision.getClusterId() == this.clusterId) {
- // both revisions were created by this cluster node:
- // compare timestamps only
+ // both revisions were created by this cluster instance:
+ // compare timestamps and counters
return requestRevision.compareRevisionTime(x) >= 0;
}
// TODO currently we only compare the timestamps
return requestRevision.compareRevisionTime(x) >= 0;
}
+ /**
+ * Checks that revision x is newer than another revision.
+ *
+ * @param x the revision to check
+ * @param previous the presumed earlier revision
+ * @return true if x is newer
+ */
boolean isRevisionNewer(@Nonnull Revision x, @Nonnull Revision previous) {
// TODO currently we only compare the timestamps
- return x.compareRevisionTime(previous) > 0;
+ return revisionComparator.compare(x, previous) > 0;
}
/**
@@ -1160,9 +1177,8 @@ public class MongoMK implements MicroKer
}
headRevision = newRevision();
return headRevision.toString();
- } else {
- throw new MicroKernelException("Conflicting concurrent change");
}
+ throw new MicroKernelException("Conflicting concurrent change");
}
@Override
Modified: jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/Revision.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/Revision.java?rev=1475812&r1=1475811&r2=1475812&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/Revision.java
(original)
+++ jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/Revision.java
Thu Apr 25 15:06:42 2013
@@ -16,18 +16,28 @@
*/
package org.apache.jackrabbit.mongomk;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
/**
* A revision.
*/
public class Revision {
- static long timestampOffset = java.sql.Timestamp.valueOf("2013-01-01 00:00:00.0").getTime()
/ 100;
- static volatile long lastTimestamp;
- static volatile int count;
+ private static volatile long lastTimestamp;
+
+ private static volatile long lastRevisionTimestamp;
+ private static volatile int lastRevisionCount;
/**
- * The timestamp in milliseconds since 2013 (unlike in seconds since 1970 as
- * in MongoDB).
+ * The timestamp in milliseconds since 1970 (unlike in seconds as in
+ * MongoDB). The timestamp is local to the machine that generated the
+ * revision, such that timestamps of revisions can only be compared if the
+ * machine id is the same.
*/
private long timestamp;
@@ -49,7 +59,8 @@ public class Revision {
}
/**
- * Compare the time part of two revisions.
+ * Compare the time part of two revisions. If they contain the same time,
+ * the counter is compared.
*
* @return -1 if this revision occurred earlier, 1 if later, 0 if equal
*/
@@ -71,31 +82,33 @@ public class Revision {
long timestamp = getCurrentTimestamp();
int c;
synchronized (Revision.class) {
- if (timestamp > lastTimestamp) {
- lastTimestamp = timestamp;
- c = count = 0;
- } else if (timestamp < lastTimestamp) {
- timestamp = lastTimestamp;
- c = ++count;
+ if (timestamp == lastRevisionTimestamp) {
+ c = ++lastRevisionCount;
} else {
- c = ++count;
- }
- if (c >= 0xfff) {
- timestamp++;
- c = 0;
- lastTimestamp = Math.max(timestamp, lastTimestamp);
+ lastRevisionTimestamp = timestamp;
+ lastRevisionCount = c = 0;
}
}
return new Revision(timestamp, c, clusterId);
}
/**
- * Get the timestamp value of the current date and time.
+ * Get the timestamp value of the current date and time. Within the same
+ * process, the returned value is never smaller than a previously returned
+ * value, even if the system time was changed.
*
* @return the timestamp
*/
public static long getCurrentTimestamp() {
- return System.currentTimeMillis() / 100 - timestampOffset;
+ long timestamp = System.currentTimeMillis();
+ if (timestamp < lastTimestamp) {
+ // protect against decreases in the system time,
+ // time machines, and other fluctuations in the time continuum
+ timestamp = lastTimestamp;
+ } else if (timestamp > lastTimestamp) {
+ lastTimestamp = timestamp;
+ }
+ return timestamp;
}
/**
@@ -106,7 +119,7 @@ public class Revision {
* @return the difference in milliseconds
*/
public static long getTimestampDifference(long a, long b) {
- return (a - b) * 100;
+ return a - b;
}
public static Revision fromString(String rev) {
@@ -144,6 +157,10 @@ public class Revision {
public long getTimestamp() {
return timestamp;
}
+
+ public int getCounter() {
+ return counter;
+ }
public int hashCode() {
return (int) (timestamp >>> 32) ^ (int) timestamp ^ counter ^ clusterId;
@@ -164,5 +181,182 @@ public class Revision {
public int getClusterId() {
return clusterId;
}
+
+ /**
+ * Revision ranges allow to compare revisions ids of different cluster instances. A
+ * range tells when a list of revisions from a certain cluster instance was seen by
+ * the current process.
+ */
+ static class RevisionRange {
+
+ /**
+ * The newest revision for the given cluster instance and time.
+ */
+ Revision revision;
+
+ /**
+ * The (local) timestamp; the time when this revision was seen by this
+ * cluster instance.
+ */
+ long timestamp;
+
+ public String toString() {
+ return revision + ":" + timestamp;
+ }
+
+ }
+
+ /**
+ * A facility that is able to compare revisions of different cluster instances.
+ * It contains a map of revision ranges.
+ */
+ public static class RevisionComparator implements Comparator<Revision> {
+
+ /**
+ * The map of cluster instances to lists of revision ranges.
+ */
+ private final ConcurrentMap<Integer, List<RevisionRange>> map =
+ new ConcurrentHashMap<Integer, List<RevisionRange>>();
+
+ /**
+ * When comparing revisions that occurred before, the timestamp is ignored.
+ */
+ private long oldestTimestamp;
+
+ /**
+ * Forget the order of older revisions. After calling this method, when comparing
+ * revisions that happened before the given value, the timestamp order is used
+ * (time dilation is ignored for older events).
+ *
+ * @param timestamp the time in milliseconds (see {@link #getCurrentTimestamp})
+ */
+ public void purge(long timestamp) {
+ oldestTimestamp = timestamp;
+ for (int clusterId : map.keySet()) {
+ while (true) {
+ List<RevisionRange> list = map.get(clusterId);
+ List<RevisionRange> newList = purge(list);
+ if (newList == null) {
+ // retry if removing was not successful
+ if (map.remove(clusterId, list)) {
+ break;
+ }
+ } else if (newList == list) {
+ // no change
+ break;
+ } else {
+ // retry if replacing was not successful
+ if (map.replace(clusterId, list, newList)) {
+ break;
+ }
+ }
+ }
+ }
+ }
+
+ private List<RevisionRange> purge(List<RevisionRange> list) {
+ int i = 0;
+ for (; i < list.size(); i++) {
+ RevisionRange r = list.get(i);
+ if (r.timestamp > oldestTimestamp) {
+ break;
+ }
+ }
+ if (i > list.size() - 1) {
+ return null;
+ } else if (i == 0) {
+ return list;
+ }
+ return new ArrayList<RevisionRange>(list.subList(i, list.size()));
+ }
+
+ /**
+ * Add the revision to the top of the queue for the given cluster node.
+ * If an entry for this timestamp already exists, it is replaced.
+ *
+ * @param r the revision
+ * @param timestamp the timestamp
+ */
+ public void add(Revision r, long timestamp) {
+ int clusterId = r.getClusterId();
+ while (true) {
+ List<RevisionRange> list = map.get(clusterId);
+ List<RevisionRange> newList;
+ if (list == null) {
+ newList = new ArrayList<RevisionRange>();
+ } else {
+ RevisionRange last = list.get(list.size() - 1);
+ if (last.timestamp == timestamp) {
+ last.revision = r;
+ return;
+ }
+ newList = new ArrayList<RevisionRange>(list);
+ }
+ RevisionRange range = new RevisionRange();
+ range.timestamp = timestamp;
+ range.revision = r;
+ newList.add(range);
+ if (list == null) {
+ if (map.putIfAbsent(clusterId, newList) == null) {
+ return;
+ }
+ } else {
+ if (map.replace(clusterId, list, newList)) {
+ return;
+ }
+ }
+ }
+ }
+
+ @Override
+ public int compare(Revision o1, Revision o2) {
+ if (o1.getClusterId() == o2.getClusterId()) {
+ return o1.compareRevisionTime(o2);
+ }
+ RevisionRange range1 = getRevisionRange(o1);
+ RevisionRange range2 = getRevisionRange(o2);
+ if (range1 == null || range2 == null) {
+ return o1.compareRevisionTime(o2);
+ }
+ if (range1.timestamp != range2.timestamp) {
+ return range1.timestamp < range2.timestamp ? -1 : 1;
+ }
+ int result = o1.compareRevisionTime(o2);
+ if (result != 0) {
+ return result;
+ }
+ return o1.getClusterId() < o2.getClusterId() ? -1 : 1;
+ }
+
+ private RevisionRange getRevisionRange(Revision r) {
+ List<RevisionRange> list = map.get(r.getClusterId());
+ if (list == null) {
+ return null;
+ }
+ // search from latest backward
+ // (binary search could be used, but we expect most queries
+ // at the end of the list)
+ for (int i = list.size() - 1; i >= 0; i--) {
+ RevisionRange range = list.get(i);
+ if (r.compareRevisionTime(range.revision) >= 0) {
+ return range;
+ }
+ }
+ return null;
+ }
+
+ public String toString() {
+ StringBuilder buff = new StringBuilder();
+ for (int clusterId : new TreeSet<Integer>(map.keySet())) {
+ buff.append(clusterId).append(':');
+ for (RevisionRange r : map.get(clusterId)) {
+ buff.append(' ').append(r);
+ }
+ buff.append("; ");
+ }
+ return buff.toString();
+ }
+
+ }
}
Modified: jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/UpdateOp.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/UpdateOp.java?rev=1475812&r1=1475811&r2=1475812&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/UpdateOp.java
(original)
+++ jackrabbit/oak/trunk/oak-mongomk/src/main/java/org/apache/jackrabbit/mongomk/UpdateOp.java
Thu Apr 25 15:06:42 2013
@@ -248,12 +248,12 @@ public class UpdateOp {
/**
* A MongoDB operation for a given key within a document.
*/
- static class Operation {
+ public static class Operation {
/**
* The MongoDB operation type.
*/
- enum Type {
+ public enum Type {
/**
* Set the value.
@@ -318,6 +318,7 @@ public class UpdateOp {
case SET:
case REMOVE_MAP_ENTRY:
case SET_MAP:
+ case CONTAINS_MAP_ENTRY:
// nothing to do
break;
case SET_MAP_ENTRY:
Added: jackrabbit/oak/trunk/oak-mongomk/src/test/java/org/apache/jackrabbit/mongomk/RevisionTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-mongomk/src/test/java/org/apache/jackrabbit/mongomk/RevisionTest.java?rev=1475812&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-mongomk/src/test/java/org/apache/jackrabbit/mongomk/RevisionTest.java
(added)
+++ jackrabbit/oak/trunk/oak-mongomk/src/test/java/org/apache/jackrabbit/mongomk/RevisionTest.java
Thu Apr 25 15:06:42 2013
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jackrabbit.mongomk;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.jackrabbit.mongomk.Revision.RevisionComparator;
+import org.junit.Test;
+
+/**
+ * Tests the revision class
+ */
+public class RevisionTest {
+
+ @Test
+ public void fromStringToString() {
+ for (int i = 0; i < 10000; i++) {
+ Revision r = Revision.newRevision(i);
+ // System.out.println(r);
+ Revision r2 = Revision.fromString(r.toString());
+ assertEquals(r.toString(), r2.toString());
+ assertEquals(r.hashCode(), r2.hashCode());
+ assertTrue(r.equals(r2));
+ }
+ }
+
+ @Test
+ public void difference() throws InterruptedException {
+ Revision r0 = Revision.newRevision(0);
+ Revision r1 = Revision.newRevision(0);
+ long timestamp = Revision.getCurrentTimestamp();
+ assertTrue(Revision.getTimestampDifference(r1.getTimestamp(), r0.getTimestamp())
< 10);
+ assertTrue(Revision.getTimestampDifference(timestamp, r0.getTimestamp()) < 10);
+ Thread.sleep(2);
+ Revision r2 = Revision.newRevision(0);
+ assertTrue(Revision.getTimestampDifference(r2.getTimestamp(), r0.getTimestamp())
> 0);
+ assertTrue(Revision.getTimestampDifference(r2.getTimestamp(), r0.getTimestamp())
< 20);
+ }
+
+ @Test
+ public void equalsHashCode() {
+ Revision a = Revision.newRevision(0);
+ Revision b = Revision.newRevision(0);
+ assertTrue(a.equals(a));
+ assertFalse(a.equals(b));
+ assertFalse(b.equals(a));
+ assertFalse(a.hashCode() == b.hashCode());
+ Revision a1 = Revision.fromString(a.toString());
+ assertTrue(a.equals(a1));
+ assertTrue(a1.equals(a));
+ Revision a2 = new Revision(a.getTimestamp(), a.getCounter(), a.getClusterId());
+ assertTrue(a.equals(a2));
+ assertTrue(a2.equals(a));
+ assertEquals(a.hashCode(), a1.hashCode());
+ assertEquals(a.hashCode(), a2.hashCode());
+ Revision x1 = new Revision(a.getTimestamp() + 1, a.getCounter(), a.getClusterId());
+ assertFalse(a.equals(x1));
+ assertFalse(x1.equals(a));
+ assertFalse(a.hashCode() == x1.hashCode());
+ Revision x2 = new Revision(a.getTimestamp(), a.getCounter() + 1, a.getClusterId());
+ assertFalse(a.equals(x2));
+ assertFalse(x2.equals(a));
+ assertFalse(a.hashCode() == x2.hashCode());
+ Revision x3 = new Revision(a.getTimestamp(), a.getCounter(), a.getClusterId() + 1);
+ assertFalse(a.equals(x3));
+ assertFalse(x3.equals(a));
+ assertFalse(a.hashCode() == x3.hashCode());
+ }
+
+ @Test
+ public void compare() throws InterruptedException {
+ Revision last = Revision.newRevision(0);
+ try {
+ last.compareRevisionTime(null);
+ fail();
+ } catch (NullPointerException e) {
+ // expected
+ }
+ for (int i = 0; i < 1000; i++) {
+ Revision r = Revision.newRevision(0);
+ assertTrue(r.compareRevisionTime(r) == 0);
+ assertTrue(r.compareRevisionTime(last) > 0);
+ assertTrue(last.compareRevisionTime(r) < 0);
+ last = r;
+ if (i % 100 == 0) {
+ // ensure the timestamp part changes as well
+ Thread.sleep(1);
+ }
+ }
+ }
+
+ @Test
+ public void revisionComparatorSimple() {
+ RevisionComparator comp = new RevisionComparator();
+ Revision r1 = Revision.newRevision(0);
+ Revision r2 = Revision.newRevision(0);
+ assertEquals(r1.compareRevisionTime(r2), comp.compare(r1, r2));
+ assertEquals(r2.compareRevisionTime(r1), comp.compare(r2, r1));
+ assertEquals(r1.compareRevisionTime(r1), comp.compare(r1, r1));
+ }
+
+ @Test
+ public void revisionComparatorCluster() {
+
+ RevisionComparator comp = new RevisionComparator();
+
+ Revision r1c1 = new Revision(0x110, 0, 1);
+ Revision r2c1 = new Revision(0x120, 0, 1);
+ Revision r3c1 = new Revision(0x130, 0, 1);
+ Revision r1c2 = new Revision(0x100, 0, 2);
+ Revision r2c2 = new Revision(0x200, 0, 2);
+ Revision r3c2 = new Revision(0x300, 0, 2);
+
+ // first, only timestamps are compared
+ assertEquals(1, comp.compare(r1c1, r1c2));
+ assertEquals(-1, comp.compare(r2c1, r2c2));
+ assertEquals(-1, comp.compare(r3c1, r3c2));
+
+ // now we declare r2+r3 of c1 to be after r2+r3 of c2
+ comp.add(r2c1, 20);
+ comp.add(r2c2, 10);
+
+ assertEquals(
+ "1: r120-0-1:20; " +
+ "2: r200-0-2:10; ", comp.toString());
+
+ assertEquals(1, comp.compare(r1c1, r1c2));
+ assertEquals(1, comp.compare(r2c1, r2c2));
+ assertEquals(1, comp.compare(r3c1, r3c2));
+
+ // now we declare r3 of c1 to be before r3 of c2
+ // (with the same range timestamp,
+ // the revision timestamps are compared)
+ comp.add(r3c1, 30);
+ comp.add(r3c2, 30);
+
+ assertEquals(
+ "1: r120-0-1:20 r130-0-1:30; " +
+ "2: r200-0-2:10 r300-0-2:30; ", comp.toString());
+
+ assertEquals(1, comp.compare(r1c1, r1c2));
+ assertEquals(1, comp.compare(r2c1, r2c2));
+ assertEquals(-1, comp.compare(r3c1, r3c2));
+ // reverse
+ assertEquals(-1, comp.compare(r1c2, r1c1));
+ assertEquals(-1, comp.compare(r2c2, r2c1));
+ assertEquals(1, comp.compare(r3c2, r3c1));
+
+ // get rid of old timestamps
+ comp.purge(10);
+ assertEquals(
+ "1: r120-0-1:20 r130-0-1:30; " +
+ "2: r300-0-2:30; ", comp.toString());
+ comp.purge(20);
+ assertEquals(
+ "1: r130-0-1:30; " +
+ "2: r300-0-2:30; ", comp.toString());
+
+ // update an entry
+ comp.add(new Revision(0x301, 1, 2), 30);
+ assertEquals(
+ "1: r130-0-1:30; " +
+ "2: r301-1-2:30; ", comp.toString());
+
+ comp.purge(30);
+ assertEquals("", comp.toString());
+
+ }
+
+}
Modified: jackrabbit/oak/trunk/oak-mongomk/src/test/java/org/apache/jackrabbit/mongomk/SimpleTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-mongomk/src/test/java/org/apache/jackrabbit/mongomk/SimpleTest.java?rev=1475812&r1=1475811&r2=1475812&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-mongomk/src/test/java/org/apache/jackrabbit/mongomk/SimpleTest.java
(original)
+++ jackrabbit/oak/trunk/oak-mongomk/src/test/java/org/apache/jackrabbit/mongomk/SimpleTest.java
Thu Apr 25 15:06:42 2013
@@ -70,18 +70,6 @@ public class SimpleTest {
}
@Test
- public void revision() {
- for (int i = 0; i < 1000; i++) {
- Revision r = Revision.newRevision(i);
- // System.out.println(r);
- Revision r2 = Revision.fromString(r.toString());
- assertEquals(r.toString(), r2.toString());
- assertEquals(r.hashCode(), r2.hashCode());
- assertTrue(r.equals(r2));
- }
- }
-
- @Test
public void addNodeGetNode() {
MongoMK mk = new MongoMK.Builder().open();
Revision rev = mk.newRevision();
Modified: jackrabbit/oak/trunk/oak-mongomk/src/test/java/org/apache/jackrabbit/mongomk/blob/MongoMKWriteTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-mongomk/src/test/java/org/apache/jackrabbit/mongomk/blob/MongoMKWriteTest.java?rev=1475812&r1=1475811&r2=1475812&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-mongomk/src/test/java/org/apache/jackrabbit/mongomk/blob/MongoMKWriteTest.java
(original)
+++ jackrabbit/oak/trunk/oak-mongomk/src/test/java/org/apache/jackrabbit/mongomk/blob/MongoMKWriteTest.java
Thu Apr 25 15:06:42 2013
@@ -26,8 +26,6 @@ import org.apache.jackrabbit.mk.util.Mic
import org.apache.jackrabbit.mongomk.AbstractMongoConnectionTest;
import org.junit.Test;
-import com.mongodb.DB;
-
/**
* Tests for {@code MongoMicroKernel#write(java.io.InputStream)}
*/
|