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 al...@apache.org on 2014/08/21 16:32:01 UTC

svn commit: r1619411 - in /jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment: SegmentBlob.java file/CompactionGainEstimate.java file/FileStore.java file/TarEntryVisitor.java file/TarReader.java file/TarWriter.java

Author: alexparvulescu
Date: Thu Aug 21 14:32:01 2014
New Revision: 1619411

URL: http://svn.apache.org/r1619411
Log:
OAK-2019 Compact only if needed
 - patch by Jukka with minor additions


Added:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/CompactionGainEstimate.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarEntryVisitor.java   (with props)
Modified:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentBlob.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/FileStore.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarReader.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarWriter.java

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentBlob.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentBlob.java?rev=1619411&r1=1619410&r2=1619411&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentBlob.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentBlob.java Thu Aug 21 14:32:01 2014
@@ -17,6 +17,8 @@
 package org.apache.jackrabbit.oak.plugins.segment;
 
 import static com.google.common.base.Charsets.UTF_8;
+import static com.google.common.collect.Sets.newIdentityHashSet;
+import static java.util.Collections.emptySet;
 import static org.apache.jackrabbit.oak.plugins.segment.Segment.MEDIUM_LIMIT;
 import static org.apache.jackrabbit.oak.plugins.segment.Segment.SMALL_LIMIT;
 import static org.apache.jackrabbit.oak.plugins.segment.SegmentWriter.BLOCK_SIZE;
@@ -24,6 +26,7 @@ import static org.apache.jackrabbit.oak.
 import java.io.BufferedInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.Set;
 
 import javax.annotation.CheckForNull;
 import javax.annotation.Nonnull;
@@ -37,6 +40,14 @@ import org.apache.jackrabbit.oak.spi.blo
  */
 public class SegmentBlob extends Record implements Blob {
 
+    public static Iterable<SegmentId> getBulkSegmentIds(Blob blob) {
+        if (blob instanceof SegmentBlob) {
+            return ((SegmentBlob) blob).getBulkSegmentIds();
+        } else {
+            return emptySet();
+        }
+    }
+
     SegmentBlob(RecordId id) {
         super(id);
     }
@@ -207,4 +218,24 @@ public class SegmentBlob extends Record 
         return new String(bytes, UTF_8);
     }
 
+    private Iterable<SegmentId> getBulkSegmentIds() {
+        Segment segment = getSegment();
+        int offset = getOffset();
+        byte head = segment.readByte(offset);
+        if ((head & 0xe0) == 0xc0) {
+            // 110x xxxx: long value
+            long length = (segment.readLong(offset) & 0x1fffffffffffffffL) + MEDIUM_LIMIT;
+            int listSize = (int) ((length + BLOCK_SIZE - 1) / BLOCK_SIZE);
+            ListRecord list = new ListRecord(
+                    segment.readRecordId(offset + 8), listSize);
+            Set<SegmentId> ids = newIdentityHashSet();
+            for (RecordId id : list.getEntries()) {
+                ids.add(id.getSegmentId());
+            }
+            return ids;
+        } else {
+            return emptySet();
+        }
+    }
+
 }

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/CompactionGainEstimate.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/CompactionGainEstimate.java?rev=1619411&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/CompactionGainEstimate.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/CompactionGainEstimate.java Thu Aug 21 14:32:01 2014
@@ -0,0 +1,101 @@
+/*
+ * 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.oak.plugins.segment.file;
+
+import static org.apache.jackrabbit.oak.api.Type.BINARIES;
+
+import java.io.File;
+import java.util.UUID;
+
+import org.apache.jackrabbit.oak.api.Blob;
+import org.apache.jackrabbit.oak.api.PropertyState;
+import org.apache.jackrabbit.oak.plugins.segment.SegmentBlob;
+import org.apache.jackrabbit.oak.plugins.segment.SegmentId;
+import org.apache.jackrabbit.oak.spi.state.ChildNodeEntry;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnel;
+import com.google.common.hash.PrimitiveSink;
+
+class CompactionGainEstimate implements TarEntryVisitor {
+
+    private static final Funnel<UUID> UUID_FUNNEL = new Funnel<UUID>() {
+        @Override
+        public void funnel(UUID from, PrimitiveSink into) {
+            into.putLong(from.getMostSignificantBits());
+            into.putLong(from.getLeastSignificantBits());
+        }
+    };
+
+    private final BloomFilter<UUID> uuids;
+
+    private long totalSize = 0;
+
+    private long reachableSize = 0;
+
+    CompactionGainEstimate(NodeState node, int estimatedBulkCount) {
+        uuids = BloomFilter.create(UUID_FUNNEL, estimatedBulkCount);
+        collectBulkSegments(node);
+    }
+
+    private void collectBulkSegments(NodeState node) {
+        for (PropertyState property : node.getProperties()) {
+            for (Blob blob : property.getValue(BINARIES)) {
+                for (SegmentId id : SegmentBlob.getBulkSegmentIds(blob)) {
+                    uuids.put(new UUID(
+                            id.getMostSignificantBits(),
+                            id.getLeastSignificantBits()));
+                }
+            }
+        }
+        for (ChildNodeEntry child : node.getChildNodeEntries()) {
+            collectBulkSegments(child.getNodeState());
+        }
+    }
+
+    /**
+     * Returns a percentage estimate (scale 0-100) for how much disk space
+     * running compaction (and cleanup) could potentially release.
+     *
+     * @return percentage of disk space that could be freed with compaction
+     */
+    public long estimateCompactionGain() {
+        return 100 * (totalSize - reachableSize) / totalSize;
+    }
+
+    public long getTotalSize() {
+        return totalSize;
+    }
+
+    public long getReachableSize() {
+        return reachableSize;
+    }
+
+    //---------------------------------------------------< TarEntryVisitor >--
+
+    @Override
+    public void visit(long msb, long lsb, File file, int offset, int size) {
+        int entrySize = TarReader.getEntrySize(size);
+        totalSize += entrySize;
+        if (SegmentId.isDataSegmentId(lsb)
+                || uuids.mightContain(new UUID(msb, lsb))) {
+            reachableSize += entrySize;
+        }
+    }
+
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/CompactionGainEstimate.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/FileStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/FileStore.java?rev=1619411&r1=1619410&r2=1619411&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/FileStore.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/FileStore.java Thu Aug 21 14:32:01 2014
@@ -134,6 +134,11 @@ public class FileStore implements Segmen
     private final AtomicBoolean cleanupNeeded = new AtomicBoolean(false);
 
     /**
+     * Flag to set the compaction on pause.
+     */
+    private volatile boolean pauseCompaction = true;
+
+    /**
      * List of old tar file generations that are waiting to be removed. They can
      * not be removed immediately, because they first need to be closed, and the
      * JVM needs to release the memory mapped file references.
@@ -249,7 +254,30 @@ public class FileStore implements Segmen
                 new Runnable() {
                     @Override
                     public void run() {
-                        compact();
+                        log.info("TarMK compaction started");
+                        long time = System.currentTimeMillis();
+                        CompactionGainEstimate estimate = estimateCompactionGain();
+                        long gain = estimate.estimateCompactionGain();
+                        time = System.currentTimeMillis() - time;
+                        if (gain >= 10) {
+                            log.info(
+                                    "Estimated compaction in {}ms, gain is {}% ({}/{}), so running compaction",
+                                    new Object[] { time, gain,
+                                            estimate.getReachableSize(),
+                                            estimate.getTotalSize() });
+                            if (!pauseCompaction) {
+                                compact();
+                            } else {
+                                log.info("TarMK compaction paused");
+                            }
+                        } else {
+                            log.info(
+                                    "Estimated compaction in {}ms, gain is {}% ({}/{}), so skipping compaction for now",
+                                    new Object[] { time, gain,
+                                            estimate.getReachableSize(),
+                                            estimate.getTotalSize() });
+                        }
+                        cleanupNeeded.set(true);
                     }
                 });
 
@@ -336,6 +364,30 @@ public class FileStore implements Segmen
         return size;
     }
 
+    /**
+     * Returns the number of segments in this TarMK instance.
+     *
+     * @return number of segments
+     */
+    private synchronized int count() {
+        int count = writer.count();
+        for (TarReader reader : readers) {
+            count += reader.count();
+        }
+        return count;
+    }
+
+    CompactionGainEstimate estimateCompactionGain() {
+        CompactionGainEstimate estimate = new CompactionGainEstimate(getHead(),
+                count());
+        synchronized (this) {
+            for (TarReader reader : readers) {
+                reader.accept(estimate);
+            }
+        }
+        return estimate;
+    }
+
     public void flush() throws IOException {
         synchronized (persistedHead) {
             RecordId before = persistedHead.get();
@@ -429,7 +481,7 @@ public class FileStore implements Segmen
      */
     public void compact() {
         long start = System.nanoTime();
-        log.info("TarMK compaction started");
+        log.info("TarMK compaction running");
 
         SegmentWriter writer = new SegmentWriter(this, tracker);
         Compactor compactor = new Compactor(writer);
@@ -456,7 +508,6 @@ public class FileStore implements Segmen
 
         log.info("TarMK compaction completed in {}ms", MILLISECONDS
                 .convert(System.nanoTime() - start, NANOSECONDS));
-        cleanupNeeded.set(true);
     }
 
     public synchronized Iterable<SegmentId> getSegmentIds() {
@@ -658,4 +709,8 @@ public class FileStore implements Segmen
         return index;
     }
 
+    public void setPauseCompaction(boolean pauseCompaction) {
+        this.pauseCompaction = pauseCompaction;
+    }
+
 }

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarEntryVisitor.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarEntryVisitor.java?rev=1619411&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarEntryVisitor.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarEntryVisitor.java Thu Aug 21 14:32:01 2014
@@ -0,0 +1,25 @@
+/*
+ * 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.oak.plugins.segment.file;
+
+import java.io.File;
+
+interface TarEntryVisitor {
+
+    void visit(long msb, long lsb, File file, int offset, int size);
+
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarEntryVisitor.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarReader.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarReader.java?rev=1619411&r1=1619410&r2=1619411&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarReader.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarReader.java Thu Aug 21 14:32:01 2014
@@ -67,7 +67,7 @@ class TarReader {
     /** The tar file block size. */
     private static final int BLOCK_SIZE = TarWriter.BLOCK_SIZE;
 
-    private static final int getEntrySize(int size) {
+    static int getEntrySize(int size) {
         return BLOCK_SIZE + size + TarWriter.getPaddingSize(size);
     }
 
@@ -485,6 +485,34 @@ class TarReader {
         return file.length();
     }
 
+    /**
+     * Returns the number of segments in this tar file.
+     *
+     * @return number of segments
+     */
+    int count() {
+        return index.capacity() / 24;
+    }
+
+    /**
+     * Iterates over all entries in this tar file and calls
+     * {@link TarEntryVisitor#visit(long, long, File, int, int)} on them.
+     *
+     * @param visitor entry visitor
+     */
+    void accept(TarEntryVisitor visitor) {
+        int position = index.position();
+        while (position < index.limit()) {
+            visitor.visit(
+                    index.getLong(position),
+                    index.getLong(position + 8),
+                    file,
+                    index.getInt(position + 16),
+                    index.getInt(position + 20));
+            position += 24;
+        }
+    }
+
     Set<UUID> getUUIDs() {
         Set<UUID> uuids = newHashSetWithExpectedSize(index.remaining() / 24);
         int position = index.position();

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarWriter.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarWriter.java?rev=1619411&r1=1619410&r2=1619411&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarWriter.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/file/TarWriter.java Thu Aug 21 14:32:01 2014
@@ -143,6 +143,15 @@ class TarWriter {
         this.file = file;
     }
 
+    /**
+     * Returns the number of segments written so far to this tar file.
+     *
+     * @return number of segments written so far
+     */
+    synchronized int count() {
+        return index.size();
+    }
+
     synchronized Set<UUID> getUUIDs() {
         return newHashSet(index.keySet());
     }