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 2018/07/19 07:32:02 UTC

svn commit: r1836251 - in /jackrabbit/oak/trunk/oak-store-document/src: main/java/org/apache/jackrabbit/oak/plugins/document/ main/java/org/apache/jackrabbit/oak/plugins/document/util/ test/java/org/apache/jackrabbit/oak/plugins/document/ test/java/org...

Author: mreutegg
Date: Thu Jul 19 07:32:01 2018
New Revision: 1836251

URL: http://svn.apache.org/viewvc?rev=1836251&view=rev
Log:
OAK-7640: Prevent commits in the past

Modified:
    jackrabbit/oak/trunk/oak-store-document/src/main/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStore.java
    jackrabbit/oak/trunk/oak-store-document/src/main/java/org/apache/jackrabbit/oak/plugins/document/util/Utils.java
    jackrabbit/oak/trunk/oak-store-document/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStoreTest.java
    jackrabbit/oak/trunk/oak-store-document/src/test/java/org/apache/jackrabbit/oak/plugins/document/util/UtilsTest.java

Modified: jackrabbit/oak/trunk/oak-store-document/src/main/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-store-document/src/main/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStore.java?rev=1836251&r1=1836250&r2=1836251&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-store-document/src/main/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStore.java (original)
+++ jackrabbit/oak/trunk/oak-store-document/src/main/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStore.java Thu Jul 19 07:32:01 2018
@@ -542,10 +542,8 @@ public final class DocumentNodeStore
             clusterNodeInfo = ClusterNodeInfo.getInstance(nonLeaseCheckingStore,
                     new RecoveryHandlerImpl(nonLeaseCheckingStore, clock, lastRevSeeker),
                     null, null, cid);
+            checkRevisionAge(nonLeaseCheckingStore, clusterNodeInfo, clock);
         }
-        // TODO we should ensure revisions generated from now on
-        // are never "older" than revisions already in the repository for
-        // this cluster id
         this.clusterId = clusterNodeInfo.getId();
 
         clusterNodeInfo.setLeaseCheckMode(builder.getLeaseCheckMode());
@@ -2544,6 +2542,31 @@ public final class DocumentNodeStore
             }
         }
     }
+
+    /**
+     * Checks the revision age as defined in
+     * {@link Utils#checkRevisionAge(DocumentStore, ClusterNodeInfo, Clock)}
+     * and disposes the passed cluster node {@code info} if the check fails.
+     *
+     * @param store the document store from where to read the root document.
+     * @param info the cluster node info with the clusterId.
+     * @param clock the clock to get the current time.
+     * @throws DocumentStoreException if the check fails.
+     */
+    private static void checkRevisionAge(DocumentStore store,
+                                         ClusterNodeInfo info,
+                                         Clock clock)
+            throws DocumentStoreException {
+        boolean success = false;
+        try {
+            Utils.checkRevisionAge(store, info, clock);
+            success = true;
+        } finally {
+            if (!success) {
+                info.dispose();
+            }
+        }
+    }
 
     private void pushJournalEntry(Revision r) throws DocumentStoreException {
         if (!changes.hasChanges()) {

Modified: jackrabbit/oak/trunk/oak-store-document/src/main/java/org/apache/jackrabbit/oak/plugins/document/util/Utils.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-store-document/src/main/java/org/apache/jackrabbit/oak/plugins/document/util/Utils.java?rev=1836251&r1=1836250&r2=1836251&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-store-document/src/main/java/org/apache/jackrabbit/oak/plugins/document/util/Utils.java (original)
+++ jackrabbit/oak/trunk/oak-store-document/src/main/java/org/apache/jackrabbit/oak/plugins/document/util/Utils.java Thu Jul 19 07:32:01 2018
@@ -38,8 +38,10 @@ import com.google.common.collect.Abstrac
 import org.apache.jackrabbit.oak.commons.OakVersion;
 import org.apache.jackrabbit.oak.commons.PathUtils;
 import org.apache.jackrabbit.oak.commons.StringUtils;
+import org.apache.jackrabbit.oak.plugins.document.ClusterNodeInfo;
 import org.apache.jackrabbit.oak.plugins.document.Collection;
 import org.apache.jackrabbit.oak.plugins.document.DocumentStore;
+import org.apache.jackrabbit.oak.plugins.document.DocumentStoreException;
 import org.apache.jackrabbit.oak.plugins.document.NodeDocument;
 import org.apache.jackrabbit.oak.plugins.document.Revision;
 import org.apache.jackrabbit.oak.plugins.document.RevisionVector;
@@ -931,4 +933,39 @@ public class Utils {
         }
         return v;
     }
+
+    /**
+     * Check the revision age on the root document for the given cluster node
+     * info. The check will fail with a {@link DocumentStoreException} if the
+     * {@code _lastRev} timestamp for the cluster node is newer then the current
+     * {@code clock} time. The check will not fail if the root document does
+     * not exist or does not have a {@code _lastRev} entry for the cluster node.
+     *
+     * @param store the document store from where to read the root document.
+     * @param info the cluster node info with the clusterId.
+     * @param clock the clock to get the current time.
+     * @throws DocumentStoreException if the check fails.
+     */
+    public static void checkRevisionAge(DocumentStore store,
+                                        ClusterNodeInfo info,
+                                        Clock clock)
+            throws DocumentStoreException {
+        NodeDocument root = store.find(Collection.NODES, getIdFromPath("/"));
+        if (root == null) {
+            return;
+        }
+        int clusterId = info.getId();
+        Revision rev = root.getLastRev().get(clusterId);
+        if (rev == null) {
+            return;
+        }
+        long now = clock.getTime();
+        if (rev.getTimestamp() > now) {
+            String msg = String.format("Cluster id %d has a _lastRev %s (%s) " +
+                    "newer than current time %s. Please check system time on " +
+                    "cluster nodes.", clusterId, rev.toString(),
+                    timestampToString(rev.getTimestamp()), timestampToString(now));
+            throw new DocumentStoreException(msg);
+        }
+    }
 }

Modified: jackrabbit/oak/trunk/oak-store-document/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStoreTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-store-document/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStoreTest.java?rev=1836251&r1=1836250&r2=1836251&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-store-document/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStoreTest.java (original)
+++ jackrabbit/oak/trunk/oak-store-document/src/test/java/org/apache/jackrabbit/oak/plugins/document/DocumentNodeStoreTest.java Thu Jul 19 07:32:01 2018
@@ -21,6 +21,7 @@ import static java.util.Collections.empt
 import static java.util.Collections.synchronizedList;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.jackrabbit.oak.api.CommitFailedException.CONSTRAINT;
+import static org.apache.jackrabbit.oak.plugins.document.Collection.CLUSTER_NODES;
 import static org.apache.jackrabbit.oak.plugins.document.Collection.JOURNAL;
 import static org.apache.jackrabbit.oak.plugins.document.Collection.NODES;
 import static org.apache.jackrabbit.oak.plugins.document.Collection.SETTINGS;
@@ -3777,6 +3778,45 @@ public class DocumentNodeStoreTest {
         }
     }
 
+    @Test
+    public void preventCommitInPast() throws Exception {
+        Clock clock = new Clock.Virtual();
+        clock.waitUntil(System.currentTimeMillis());
+        Revision.setClock(clock);
+        ClusterNodeInfo.setClock(clock);
+
+        DocumentStore store = new MemoryDocumentStore();
+        DocumentNodeStore ns = builderProvider.newBuilder()
+                .setLeaseCheckMode(LeaseCheckMode.LENIENT)
+                .setClusterId(1).clock(clock)
+                .setDocumentStore(store).build();
+        // wait two minutes
+        clock.waitUntil(clock.getTime() + TimeUnit.MINUTES.toMillis(2));
+        doSomeChange(ns);
+        ns.dispose();
+
+        long now = clock.getTime();
+        // rewind time by one minute
+        clock = new Clock.Virtual();
+        clock.waitUntil(now - TimeUnit.MINUTES.toMillis(1));
+        Revision.setClock(clock);
+        ClusterNodeInfo.setClock(clock);
+
+        try {
+            builderProvider.newBuilder()
+                    .setClusterId(1).clock(clock)
+                    .setDocumentStore(store).build();
+            fail("must fail with DocumentStoreException");
+        } catch (DocumentStoreException e) {
+            assertThat(e.getMessage(), containsString("newer than current time"));
+        }
+
+        // cluster node info 1 must not be active
+        ClusterNodeInfoDocument info = store.find(CLUSTER_NODES, "1");
+        assertNotNull(info);
+        assertFalse(info.isActive());
+    }
+
     private void getChildNodeCountTest(int numChildren,
                                        Iterable<Long> maxValues,
                                        Iterable<Long> expectedValues)

Modified: jackrabbit/oak/trunk/oak-store-document/src/test/java/org/apache/jackrabbit/oak/plugins/document/util/UtilsTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-store-document/src/test/java/org/apache/jackrabbit/oak/plugins/document/util/UtilsTest.java?rev=1836251&r1=1836250&r2=1836251&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-store-document/src/test/java/org/apache/jackrabbit/oak/plugins/document/util/UtilsTest.java (original)
+++ jackrabbit/oak/trunk/oak-store-document/src/test/java/org/apache/jackrabbit/oak/plugins/document/util/UtilsTest.java Thu Jul 19 07:32:01 2018
@@ -28,8 +28,12 @@ import com.google.common.collect.Iterabl
 import org.apache.commons.codec.binary.Hex;
 import org.apache.jackrabbit.oak.api.CommitFailedException;
 import org.apache.jackrabbit.oak.commons.PathUtils;
+import org.apache.jackrabbit.oak.plugins.document.ClusterNodeInfo;
+import org.apache.jackrabbit.oak.plugins.document.Collection;
 import org.apache.jackrabbit.oak.plugins.document.DocumentMK;
 import org.apache.jackrabbit.oak.plugins.document.DocumentNodeStore;
+import org.apache.jackrabbit.oak.plugins.document.DocumentStore;
+import org.apache.jackrabbit.oak.plugins.document.DocumentStoreException;
 import org.apache.jackrabbit.oak.plugins.document.NodeDocument;
 import org.apache.jackrabbit.oak.plugins.document.Revision;
 import org.apache.jackrabbit.oak.plugins.document.RevisionVector;
@@ -43,15 +47,20 @@ import org.apache.jackrabbit.oak.stats.C
 import org.junit.Ignore;
 import org.junit.Test;
 
+import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.Matchers.greaterThan;
 import static org.hamcrest.Matchers.lessThan;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 /**
  * Tests for {@link Utils}.
@@ -344,4 +353,43 @@ public class UtilsTest {
         Utils.closeIfCloseable(Utils.abortingIterable(iterable, s -> true));
         assertTrue(closed.get());
     }
+
+    @Test
+    public void checkRevisionAge() throws Exception {
+        DocumentStore store = new MemoryDocumentStore();
+        ClusterNodeInfo info = mock(ClusterNodeInfo.class);
+        when(info.getId()).thenReturn(2);
+
+        Clock clock = new Clock.Virtual();
+        clock.waitUntil(System.currentTimeMillis());
+
+        // store is empty -> fine
+        Utils.checkRevisionAge(store, info, clock);
+
+        UpdateOp op = new UpdateOp(Utils.getIdFromPath("/"), true);
+        NodeDocument.setLastRev(op, new Revision(clock.getTime(), 0, 1));
+        assertTrue(store.create(Collection.NODES, Collections.singletonList(op)));
+
+        // root document does not have a lastRev entry for clusterId 2
+        Utils.checkRevisionAge(store, info, clock);
+
+        long lastRevTime = clock.getTime();
+        op = new UpdateOp(Utils.getIdFromPath("/"), false);
+        NodeDocument.setLastRev(op, new Revision(lastRevTime, 0, 2));
+        assertNotNull(store.findAndUpdate(Collection.NODES, op));
+
+        // lastRev entry for clusterId 2 is older than current time
+        Utils.checkRevisionAge(store, info, clock);
+
+        // rewind time
+        clock = new Clock.Virtual();
+        clock.waitUntil(lastRevTime - 1000);
+        try {
+            // now the check must fail
+            Utils.checkRevisionAge(store, info, clock);
+            fail("must fail with DocumentStoreException");
+        } catch (DocumentStoreException e) {
+            assertThat(e.getMessage(), containsString("newer than current time"));
+        }
+    }
 }