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/06/09 16:02:39 UTC

svn commit: r1601388 - in /jackrabbit/oak/trunk/oak-core/src: main/java/org/apache/jackrabbit/oak/plugins/segment/ main/java/org/apache/jackrabbit/oak/plugins/segment/file/ main/java/org/apache/jackrabbit/oak/plugins/segment/http/ main/java/org/apache/...

Author: alexparvulescu
Date: Mon Jun  9 14:02:38 2014
New Revision: 1601388

URL: http://svn.apache.org/r1601388
Log:
OAK-1804 TarMK compaction
 - the compaction map is now available and used in the fast equals impl

Added:
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/segment/CompactorTest.java   (with props)
Modified:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/Compactor.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/MapRecord.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/Record.java
    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/SegmentNodeState.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStore.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentStore.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/Template.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/http/HttpStore.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/memory/MemoryStore.java

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/Compactor.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/Compactor.java?rev=1601388&r1=1601387&r2=1601388&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/Compactor.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/Compactor.java Mon Jun  9 14:02:38 2014
@@ -24,9 +24,11 @@ import static org.apache.jackrabbit.oak.
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.jackrabbit.oak.api.Blob;
 import org.apache.jackrabbit.oak.api.PropertyState;
@@ -42,6 +44,7 @@ import org.apache.jackrabbit.oak.spi.sta
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.ImmutableSortedMap;
 import com.google.common.hash.Hashing;
 
 /**
@@ -52,7 +55,10 @@ public class Compactor {
     /** Logger instance */
     private static final Logger log = LoggerFactory.getLogger(Compactor.class);
 
-    public static void compact(SegmentStore store) {
+    private static int ENTRY_KEY_SIZE = 20;
+    private static int ENTRY_SIZE = 2 * ENTRY_KEY_SIZE;
+
+    public static ByteBuffer compact(SegmentStore store) {
         SegmentWriter writer = store.getTracker().getWriter();
         Compactor compactor = new Compactor(writer);
 
@@ -74,6 +80,111 @@ public class Compactor {
             before = head;
             after = builder.getNodeState();
         }
+        return mapToByteBuffer(compactor.getCompacted());
+    }
+
+    /**
+     * Serializes the records map to a ByteBuffer, this allows the records to be
+     * GC'ed while maintaining a fast lookup structure.
+     */
+    static ByteBuffer mapToByteBuffer(Map<RecordId, RecordId> in) {
+        ByteBuffer buffer = ByteBuffer.allocate(in.size() * ENTRY_SIZE);
+        Map<RecordId, RecordId> sort = ImmutableSortedMap.copyOf(in);
+        for (Entry<RecordId, RecordId> e : sort.entrySet()) {
+            RecordId k = e.getKey();
+            buffer.putLong(k.getSegmentId().getMostSignificantBits());
+            buffer.putLong(k.getSegmentId().getLeastSignificantBits());
+            buffer.putInt(k.getOffset());
+            RecordId v = e.getValue();
+            buffer.putLong(v.getSegmentId().getMostSignificantBits());
+            buffer.putLong(v.getSegmentId().getLeastSignificantBits());
+            buffer.putInt(v.getOffset());
+        }
+        return buffer;
+    }
+
+    /**
+     * Locks down the RecordId persistence structure
+     */
+    static long[] recordAsKey(RecordId r) {
+        return new long[] { r.getSegmentId().getMostSignificantBits(),
+                r.getSegmentId().getLeastSignificantBits(), r.getOffset() };
+    }
+
+    /**
+     * Looks for the mapping for a given entry, if none is found, it returns the
+     * original key
+     */
+    static long[] readEntry(ByteBuffer compaction, RecordId rid) {
+        long[] entry = recordAsKey(rid);
+        int position = findEntry(compaction, entry[0], entry[1], entry[2]);
+        if (position != -1) {
+            long msb = compaction.getLong(position + ENTRY_KEY_SIZE);
+            long lsb = compaction.getLong(position + ENTRY_KEY_SIZE + 8);
+            long offset = compaction.getInt(position + ENTRY_KEY_SIZE + 16);
+            return new long[] { msb, lsb, offset };
+        }
+        return entry;
+    }
+
+    private static int findEntry(ByteBuffer index, long msb, long lsb, long offset) {
+
+        // this a copy of the TarReader#findEntry with tiny changes around the
+        // entry sizes
+
+        // The segment identifiers are randomly generated with uniform
+        // distribution, so we can use interpolation search to find the
+        // matching entry in the index. The average runtime is O(log log n).
+
+        int entrySize = ENTRY_SIZE;
+
+        int lowIndex = 0;
+        int highIndex = /* index.remaining() / */  index.capacity() / entrySize -1;
+        float lowValue = Long.MIN_VALUE;
+        float highValue = Long.MAX_VALUE;
+        float targetValue = msb;
+
+        while (lowIndex <= highIndex) {
+            int guessIndex = lowIndex + Math.round(
+                    (highIndex - lowIndex)
+                    * (targetValue - lowValue)
+                    / (highValue - lowValue));
+            int position = /* index.position() + */ guessIndex * entrySize;
+            long m = index.getLong(position);
+            if (msb < m) {
+                highIndex = guessIndex - 1;
+                highValue = m;
+            } else if (msb > m) {
+                lowIndex = guessIndex + 1;
+                lowValue = m;
+            } else {
+                // getting close...
+                long l = index.getLong(position + 8);
+                if (lsb < l) {
+                    highIndex = guessIndex - 1;
+                    highValue = m;
+                } else if (lsb > l) {
+                    lowIndex = guessIndex + 1;
+                    lowValue = m;
+                } else {
+                    // getting even closer...
+                    int o = index.getInt(position + 16);
+                    if (offset < o) {
+                        highIndex = guessIndex - 1;
+                        highValue = m;
+                    } else if (offset > o) {
+                        lowIndex = guessIndex + 1;
+                        lowValue = m;
+                    } else {
+                        // found it!
+                        return position;
+                    }
+                }
+            }
+        }
+
+        // not found
+        return -1;
     }
 
     private final SegmentWriter writer;
@@ -255,4 +366,8 @@ public class Compactor {
         }
     }
 
+    private Map<RecordId, RecordId> getCompacted() {
+        return compacted;
+    }
+
 }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/MapRecord.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/MapRecord.java?rev=1601388&r1=1601387&r2=1601388&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/MapRecord.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/MapRecord.java Mon Jun  9 14:02:38 2014
@@ -364,7 +364,7 @@ class MapRecord extends Record {
     }
 
     boolean compare(MapRecord before, final NodeStateDiff diff) {
-        if (fastEquals(this, before)) {
+        if (fastEquals(this, before, getStore())) {
             return true;
         }
 

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/Record.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/Record.java?rev=1601388&r1=1601387&r2=1601388&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/Record.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/Record.java Mon Jun  9 14:02:38 2014
@@ -16,6 +16,10 @@
  */
 package org.apache.jackrabbit.oak.plugins.segment;
 
+import static org.apache.jackrabbit.oak.plugins.segment.Compactor.readEntry;
+
+import java.nio.ByteBuffer;
+
 import javax.annotation.Nonnull;
 
 /**
@@ -23,16 +27,22 @@ import javax.annotation.Nonnull;
  */
 class Record {
 
-    static boolean fastEquals(Object a, Object b) {
-        return a instanceof Record && fastEquals((Record) a, b);
+    static boolean fastEquals(Object a, Object b, SegmentStore store) {
+        return a instanceof Record && fastEquals((Record) a, b, store);
     }
 
-    static boolean fastEquals(Record a, Object b) {
-        return b instanceof Record && fastEquals(a, (Record) b);
+    static boolean fastEquals(Record a, Object b, SegmentStore store) {
+        return b instanceof Record && fastEquals(a, (Record) b, store);
     }
 
-    static boolean fastEquals(Record a, Record b) {
-        return a.segmentId == b.segmentId && a.offset == b.offset;
+    static boolean fastEquals(Record a, Record b, SegmentStore store) {
+        ByteBuffer compaction = store.getCompactionMap();
+        if (compaction == null) {
+            return a.segmentId == b.segmentId && a.offset == b.offset;
+        }
+        long[] aId = readEntry(compaction, a.getRecordId());
+        long[] bId = readEntry(compaction, b.getRecordId());
+        return aId[0] == bId[0] && aId[1] == bId[1] && aId[2] == bId[2];
     }
 
     /**
@@ -69,6 +79,15 @@ class Record {
     }
 
     /**
+     * Returns the segment store.
+     *
+     * @return segment store
+     */
+    public SegmentStore getStore() {
+        return segmentId.getTracker().getStore();
+    }
+
+    /**
      * Returns the identifier of this record.
      *
      * @return record identifier
@@ -113,7 +132,7 @@ class Record {
 
     @Override
     public boolean equals(Object that) {
-        return fastEquals(this, that);
+        return fastEquals(this, that, getStore());
     }
 
     @Override

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=1601388&r1=1601387&r2=1601388&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 Mon Jun  9 14:02:38 2014
@@ -178,7 +178,7 @@ public class SegmentBlob extends Record 
 
     @Override
     public boolean equals(Object object) {
-        if (object == this || fastEquals(this, object)) {
+        if (object == this || fastEquals(this, object, getStore())) {
             return true;
         } else {
             return object instanceof Blob

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeState.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeState.java?rev=1601388&r1=1601387&r2=1601388&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeState.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeState.java Mon Jun  9 14:02:38 2014
@@ -381,7 +381,7 @@ public class SegmentNodeState extends Re
 
     @Override
     public boolean compareAgainstBaseState(NodeState base, NodeStateDiff diff) {
-        if (this == base || fastEquals(this, base)) {
+        if (this == base || fastEquals(this, base, getStore())) {
              return true; // no changes
         } else if (base == EMPTY_NODE || !base.exists()) { // special case
             return EmptyNodeState.compareAgainstEmptyState(this, diff);
@@ -476,7 +476,7 @@ public class SegmentNodeState extends Re
                 if (!diff.childNodeAdded(afterChildName, afterNode)) {
                     return false;
                 }
-            } else if (!fastEquals(afterNode, beforeNode)) {
+            } else if (!fastEquals(afterNode, beforeNode, getStore())) {
                 if (!diff.childNodeChanged(
                         afterChildName, beforeNode, afterNode)) {
                     return false;
@@ -512,7 +512,7 @@ public class SegmentNodeState extends Re
                     NodeState beforeChild =
                             beforeTemplate.getChildNode(beforeChildName, beforeId);
                     if (beforeChild.exists()) {
-                        if (!fastEquals(afterChild, beforeChild)
+                        if (!fastEquals(afterChild, beforeChild, getStore())
                                 && !diff.childNodeChanged(
                                         childName, beforeChild, afterChild)) {
                             return false;
@@ -550,7 +550,7 @@ public class SegmentNodeState extends Re
 
     @Override
     public boolean equals(Object object) {
-        if (this == object || fastEquals(this, object)) {
+        if (this == object || fastEquals(this, object, getStore())) {
             return true;
         } else if (object instanceof SegmentNodeState) {
             SegmentNodeState that = (SegmentNodeState) object;

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStore.java?rev=1601388&r1=1601387&r2=1601388&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStore.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStore.java Mon Jun  9 14:02:38 2014
@@ -150,7 +150,7 @@ public class SegmentNodeStore implements
 
         NodeState root = getRoot();
         SegmentNodeState before = snb.getBaseState();
-        if (!fastEquals(before, root)) {
+        if (!fastEquals(before, root, store)) {
             SegmentNodeState after = snb.getNodeState();
             snb.reset(root);
             after.compareAgainstBaseState(
@@ -328,7 +328,7 @@ public class SegmentNodeStore implements
         private SegmentNodeBuilder prepare() throws CommitFailedException {
             SegmentNodeState state = head.get();
             SegmentNodeBuilder builder = state.builder();
-            if (fastEquals(before, state.getChildNode(ROOT))) {
+            if (fastEquals(before, state.getChildNode(ROOT), store)) {
                 // use a shortcut when there are no external changes
                 builder.setChildNode(
                         ROOT, hook.processCommit(before, after, info));
@@ -416,7 +416,7 @@ public class SegmentNodeStore implements
         NodeState execute()
                 throws CommitFailedException, InterruptedException {
             // only do the merge if there are some changes to commit
-            if (!fastEquals(before, after)) {
+            if (!fastEquals(before, after, store)) {
                 long timeout = optimisticMerge();
                 if (timeout >= 0) {
                     pessimisticMerge(timeout);

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentStore.java?rev=1601388&r1=1601387&r2=1601388&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentStore.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentStore.java Mon Jun  9 14:02:38 2014
@@ -16,6 +16,8 @@
  */
 package org.apache.jackrabbit.oak.plugins.segment;
 
+import java.nio.ByteBuffer;
+
 import javax.annotation.CheckForNull;
 import javax.annotation.Nonnull;
 
@@ -84,4 +86,7 @@ public interface SegmentStore {
      */
     void gc();
 
+    @CheckForNull
+    ByteBuffer getCompactionMap();
+
 }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/Template.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/Template.java?rev=1601388&r1=1601387&r2=1601388&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/Template.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/Template.java Mon Jun  9 14:02:38 2014
@@ -240,7 +240,7 @@ public class Template {
             // TODO: Leverage the HAMT data structure for the comparison
             MapRecord thisMap = getChildNodeMap(thisId);
             MapRecord thatMap = getChildNodeMap(thatId);
-            if (fastEquals(thisMap, thatMap)) {
+            if (fastEquals(thisMap, thatMap, thisMap.getStore())) {
                 return true; // shortcut
             } else if (thisMap.size() != thatMap.size()) {
                 return false; // shortcut

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=1601388&r1=1601387&r2=1601388&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 Mon Jun  9 14:02:38 2014
@@ -137,6 +137,12 @@ public class FileStore implements Segmen
     private int compactThreshold = 10;
 
     /**
+     * Serialized map that contains the link between the old record ids and new record ids
+     * of the compacted states.
+     */
+    private final AtomicReference<ByteBuffer> compactionMap;
+
+    /**
      * List of old tar file generations that are waiting to be removed.
      */
     private final LinkedList<File> toBeRemoved = newLinkedList();
@@ -266,6 +272,8 @@ public class FileStore implements Segmen
         flushThread.setPriority(Thread.MIN_PRIORITY);
         flushThread.start();
 
+        compactionMap = new AtomicReference<ByteBuffer>(null);
+
         log.info("TarMK opened: {} (mmap={})", directory, memoryMapping);
     }
 
@@ -407,10 +415,11 @@ public class FileStore implements Segmen
         }
     }
 
-    private void compact() {
+    public void compact() {
         if (compactNeeded.getAndSet(false)) {
             long start = System.nanoTime();
-            Compactor.compact(this);
+            tracker.getWriter().dropCache();
+            compactionMap.set(Compactor.compact(this));
             log.info("TarMK Compaction: Completed in {}ms", MILLISECONDS
                     .convert(System.nanoTime() - start, NANOSECONDS));
             cleanupNeeded.set(true);
@@ -627,4 +636,9 @@ public class FileStore implements Segmen
         return index;
     }
 
+    @Override
+    public ByteBuffer getCompactionMap() {
+        return compactionMap.get();
+    }
+
 }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/http/HttpStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/http/HttpStore.java?rev=1601388&r1=1601387&r2=1601388&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/http/HttpStore.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/http/HttpStore.java Mon Jun  9 14:02:38 2014
@@ -167,4 +167,9 @@ public class HttpStore implements Segmen
         // TODO: distributed gc
     }
 
+    @Override
+    public ByteBuffer getCompactionMap() {
+        return null;
+    }
+
 }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/memory/MemoryStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/memory/MemoryStore.java?rev=1601388&r1=1601387&r2=1601388&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/memory/MemoryStore.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/memory/MemoryStore.java Mon Jun  9 14:02:38 2014
@@ -1,128 +1,133 @@
-/*
- * 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.memory;
-
-import static org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState.EMPTY_NODE;
-
-import java.nio.ByteBuffer;
-import java.util.concurrent.ConcurrentMap;
-
-import javax.annotation.Nonnull;
-
-import org.apache.jackrabbit.oak.api.Blob;
-import org.apache.jackrabbit.oak.plugins.segment.Segment;
-import org.apache.jackrabbit.oak.plugins.segment.SegmentId;
-import org.apache.jackrabbit.oak.plugins.segment.SegmentTracker;
-import org.apache.jackrabbit.oak.plugins.segment.SegmentNodeState;
-import org.apache.jackrabbit.oak.plugins.segment.SegmentStore;
-import org.apache.jackrabbit.oak.plugins.segment.SegmentWriter;
-import org.apache.jackrabbit.oak.spi.blob.BlobStore;
-import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
-import org.apache.jackrabbit.oak.spi.state.NodeState;
-
-import com.google.common.collect.Maps;
-
-public class MemoryStore implements SegmentStore {
-
-    private final SegmentTracker tracker = new SegmentTracker(this);
-
-    private SegmentNodeState head;
-
-    private final ConcurrentMap<SegmentId, Segment> segments =
-            Maps.newConcurrentMap();
-
-    public MemoryStore(NodeState root) {
-        NodeBuilder builder = EMPTY_NODE.builder();
-        builder.setChildNode("root", root);
-
-        SegmentWriter writer = tracker.getWriter();
-        this.head = writer.writeNode(builder.getNodeState());
-        writer.flush();
-    }
-
-    public MemoryStore() {
-        this(EMPTY_NODE);
-    }
-
-    @Override
-    public SegmentTracker getTracker() {
-        return tracker;
-    }
-
-    @Override
-    public synchronized SegmentNodeState getHead() {
-        return head;
-    }
-
-    @Override
-    public synchronized boolean setHead(SegmentNodeState base, SegmentNodeState head) {
-        if (this.head.getRecordId().equals(base.getRecordId())) {
-            this.head = head;
-            return true;
-        } else {
-            return false;
-        }
-    }
-
-    @Override
-    public boolean containsSegment(SegmentId id) {
-        return id.getTracker() == tracker || segments.containsKey(id);
-    }
-
-    @Override @Nonnull
-    public Segment readSegment(SegmentId id) {
-        Segment segment = segments.get(id);
-        if (segment != null) {
-            return segment;
-        } else {
-            throw new IllegalArgumentException("Segment not found: " + id);
-        }
-    }
-
-    @Override
-    public void writeSegment(
-            SegmentId id, byte[] data, int offset, int length) {
-        ByteBuffer buffer = ByteBuffer.allocate(length);
-        buffer.put(data, offset, length);
-        buffer.rewind();
-        Segment segment = new Segment(tracker, id, buffer);
-        if (segments.putIfAbsent(id, segment) != null) {
-            throw new IllegalStateException("Segment override: " + id);
-        }
-    }
-
-    @Override
-    public void close() {
-    }
-
-    @Override
-    public Blob readBlob(String reference) {
-        return null;
-    }
-
-    @Override
-    public BlobStore getBlobStore() {
-        return null;
-    }
-
-    @Override
-    public void gc() {
-        System.gc();
-        segments.keySet().retainAll(tracker.getReferencedSegmentIds());
-    }
-
-}
+/*
+ * 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.memory;
+
+import static org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState.EMPTY_NODE;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.ConcurrentMap;
+
+import javax.annotation.Nonnull;
+
+import org.apache.jackrabbit.oak.api.Blob;
+import org.apache.jackrabbit.oak.plugins.segment.Segment;
+import org.apache.jackrabbit.oak.plugins.segment.SegmentId;
+import org.apache.jackrabbit.oak.plugins.segment.SegmentTracker;
+import org.apache.jackrabbit.oak.plugins.segment.SegmentNodeState;
+import org.apache.jackrabbit.oak.plugins.segment.SegmentStore;
+import org.apache.jackrabbit.oak.plugins.segment.SegmentWriter;
+import org.apache.jackrabbit.oak.spi.blob.BlobStore;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+
+import com.google.common.collect.Maps;
+
+public class MemoryStore implements SegmentStore {
+
+    private final SegmentTracker tracker = new SegmentTracker(this);
+
+    private SegmentNodeState head;
+
+    private final ConcurrentMap<SegmentId, Segment> segments =
+            Maps.newConcurrentMap();
+
+    public MemoryStore(NodeState root) {
+        NodeBuilder builder = EMPTY_NODE.builder();
+        builder.setChildNode("root", root);
+
+        SegmentWriter writer = tracker.getWriter();
+        this.head = writer.writeNode(builder.getNodeState());
+        writer.flush();
+    }
+
+    public MemoryStore() {
+        this(EMPTY_NODE);
+    }
+
+    @Override
+    public SegmentTracker getTracker() {
+        return tracker;
+    }
+
+    @Override
+    public synchronized SegmentNodeState getHead() {
+        return head;
+    }
+
+    @Override
+    public synchronized boolean setHead(SegmentNodeState base, SegmentNodeState head) {
+        if (this.head.getRecordId().equals(base.getRecordId())) {
+            this.head = head;
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public boolean containsSegment(SegmentId id) {
+        return id.getTracker() == tracker || segments.containsKey(id);
+    }
+
+    @Override @Nonnull
+    public Segment readSegment(SegmentId id) {
+        Segment segment = segments.get(id);
+        if (segment != null) {
+            return segment;
+        } else {
+            throw new IllegalArgumentException("Segment not found: " + id);
+        }
+    }
+
+    @Override
+    public void writeSegment(
+            SegmentId id, byte[] data, int offset, int length) {
+        ByteBuffer buffer = ByteBuffer.allocate(length);
+        buffer.put(data, offset, length);
+        buffer.rewind();
+        Segment segment = new Segment(tracker, id, buffer);
+        if (segments.putIfAbsent(id, segment) != null) {
+            throw new IllegalStateException("Segment override: " + id);
+        }
+    }
+
+    @Override
+    public void close() {
+    }
+
+    @Override
+    public Blob readBlob(String reference) {
+        return null;
+    }
+
+    @Override
+    public BlobStore getBlobStore() {
+        return null;
+    }
+
+    @Override
+    public void gc() {
+        System.gc();
+        segments.keySet().retainAll(tracker.getReferencedSegmentIds());
+    }
+
+    @Override
+    public ByteBuffer getCompactionMap() {
+        return null;
+    }
+
+}

Added: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/segment/CompactorTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/segment/CompactorTest.java?rev=1601388&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/segment/CompactorTest.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/segment/CompactorTest.java Mon Jun  9 14:02:38 2014
@@ -0,0 +1,85 @@
+/*
+ * 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;
+
+import static org.apache.jackrabbit.oak.plugins.segment.Compactor.mapToByteBuffer;
+import static org.apache.jackrabbit.oak.plugins.segment.Compactor.readEntry;
+import static org.apache.jackrabbit.oak.plugins.segment.Compactor.recordAsKey;
+import static org.apache.jackrabbit.oak.plugins.segment.Segment.RECORD_ALIGN_BITS;
+import static org.apache.jackrabbit.oak.plugins.segment.Segment.MAX_SEGMENT_SIZE;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+
+import org.apache.jackrabbit.oak.plugins.segment.memory.MemoryStore;
+import org.junit.Test;
+
+public class CompactorTest {
+
+    @Test
+    public void mapSerializationTest() {
+
+        final int maxExistingEntries = 100000;
+        final int maxNonExistingEntries = 10000;
+        final int seed = new Random().nextInt();
+
+        SegmentTracker factory = new MemoryStore().getTracker();
+        Map<RecordId, RecordId> map = new HashMap<RecordId, RecordId>();
+
+        Random r = new Random(seed);
+        int existing = r.nextInt(maxExistingEntries);
+        int nonExisting = r.nextInt(maxNonExistingEntries);
+
+        for (int i = 0; i < existing; i++) {
+            RecordId k = new RecordId(factory.newDataSegmentId(),
+                    asValidOffset(r.nextInt(MAX_SEGMENT_SIZE)));
+            RecordId v = new RecordId(factory.newDataSegmentId(),
+                    asValidOffset(r.nextInt(MAX_SEGMENT_SIZE)));
+            map.put(k, v);
+        }
+        ByteBuffer compaction = mapToByteBuffer(map);
+
+        // not serialized, expecting the same value back
+        for (int i = 0; i < nonExisting; i++) {
+            RecordId k = new RecordId(factory.newDataSegmentId(),
+                    asValidOffset(r.nextInt(MAX_SEGMENT_SIZE)));
+            assertFalse("Clash on recordids", map.containsKey(k));
+            map.put(k, k);
+        }
+
+        for (Entry<RecordId, RecordId> e : map.entrySet()) {
+            long[] v = recordAsKey(e.getValue());
+            long[] vl = readEntry(compaction, e.getKey());
+            assertArrayEquals("Failed with seed " + seed, vl, v);
+        }
+    }
+
+    private int asValidOffset(int random) {
+        while (random > 0) {
+            if (random % (1 << RECORD_ALIGN_BITS) == 0) {
+                return random;
+            }
+            random--;
+        }
+        return random;
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/segment/CompactorTest.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain