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 md...@apache.org on 2016/04/20 12:04:25 UTC

svn commit: r1740091 - in /jackrabbit/oak/trunk/oak-segment-next/src: main/java/org/apache/jackrabbit/oak/plugins/segment/ main/java/org/apache/jackrabbit/oak/plugins/segment/compaction/ main/java/org/apache/jackrabbit/oak/plugins/segment/file/ test/ja...

Author: mduerig
Date: Wed Apr 20 10:04:24 2016
New Revision: 1740091

URL: http://svn.apache.org/viewvc?rev=1740091&view=rev
Log:
OAK-3348: Cross gc sessions might introduce references to pre-compacted segments
* Remove SetHead closure as atomicity of compaction is now implicit (the GC generation is tied to the head via its segment and there is no need to synchronise caches and the compaction map anymore)
* Implement exclusive locking for file store for force compacting

Modified:
    jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStore.java
    jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStoreService.java
    jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/compaction/CompactionStrategy.java
    jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/FileStore.java
    jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/CompactionAndCleanupIT.java
    jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/HeavyWriteIT.java
    jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentDataStoreBlobGCIT.java
    jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentIdTableTest.java
    jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentOverflowExceptionIT.java
    jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentVersionTest.java

Modified: jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStore.java?rev=1740091&r1=1740090&r2=1740091&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStore.java (original)
+++ jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStore.java Wed Apr 20 10:04:24 2016
@@ -106,19 +106,11 @@ public class SegmentNodeStore implements
                     cloneBinaries,
                     CompactionStrategy.CleanupType.valueOf(cleanup),
                     cleanupTs,
-                    memoryThreshold) {
-
-                @Override
-                public boolean compacted(Callable<Boolean> setHead) throws Exception {
-                    // Need to guard against concurrent commits to avoid
-                    // mixed segments. See OAK-2192.
-                    return segmentNodeStore.locked(setHead, lockWaitTime, SECONDS);
-                }
-
-            };
+                    memoryThreshold);
 
             compactionStrategy.setRetryCount(retryCount);
             compactionStrategy.setForceAfterFail(forceAfterFail);
+            compactionStrategy.setLockWaitTime(lockWaitTime);
             compactionStrategy.setGainThreshold(gainThreshold);
 
             return this;

Modified: jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStoreService.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStoreService.java?rev=1740091&r1=1740090&r2=1740091&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStoreService.java (original)
+++ jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStoreService.java Wed Apr 20 10:04:24 2016
@@ -18,7 +18,6 @@ package org.apache.jackrabbit.oak.plugin
 
 import static com.google.common.base.Preconditions.checkState;
 import static java.util.Collections.emptyMap;
-import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.jackrabbit.oak.commons.PropertiesUtil.toBoolean;
 import static org.apache.jackrabbit.oak.commons.PropertiesUtil.toInteger;
 import static org.apache.jackrabbit.oak.commons.PropertiesUtil.toLong;
@@ -42,7 +41,6 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.Dictionary;
 import java.util.Hashtable;
-import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.felix.scr.annotations.Activate;
@@ -488,26 +486,11 @@ public class SegmentNodeStoreService ext
         byte memoryThreshold = getMemoryThreshold();
         byte gainThreshold = getGainThreshold();
 
-        // This is indeed a dirty hack, but it's needed to break a circular
-        // dependency between different components. The FileStore needs the
-        // CompactionStrategy, the CompactionStrategy needs the
-        // SegmentNodeStore, and the SegmentNodeStore needs the FileStore.
-
-        CompactionStrategy compactionStrategy = new CompactionStrategy(pauseCompaction, cloneBinaries, cleanupType, cleanupTs, memoryThreshold) {
-
-            @Override
-            public boolean compacted(Callable<Boolean> setHead) throws Exception {
-                // Need to guard against concurrent commits to avoid
-                // mixed segments. See OAK-2192.
-                return segmentNodeStore.locked(setHead, lockWaitTime, SECONDS);
-            }
-
-        };
-
+        CompactionStrategy compactionStrategy = new CompactionStrategy(pauseCompaction, cloneBinaries, cleanupType, cleanupTs, memoryThreshold);
         compactionStrategy.setRetryCount(retryCount);
         compactionStrategy.setForceAfterFail(forceAfterFail);
         compactionStrategy.setGainThreshold(gainThreshold);
-
+        compactionStrategy.setLockWaitTime(lockWaitTime);
         return compactionStrategy;
     }
 

Modified: jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/compaction/CompactionStrategy.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/compaction/CompactionStrategy.java?rev=1740091&r1=1740090&r2=1740091&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/compaction/CompactionStrategy.java (original)
+++ jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/compaction/CompactionStrategy.java Wed Apr 20 10:04:24 2016
@@ -22,15 +22,13 @@ import static com.google.common.base.Pre
 import static com.google.common.base.Preconditions.checkNotNull;
 import static java.lang.System.currentTimeMillis;
 
-import java.util.concurrent.Callable;
-
 import javax.annotation.Nonnull;
 
 import org.apache.jackrabbit.oak.plugins.segment.SegmentId;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public abstract class CompactionStrategy {
+public class CompactionStrategy {
     private static final Logger LOG = LoggerFactory.getLogger(CompactionStrategy.class);
 
     public enum CleanupType {
@@ -93,12 +91,7 @@ public abstract class CompactionStrategy
      * No compaction at all
      */
     public static final CompactionStrategy NO_COMPACTION = new CompactionStrategy(
-            true, false, CleanupType.CLEAN_NONE, 0, MEMORY_THRESHOLD_DEFAULT) {
-        @Override
-        public boolean compacted(@Nonnull Callable<Boolean> setHead) throws Exception {
-            return false;
-        }
-    };
+            true, false, CleanupType.CLEAN_NONE, 0, MEMORY_THRESHOLD_DEFAULT);
 
     private boolean paused;
 
@@ -121,6 +114,8 @@ public abstract class CompactionStrategy
 
     private boolean forceAfterFail = FORCE_AFTER_FAIL_DEFAULT;
 
+    private int lockWaitTime = 60;
+
     private long compactionStart = currentTimeMillis();
 
     /**
@@ -128,7 +123,7 @@ public abstract class CompactionStrategy
      */
     private byte gainThreshold = GAIN_THRESHOLD_DEFAULT;
 
-    protected CompactionStrategy(boolean paused,
+    public CompactionStrategy(boolean paused,
             boolean cloneBinaries, @Nonnull CleanupType cleanupType, long olderThan, byte memoryThreshold) {
         checkArgument(olderThan >= 0);
         this.paused = paused;
@@ -236,6 +231,14 @@ public abstract class CompactionStrategy
         this.forceAfterFail = forceAfterFail;
     }
 
+    public void setLockWaitTime(int lockWaitTime) {
+        this.lockWaitTime = lockWaitTime;
+    }
+
+    public int getLockWaitTime() {
+        return lockWaitTime;
+    }
+
     /**
      * Get the number of tries to compact concurrent commits on top of already
      * compacted commits
@@ -272,8 +275,6 @@ public abstract class CompactionStrategy
         this.gainThreshold = gainThreshold;
     }
 
-    public abstract boolean compacted(@Nonnull Callable<Boolean> setHead) throws Exception;
-
     /**
      * Check if the approximate repository size is getting too big compared with
      * the available space on disk.

Modified: jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/FileStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/FileStore.java?rev=1740091&r1=1740090&r2=1740091&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/FileStore.java (original)
+++ jackrabbit/oak/trunk/oak-segment-next/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/FileStore.java Wed Apr 20 10:04:24 2016
@@ -25,6 +25,7 @@ import static com.google.common.collect.
 import static com.google.common.collect.Maps.newLinkedHashMap;
 import static com.google.common.collect.Sets.newHashSet;
 import static java.lang.String.format;
+import static java.lang.Thread.currentThread;
 import static java.util.Collections.emptyMap;
 import static java.util.Collections.singletonMap;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
@@ -47,7 +48,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
@@ -1028,56 +1029,64 @@ public class FileStore implements Segmen
             return;
         }
 
-        Callable<Boolean> setHead = new SetHead(before, after, compactor);
         try {
             int cycles = 0;
             boolean success = false;
-            while(cycles++ < compactionStrategy.getRetryCount()
-                    && !(success = compactionStrategy.compacted(setHead))) {
+            while (cycles++ < compactionStrategy.getRetryCount()
+                && !(success = setHead(before, after))) {
                 // Some other concurrent changes have been made.
                 // Rebase (and compact) those changes on top of the
                 // compacted state before retrying to set the head.
                 gcMonitor.info("TarMK GC #{}: compaction detected concurrent commits while compacting. " +
-                        "Compacting these commits. Cycle {}", gcCount, cycles);
+                    "Compacting these commits. Cycle {}", gcCount, cycles);
                 SegmentNodeState head = getHead();
                 after = compactor.compact(before, head, after);
                 gcMonitor.info("TarMK GC #{}: compacted {} against {} to {}",
                     gcCount, head.getRecordId(), before.getRecordId(), after.getRecordId());
+                before = head;
 
                 if (compactionCanceled.get()) {
                     gcMonitor.warn("TarMK GC #{}: compaction canceled: {}", gcCount, compactionCanceled);
                     return;
                 }
-
-                before = head;
-                setHead = new SetHead(head, after, compactor);
             }
-            if (!success) {
+            if (success) {
+                tracker.clearSegmentIdTables(compactionStrategy);
+                gcMonitor.compacted(new long[0], new long[0], new long[0]);
+            } else {
                 gcMonitor.info("TarMK GC #{}: compaction gave up compacting concurrent commits after {} cycles.",
-                        gcCount, cycles - 1);
+                    gcCount, cycles - 1);
                 if (compactionStrategy.getForceAfterFail()) {
                     gcMonitor.info("TarMK GC #{}: compaction force compacting remaining commits", gcCount);
                     if (!forceCompact(before, after, compactor)) {
                         gcMonitor.warn("TarMK GC #{}: compaction failed to force compact remaining commits. " +
-                                "Most likely compaction didn't get exclusive access to the store.", gcCount);
+                            "Most likely compaction didn't get exclusive access to the store.", gcCount);
                     }
                 }
             }
 
             gcMonitor.info("TarMK GC #{}: compaction completed in {} ({} ms), after {} cycles",
-                    gcCount, watch, watch.elapsed(MILLISECONDS), cycles - 1);
+                gcCount, watch, watch.elapsed(MILLISECONDS), cycles - 1);
+        } catch (InterruptedException e) {
+            gcMonitor.error("TarMK GC #" + gcCount + ": compaction interrupted", e);
+            currentThread().interrupt();
         } catch (Exception e) {
             gcMonitor.error("TarMK GC #" + gcCount + ": compaction encountered an error", e);
         }
     }
 
-    private boolean forceCompact(final NodeState before, final SegmentNodeState onto, final Compactor compactor) throws Exception {
-        return compactionStrategy.compacted(new Callable<Boolean>() {
-            @Override
-            public Boolean call() throws Exception {
-                return new SetHead(getHead(), compactor.compact(before, getHead(), onto), compactor).call();
+    private boolean forceCompact(NodeState before, SegmentNodeState onto, Compactor compactor)
+            throws InterruptedException, IOException {
+        if (rwLock.writeLock().tryLock(compactionStrategy.getLockWaitTime(), TimeUnit.SECONDS)) {
+            try {
+                SegmentNodeState head = getHead();
+                return setHead(head, compactor.compact(before, head, onto));
+            } finally {
+                rwLock.writeLock().unlock();
             }
-        });
+        } else {
+            return false;
+        }
     }
 
     public Iterable<SegmentId> getSegmentIds() {
@@ -1114,11 +1123,18 @@ public class FileStore implements Segmen
         return new SegmentNodeState(head.get());
     }
 
+    private ReadWriteLock rwLock = new ReentrantReadWriteLock();
+
     @Override
     public boolean setHead(SegmentNodeState base, SegmentNodeState head) {
-        RecordId id = this.head.get();
-        return id.equals(base.getRecordId())
+        rwLock.readLock().lock();
+        try {
+            RecordId id = this.head.get();
+            return id.equals(base.getRecordId())
                 && this.head.compareAndSet(id, head.getRecordId());
+        } finally {
+            rwLock.readLock().unlock();
+        }
     }
 
     @Override
@@ -1454,32 +1470,6 @@ public class FileStore implements Segmen
         }
     }
 
-    private class SetHead implements Callable<Boolean> {
-        private final SegmentNodeState before;
-        private final SegmentNodeState after;
-        private final Compactor compactor;
-
-        public SetHead(SegmentNodeState before, SegmentNodeState after, Compactor compactor) {
-            this.before = before;
-            this.after = after;
-            this.compactor = compactor;
-        }
-
-        @Override
-        public Boolean call() throws Exception {
-            // When used in conjunction with the SegmentNodeStore, this method
-            // needs to be called inside the commitSemaphore as doing otherwise
-            // might result in mixed segments. See OAK-2192.
-            if (setHead(before, after)) {
-                tracker.clearSegmentIdTables(compactionStrategy);
-                gcMonitor.compacted(new long[0], new long[0], new long[0]);
-                return true;
-            } else {
-                return false;
-            }
-        }
-    }
-
     public SegmentVersion getVersion() {
         return version;
     }

Modified: jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/CompactionAndCleanupIT.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/CompactionAndCleanupIT.java?rev=1740091&r1=1740090&r2=1740091&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/CompactionAndCleanupIT.java (original)
+++ jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/CompactionAndCleanupIT.java Wed Apr 20 10:04:24 2016
@@ -21,7 +21,6 @@ package org.apache.jackrabbit.oak.plugin
 
 import static com.google.common.collect.Lists.newArrayList;
 import static java.lang.Integer.getInteger;
-import static java.util.concurrent.TimeUnit.MINUTES;
 import static org.apache.commons.io.FileUtils.byteCountToDisplaySize;
 import static org.apache.jackrabbit.oak.api.Type.STRING;
 import static org.apache.jackrabbit.oak.commons.FixturesHelper.Fixture.SEGMENT_MK;
@@ -52,8 +51,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
-import javax.annotation.Nonnull;
-
 import com.google.common.io.ByteStreams;
 import org.apache.jackrabbit.oak.api.Blob;
 import org.apache.jackrabbit.oak.api.CommitFailedException;
@@ -105,13 +102,7 @@ public class CompactionAndCleanupIT {
                 .build();
         final SegmentNodeStore nodeStore = SegmentNodeStore.builder(fileStore).build();
         CompactionStrategy custom = new CompactionStrategy(false, false,
-                CLEAN_OLD, TimeUnit.HOURS.toMillis(1), (byte) 0) {
-            @Override
-            public boolean compacted(@Nonnull Callable<Boolean> setHead)
-                    throws Exception {
-                return nodeStore.locked(setHead);
-            }
-        };
+                CLEAN_OLD, TimeUnit.HOURS.toMillis(1), (byte) 0);
         // Use in memory compaction map as gains asserted later on
         // do not take additional space of the compaction map into consideration
         fileStore.setCompactionStrategy(custom);
@@ -232,13 +223,7 @@ public class CompactionAndCleanupIT {
         FileStore store = FileStore.builder(getFileStoreFolder()).withMaxFileSize(2).withMemoryMapping(true).build();
         final SegmentNodeStore nodeStore = SegmentNodeStore.builder(store).build();
         final AtomicBoolean compactionSuccess = new AtomicBoolean(true);
-        CompactionStrategy strategy = new CompactionStrategy(true, false, CLEAN_NONE, 0, (byte) 5) {
-            @Override
-            public boolean compacted(Callable<Boolean> setHead) throws Exception {
-                compactionSuccess.set(nodeStore.locked(setHead, 1, MINUTES));
-                return compactionSuccess.get();
-            }
-        };
+        CompactionStrategy strategy = new CompactionStrategy(true, false, CLEAN_NONE, 0, (byte) 5);
         strategy.setForceAfterFail(true);
         store.setCompactionStrategy(strategy);
 
@@ -362,13 +347,7 @@ public class CompactionAndCleanupIT {
             File repoDir = new File(getFileStoreFolder(), ref);
             FileStore fileStore = FileStore.builder(repoDir).withMaxFileSize(2).build();
             final SegmentNodeStore nodeStore = builder(fileStore).build();
-            fileStore.setCompactionStrategy(new CompactionStrategy(true, false, CLEAN_NONE, 0, (byte) 5) {
-                @Override
-                public boolean compacted(Callable<Boolean> setHead) throws Exception {
-                    return nodeStore.locked(setHead);
-                }
-            });
-
+            fileStore.setCompactionStrategy(new CompactionStrategy(true, false, CLEAN_NONE, 0, (byte) 5));
             try {
                 // add some content
                 NodeBuilder preGCBuilder = nodeStore.getRoot().builder();
@@ -523,13 +502,7 @@ public class CompactionAndCleanupIT {
         FileStore fileStore = FileStore.builder(getFileStoreFolder()).withMaxFileSize(1).build();
         try {
             final SegmentNodeStore nodeStore = SegmentNodeStore.builder(fileStore).build();
-            CompactionStrategy strategy = new CompactionStrategy(false, false, CLEAN_ALL, 0, (byte) 0) {
-                @Override
-                public boolean compacted(@Nonnull Callable<Boolean> setHead)
-                        throws Exception {
-                    return nodeStore.locked(setHead);
-                }
-            };
+            CompactionStrategy strategy = new CompactionStrategy(false, false, CLEAN_ALL, 0, (byte) 0);
             // CLEAN_ALL and persisted compaction map results in SNFE in compaction map segments
             fileStore.setCompactionStrategy(strategy);
 

Modified: jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/HeavyWriteIT.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/HeavyWriteIT.java?rev=1740091&r1=1740090&r2=1740091&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/HeavyWriteIT.java (original)
+++ jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/HeavyWriteIT.java Wed Apr 20 10:04:24 2016
@@ -29,11 +29,8 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Random;
 import java.util.Set;
-import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import javax.annotation.Nonnull;
-
 import org.apache.jackrabbit.oak.api.Blob;
 import org.apache.jackrabbit.oak.api.CommitFailedException;
 import org.apache.jackrabbit.oak.commons.FixturesHelper;
@@ -70,12 +67,7 @@ public class HeavyWriteIT {
         final FileStore store = FileStore.builder(getFileStoreFolder()).withMaxFileSize(128).withMemoryMapping(false).build();
         final SegmentNodeStore nodeStore = SegmentNodeStore.builder(store).build();
         CompactionStrategy custom = new CompactionStrategy(false, false,
-                CLEAN_OLD, 30000, (byte) 0) {
-            @Override
-            public boolean compacted(@Nonnull Callable<Boolean> setHead) throws Exception {
-                return nodeStore.locked(setHead);
-            }
-        };
+                CLEAN_OLD, 30000, (byte) 0);
         store.setCompactionStrategy(custom);
 
         int writes = 100;

Modified: jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentDataStoreBlobGCIT.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentDataStoreBlobGCIT.java?rev=1740091&r1=1740090&r2=1740091&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentDataStoreBlobGCIT.java (original)
+++ jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentDataStoreBlobGCIT.java Wed Apr 20 10:04:24 2016
@@ -36,13 +36,11 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
 import java.util.Set;
-import java.util.concurrent.Callable;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
-import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
 import com.google.common.base.Stopwatch;
@@ -102,12 +100,7 @@ public class SegmentDataStoreBlobGCIT {
             store = builder.build();
             CompactionStrategy compactionStrategy =
                 new CompactionStrategy(false, true,
-                    CompactionStrategy.CleanupType.CLEAN_OLD, 0, CompactionStrategy.MEMORY_THRESHOLD_DEFAULT) {
-                    @Override
-                    public boolean compacted(@Nonnull Callable<Boolean> setHead) throws Exception {
-                        return setHead.call();
-                    }
-                };
+                    CompactionStrategy.CleanupType.CLEAN_OLD, 0, CompactionStrategy.MEMORY_THRESHOLD_DEFAULT);
             store.setCompactionStrategy(compactionStrategy);
             nodeStore = SegmentNodeStore.builder(store).build();
         }

Modified: jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentIdTableTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentIdTableTest.java?rev=1740091&r1=1740090&r2=1740091&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentIdTableTest.java (original)
+++ jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentIdTableTest.java Wed Apr 20 10:04:24 2016
@@ -29,10 +29,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
-import javax.annotation.Nonnull;
-
 import junit.framework.Assert;
-
 import org.apache.jackrabbit.oak.plugins.segment.compaction.CompactionStrategy;
 import org.apache.jackrabbit.oak.plugins.segment.compaction.CompactionStrategy.CleanupType;
 import org.apache.jackrabbit.oak.plugins.segment.memory.MemoryStore;
@@ -111,13 +108,6 @@ public class SegmentIdTableTest {
         
         tbl.clearSegmentIdTables(new CompactionStrategy(false, false, 
                 CleanupType.CLEAN_NONE, originalCount, (byte) 0) {
-
-            @Override
-            public boolean compacted(@Nonnull Callable<Boolean> setHead)
-                    throws Exception {
-                return true;
-            }
-
             @Override
             public boolean canRemove(SegmentId id) {
                 return id.getMostSignificantBits() < 4;

Modified: jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentOverflowExceptionIT.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentOverflowExceptionIT.java?rev=1740091&r1=1740090&r2=1740091&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentOverflowExceptionIT.java (original)
+++ jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentOverflowExceptionIT.java Wed Apr 20 10:04:24 2016
@@ -29,9 +29,6 @@ import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.IOException;
 import java.util.Random;
-import java.util.concurrent.Callable;
-
-import javax.annotation.Nonnull;
 
 import com.google.common.collect.Iterables;
 import org.apache.jackrabbit.oak.api.Blob;
@@ -104,13 +101,7 @@ public class SegmentOverflowExceptionIT
         FileStore fileStore = FileStore.builder(getFileStoreFolder()).withGCMonitor(gcMonitor).build();
         try {
             final SegmentNodeStore nodeStore = SegmentNodeStore.builder(fileStore).build();
-            fileStore.setCompactionStrategy(new CompactionStrategy(false, false, CLEAN_OLD, 1000, MEMORY_THRESHOLD_DEFAULT) {
-                @Override
-                public boolean compacted(@Nonnull Callable<Boolean> setHead) throws Exception {
-                    return nodeStore.locked(setHead);
-                }
-            });
-
+            fileStore.setCompactionStrategy(new CompactionStrategy(false, false, CLEAN_OLD, 1000, MEMORY_THRESHOLD_DEFAULT));
             long start = System.currentTimeMillis();
             int snfeCount = 0;
             while (System.currentTimeMillis() - start < TIMEOUT) {

Modified: jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentVersionTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentVersionTest.java?rev=1740091&r1=1740090&r2=1740091&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentVersionTest.java (original)
+++ jackrabbit/oak/trunk/oak-segment-next/src/test/java/org/apache/jackrabbit/oak/plugins/segment/SegmentVersionTest.java Wed Apr 20 10:04:24 2016
@@ -32,9 +32,6 @@ import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.concurrent.Callable;
-
-import javax.annotation.Nonnull;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
@@ -170,12 +167,7 @@ public class SegmentVersionTest {
         FileStore fileStoreV11 = FileStore.builder(getFileStoreFolder()).withMaxFileSize(1).build();
         try {
             fileStoreV11.setCompactionStrategy(new CompactionStrategy(false, false,
-                    CLEAN_NONE, 0, (byte) 0) {
-                @Override
-                public boolean compacted(@Nonnull Callable<Boolean> setHead) throws Exception {
-                    return setHead.call();
-                }
-            });
+                    CLEAN_NONE, 0, (byte) 0));
             checkAllVersions(fileStoreV11.getHead(), SegmentVersion.V_10);
             fileStoreV11.compact();
             checkAllVersions(fileStoreV11.getHead(), V_11);