You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@directory.apache.org by ka...@apache.org on 2014/07/27 15:41:03 UTC

svn commit: r1613792 - in /directory/mavibot/trunk/mavibot/src: main/java/org/apache/directory/mavibot/btree/ test/java/org/apache/directory/mavibot/btree/

Author: kayyagari
Date: Sun Jul 27 13:41:02 2014
New Revision: 1613792

URL: http://svn.apache.org/r1613792
Log:
o added a class for reclaiming space by freeing up the copied pages
o the copied pages are now stored in a concurrent hashmap instead of a BTree
o the copied page map is processed after reaching a certain write commit count
o updated a test and removed another unused test

Added:
    directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/SpaceReclaimer.java
Modified:
    directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/RecordManager.java
    directory/mavibot/trunk/mavibot/src/test/java/org/apache/directory/mavibot/btree/PersistedBTreeBrowseTest.java
    directory/mavibot/trunk/mavibot/src/test/java/org/apache/directory/mavibot/btree/RecordManagerTest.java

Modified: directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/RecordManager.java
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/RecordManager.java?rev=1613792&r1=1613791&r2=1613792&view=diff
==============================================================================
--- directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/RecordManager.java (original)
+++ directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/RecordManager.java Sun Jul 27 13:41:02 2014
@@ -33,6 +33,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Queue;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.Lock;
@@ -103,10 +104,10 @@ public class RecordManager extends Abstr
     private long endOfFileOffset;
 
     /**
-     * A B-tree used to manage the page that has been copied in a new version.
+     * A Map used to hold the pages that were copied in a new version.
      * Those pages can be reclaimed when the associated version is dead.
      **/
-    /* no qualifier */ BTree<RevisionName, long[]> copiedPageBtree;
+    /* no qualifier */ Map<RevisionName, long[]> copiedPageMap = null;
 
     /** A constant for an offset on a non existing page */
     public static final long NO_PAGE = -1L;
@@ -177,12 +178,6 @@ public class RecordManager extends Abstr
     /** The previous B-tree of B-trees header offset */
     private long previousBtreeOfBtreesOffset = NO_PAGE;
 
-    /** The offset on the current copied pages B-tree */
-    /* no qualifier */ long currentCopiedPagesBtreeOffset = NO_PAGE;
-
-    /** The offset on the previous copied pages B-tree */
-    private long previousCopiedPagesBtreeOffset = NO_PAGE;
-
     /** A lock to protect the transaction handling */
     private Lock transactionLock = new ReentrantLock();
     
@@ -206,6 +201,18 @@ public class RecordManager extends Abstr
     
     /** A value stored into the transaction context for rollbacked transactions */
     private static final int ROLLBACKED_TXN = 0;
+
+    /** A lock to protect the freepage pointers */
+    private ReentrantLock freePageLock = new ReentrantLock();
+
+    /** the space reclaimer */
+    private SpaceReclaimer reclaimer;
+    
+    /** variable to keep track of the write commit count */
+    private int commitCount = 0;
+    
+    /** the threshold at which the SpaceReclaimer will be run to free the copied pages */
+    private int spaceReclaimerThreshold = 200;
     
     /**
      * Create a Record manager which will either create the underlying file
@@ -273,6 +280,11 @@ public class RecordManager extends Abstr
             {
                 loadRecordManager();
             }
+            
+            reclaimer = new SpaceReclaimer( this );
+            
+            copiedPageMap = reclaimer.readCopiedPageMap( file.getParentFile() );
+            runReclaimer();
         }
         catch ( Exception e )
         {
@@ -282,7 +294,24 @@ public class RecordManager extends Abstr
         }
     }
 
+    
+    /**
+     * runs the SpaceReclaimer to free the copied pages
+     */
+    private void runReclaimer()
+    {
+        try
+        {
+            commitCount = 0;
+            reclaimer.reclaim();
+        }
+        catch( Exception e )
+        {
+            LOG.warn( "SpaceReclaimer failed to free the pages", e );
+        }
+    }
 
+    
     /**
      * Create the mavibot file if it does not exist
      */
@@ -355,9 +384,6 @@ public class RecordManager extends Abstr
         // First, create the btree of btrees <NameRevision, Long>
         createBtreeOfBtrees();
 
-        // Now, initialize the Copied Page B-tree
-        createCopiedPagesBtree();
-
         // Inject these B-trees into the RecordManager. They are internal B-trees.
         try
         {
@@ -369,16 +395,6 @@ public class RecordManager extends Abstr
             // Inject the BtreeOfBtrees into the currentBtreeHeaders map
             currentBTreeHeaders.put( BTREE_OF_BTREES_NAME,  ((PersistedBTree<NameRevision, Long>)btreeOfBtrees).getBtreeHeader() );
             newBTreeHeaders.put( BTREE_OF_BTREES_NAME,  ((PersistedBTree<NameRevision, Long>)btreeOfBtrees).getBtreeHeader() );
-
-            // The FreePage B-tree
-            manage( copiedPageBtree, INTERNAL_BTREE );
-
-            currentCopiedPagesBtreeOffset = ((PersistedBTree<RevisionName, long[]>)copiedPageBtree).getBtreeHeader().getBTreeHeaderOffset();
-            updateRecordManagerHeader();
-            
-            // Inject the CopiedPagesBTree into the currentBtreeHeaders map
-            currentBTreeHeaders.put( COPIED_PAGE_BTREE_NAME, ((PersistedBTree<RevisionName, long[]>)copiedPageBtree).getBtreeHeader() );
-            newBTreeHeaders.put( COPIED_PAGE_BTREE_NAME, ((PersistedBTree<RevisionName, long[]>)copiedPageBtree).getBtreeHeader() );
         }
         catch ( BTreeAlreadyManagedException btame )
         {
@@ -411,22 +427,6 @@ public class RecordManager extends Abstr
 
 
     /**
-     * Create the CopiedPagesBtree
-     */
-    private void createCopiedPagesBtree()
-    {
-        PersistedBTreeConfiguration<RevisionName, long[]> configuration = new PersistedBTreeConfiguration<RevisionName, long[]>();
-        configuration.setKeySerializer( RevisionNameSerializer.INSTANCE );
-        configuration.setName( COPIED_PAGE_BTREE_NAME );
-        configuration.setValueSerializer( LongArraySerializer.INSTANCE );
-        configuration.setBtreeType( BTreeTypeEnum.COPIED_PAGES_BTREE );
-        configuration.setCacheSize( PersistedBTree.DEFAULT_CACHE_SIZE );
-
-        copiedPageBtree = BTreeFactory.createPersistedBTree( configuration );
-    }
-
-
-    /**
      * Load the BTrees from the disk.
      *
      * @throws InstantiationException
@@ -480,12 +480,6 @@ public class RecordManager extends Abstr
             // The previous BOB offset
             previousBtreeOfBtreesOffset = recordManagerHeader.getLong();
 
-            // The current Copied Pages B-tree offset
-            currentCopiedPagesBtreeOffset = recordManagerHeader.getLong();
-
-            // The previous Copied Pages B-tree offset
-            previousCopiedPagesBtreeOffset = recordManagerHeader.getLong();
-
             // read the B-tree of B-trees
             PageIO[] bobHeaderPageIos = readPageIOs( currentBtreeOfBtreesOffset, Long.MAX_VALUE );
 
@@ -494,14 +488,6 @@ public class RecordManager extends Abstr
 
             loadBtree( bobHeaderPageIos, btreeOfBtrees );
 
-            // read the copied page B-tree
-            PageIO[] copiedPagesPageIos = readPageIOs( currentCopiedPagesBtreeOffset, Long.MAX_VALUE );
-
-            copiedPageBtree = BTreeFactory.<RevisionName, long[]> createPersistedBTree( BTreeTypeEnum.COPIED_PAGES_BTREE );
-            //( ( PersistedBTree<RevisionName, long[]> ) copiedPageBtree ).setBtreeHeaderOffset( currentCopiedPagesBtreeOffset );
-
-            loadBtree( copiedPagesPageIos, copiedPageBtree );
-
             // Now, read all the B-trees from the btree of btrees
             TupleCursor<NameRevision, Long> btreeCursor = btreeOfBtrees.browse();
             Map<String, Long> loadedBtrees = new HashMap<String, Long>();
@@ -632,6 +618,13 @@ public class RecordManager extends Abstr
                 // And decrement the number of started transactions
                 decrementTxnLevel();
 
+                commitCount++;
+                
+                if( commitCount >= spaceReclaimerThreshold )
+                {
+                    runReclaimer();
+                }
+                
                 // Finally, release the global lock
                 transactionLock.unlock();
                 
@@ -669,8 +662,16 @@ public class RecordManager extends Abstr
                 // And decrement the number of started transactions
                 decrementTxnLevel();
 
+                commitCount++;
+                
+                if( commitCount >= spaceReclaimerThreshold )
+                {
+                    runReclaimer();
+                }
+
                 // Finally, release the global lock
                 transactionLock.unlock();
+                
                 return;
         }
     }
@@ -1051,7 +1052,7 @@ public class RecordManager extends Abstr
         int[] valueLengths = new int[nbElems];
 
         boolean isNotSubTree = ( btree.getType() != BTreeTypeEnum.PERSISTED_SUB );
-        
+
         // Read each key and value
         for ( int i = 0; i < nbElems; i++ )
         {
@@ -1821,12 +1822,6 @@ public class RecordManager extends Abstr
         // The offset of the copied pages B-tree
         position = writeData( RECORD_MANAGER_HEADER_BYTES, position, previousBtreeOfBtreesOffset );
 
-        // The offset of the current B-tree of B-trees
-        position = writeData( RECORD_MANAGER_HEADER_BYTES, position, currentCopiedPagesBtreeOffset );
-
-        // The offset of the copied pages B-tree
-        position = writeData( RECORD_MANAGER_HEADER_BYTES, position, previousCopiedPagesBtreeOffset );
-
         // Write the RecordManager header on disk
         RECORD_MANAGER_HEADER_BUFFER.put( RECORD_MANAGER_HEADER_BYTES );
         RECORD_MANAGER_HEADER_BUFFER.flip();
@@ -1840,8 +1835,6 @@ public class RecordManager extends Abstr
             sb.append( "First free page     : 0x" ).append( Long.toHexString( firstFreePage ) ).append( "\n" );
             sb.append( "Current BOB header  : 0x" ).append( Long.toHexString( currentBtreeOfBtreesOffset ) ).append( "\n" );
             sb.append( "Previous BOB header : 0x" ).append( Long.toHexString( previousBtreeOfBtreesOffset ) ).append( "\n" );
-            sb.append( "Current CPB header  : 0x" ).append( Long.toHexString( currentCopiedPagesBtreeOffset ) ).append( "\n" );
-            sb.append( "Previous CPB header : 0x" ).append( Long.toHexString( previousCopiedPagesBtreeOffset ) ).append( "\n" );
 
             if ( firstFreePage != NO_PAGE )
             {
@@ -1899,7 +1892,6 @@ public class RecordManager extends Abstr
 
         // Reset the old versions
         previousBtreeOfBtreesOffset = -1L;
-        previousCopiedPagesBtreeOffset = -1L;
 
         nbUpdateRMHeader.incrementAndGet();
     }
@@ -1933,12 +1925,6 @@ public class RecordManager extends Abstr
             previousBtreeOfBtreesOffset = currentBtreeOfBtreesOffset;
             currentBtreeOfBtreesOffset = newBtreeOfBtreesOffset;
         }
-
-        if ( newCopiedPageBtreeOffset != -1L )
-        {
-            previousCopiedPagesBtreeOffset = currentCopiedPagesBtreeOffset;
-            currentCopiedPagesBtreeOffset = newCopiedPageBtreeOffset;
-        }
     }
 
 
@@ -2014,10 +2000,7 @@ public class RecordManager extends Abstr
             pageOffsets[pos++] = ((AbstractPage<K, V>)page).getOffset();
         }
 
-        copiedPageBtree.insert( revisionName, pageOffsets );
-
-        // Update the CopiedPageBtree offset
-        currentCopiedPagesBtreeOffset = ((AbstractBTree<RevisionName, long[]>)copiedPageBtree).getBtreeHeader().getBTreeHeaderOffset();
+        copiedPageMap.put( revisionName, pageOffsets );
     }
 
 
@@ -2953,12 +2936,16 @@ public class RecordManager extends Abstr
         {
             nbReusedPages.incrementAndGet();
 
+            freePageLock.lock();
+            
             // We have some existing free page. Fetch it from disk
             PageIO pageIo = fetchPage( firstFreePage );
 
             // Update the firstFreePage pointer
             firstFreePage = pageIo.getNextPage();
 
+            freePageLock.unlock();
+            
             // overwrite the data of old page
             ByteBuffer data = ByteBuffer.allocateDirect( pageSize );
             pageIo.setData( data );
@@ -3047,7 +3034,6 @@ public class RecordManager extends Abstr
         }
 
         // Close the management B-trees
-        copiedPageBtree.close();
         btreeOfBtrees.close();
 
         managedBtrees.clear();
@@ -3058,6 +3044,8 @@ public class RecordManager extends Abstr
         // And close the channel
         fileChannel.close();
 
+        reclaimer.storeCopiedPageMap( file.getParentFile() );
+        
         commit();
     }
 
@@ -3381,7 +3369,7 @@ public class RecordManager extends Abstr
     {
         RevisionName revisionName = new RevisionName( revision, name );
 
-        copiedPageBtree.insert( revisionName, copiedPages );
+        copiedPageMap.put( revisionName, copiedPages );
     }
 
 
@@ -3399,11 +3387,6 @@ public class RecordManager extends Abstr
             return;
         }
 
-        if ( btree == copiedPageBtree )
-        {
-            return;
-        }
-
         NameRevision nameRevision = new NameRevision( btree.getName(), rootPage.getRevision() );
 
         ( ( AbstractBTree<NameRevision, Long> ) btreeOfBtrees ).insert( nameRevision,
@@ -3552,10 +3535,7 @@ public class RecordManager extends Abstr
                 // Deal with standard B-trees
                 RevisionName revisionName = new RevisionName( revision, btree.getName() );
 
-                copiedPageBtree.insert( revisionName, pageOffsets );
-
-                // Update the RecordManager Copiedpage Offset
-                currentCopiedPagesBtreeOffset = ((PersistedBTree<RevisionName, long[]>)copiedPageBtree).getBtreeOffset();
+                copiedPageMap.put( revisionName, pageOffsets );
             }
             else
             {
@@ -3582,6 +3562,8 @@ public class RecordManager extends Abstr
      */
     private void free( PageIO pageIo ) throws IOException
     {
+        freePageLock.lock();
+        
         // We add the Page's PageIOs before the
         // existing free pages.
         // Link it to the first free page
@@ -3590,14 +3572,65 @@ public class RecordManager extends Abstr
         LOG.debug( "Flushing the first free page" );
 
         // And flush it to disk
+        //FIXME can be flushed last after releasing the lock
         flushPages( pageIo );
 
         // We can update the firstFreePage offset
         firstFreePage = pageIo.getOffset();
+        
+        freePageLock.unlock();
     }
 
 
     /**
+     * Add an array of PageIOs to the list of free PageIOs
+     *
+     * @param offsets The offsets of the pages to be freed
+     * @throws IOException If we weren't capable of updating the file
+     */
+    public void free( long[] offsets ) throws IOException
+    {
+        if( offsets.length == 1 )
+        {
+            PageIO page = fetchPage( offsets[0] );
+            free( page );
+            return;
+        }
+        
+        PageIO[] pageIos = new PageIO[offsets.length];
+        
+        for( int i=0; i < offsets.length; i++ )
+        {
+            PageIO page = fetchPage( offsets[i] );
+            pageIos[i] = page;
+            
+            if( i > 0 )
+            {
+                pageIos[i-1].setNextPage( page.getOffset() );
+            }
+        }
+
+        freePageLock.lock();
+        
+        // We add the Page's PageIOs before the
+        // existing free pages.
+        // Link it to the first free page
+        pageIos[pageIos.length - 1].setNextPage( firstFreePage );
+
+        LOG.debug( "Flushing the first free page" );
+
+        // And flush it to disk
+        //FIXME can be flushed last after releasing the lock
+        flushPages( pageIos );
+
+        // We can update the firstFreePage offset
+        firstFreePage = pageIos[0].getOffset();
+        
+        freePageLock.unlock();
+    }
+
+    
+    /**
      * @return the keepRevisions flag
      */
     public boolean isKeepRevisions()

Added: directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/SpaceReclaimer.java
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/SpaceReclaimer.java?rev=1613792&view=auto
==============================================================================
--- directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/SpaceReclaimer.java (added)
+++ directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/SpaceReclaimer.java Sun Jul 27 13:41:02 2014
@@ -0,0 +1,255 @@
+/*
+ *   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.directory.mavibot.btree;
+
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A class used for reclaiming the copied pages.
+ *
+ * @author <a href="mailto:dev@directory.apache.org">Apache Directory Project</a>
+ */
+public class SpaceReclaimer
+{
+    /** the record manager */
+    private RecordManager rm;
+
+    private static String COPIED_PAGE_MAP_DATA_FILE = "cpm.db";
+
+    /** The LoggerFactory used by this class */
+    protected static final Logger LOG = LoggerFactory.getLogger( SpaceReclaimer.class );
+
+    /**
+     * Creates a new instance of SpaceReclaimer.
+     *
+     * @param rm the record manager
+     */
+    public SpaceReclaimer( RecordManager rm )
+    {
+        this.rm = rm;
+    }
+
+    
+    /**
+     * stores the copied page map, if not empty, in a file under the given directory
+     * 
+     * @param dir the directory where mavibot database file is present
+     */
+    /* no qualifier */ void storeCopiedPageMap( File dir )
+    {
+        if ( rm.copiedPageMap.isEmpty() )
+        {
+            LOG.debug( "Copied page map is empty, nothing to store on disk." );
+            return;
+        }
+        
+        File file = new File( dir, COPIED_PAGE_MAP_DATA_FILE );
+
+        try
+        {
+            LOG.debug( "Storing {} RevisionName of Copied page map", rm.copiedPageMap.size() );
+            
+            OutputStream fileOut = new FileOutputStream( file );
+            
+            ObjectOutputStream objOut = new ObjectOutputStream( fileOut );
+            
+            objOut.writeObject( rm.copiedPageMap );
+            
+            objOut.close();
+            
+            LOG.debug( "Successfully stored copied page map in {}", file.getAbsolutePath() );
+        }
+        catch( Exception e )
+        {
+            LOG.warn( "Failed to store the copied page map in {}", file.getAbsolutePath() );
+            LOG.warn( "", e );
+        }
+    }
+
+
+    /**
+     * reads the copied page map from the file named {@link #COPIED_PAGE_MAP_DATA_FILE} if it
+     * is present under the given directory
+     * 
+     * @param dir the directory where mavibot database file is present
+     * 
+     * @return
+     */
+    /* no qualifier */ ConcurrentHashMap<RevisionName, long[]> readCopiedPageMap( File dir )
+    {
+        
+        ConcurrentHashMap<RevisionName, long[]> map = new ConcurrentHashMap<RevisionName, long[]>();
+        
+        File file = new File( dir, COPIED_PAGE_MAP_DATA_FILE );
+        
+        if ( !file.exists() )
+        {
+            LOG.debug( "Copied page map store {} doesn't exist, returning empty map", file.getAbsolutePath() );
+            return map;
+        }
+
+        try
+        {
+            LOG.debug( "Reading Copied page map data stored in {}", file.getAbsolutePath() );
+            
+            InputStream fileIn = new FileInputStream( file );
+            
+            ObjectInputStream objIn = new ObjectInputStream( fileIn );
+            
+            map = ( ConcurrentHashMap<RevisionName, long[]> ) objIn.readObject();
+            
+            objIn.close();
+            
+            LOG.debug( "Successfully read copied page map containing {} RevisionNames", map.size() );
+        }
+        catch( Exception e )
+        {
+            LOG.warn( "Failed to read the copied page map from {}", file.getAbsolutePath() );
+            LOG.warn( "", e );
+        }
+        finally
+        {
+            boolean deleted = file.delete();
+            
+            // this is dangerous, cause during a subsequent restart the pages
+            // will be freed again, but this time they might have been in use
+            if( !deleted )
+            {
+                String warn = "Failed to delete the copied page map store " + file.getAbsolutePath() +
+                    " Make sure the approapriate permissions are given to delete this file by mavibot process." ;
+                LOG.warn( warn );
+                
+                throw new RuntimeException( warn );
+            }
+        }
+        
+        return map;
+    }
+
+    
+    /**
+     * relcaims the copied pages
+     */
+    /* no qualifier */ void reclaim()
+    {
+        //System.out.println( "reclaiming pages" );
+        try
+        {
+            Set<String> managed = rm.getManagedTrees();
+
+            for ( String name : managed )
+            {
+                PersistedBTree tree = ( PersistedBTree ) rm.getManagedTree( name );
+
+                Set<Long> inUseRevisions = new TreeSet<Long>();
+
+                // the tree might have been removed
+                if ( tree != null )
+                {
+                    Iterator<ReadTransaction> txnItr = tree.getReadTransactions().iterator();
+                    while ( txnItr.hasNext() )
+                    {
+                        inUseRevisions.add( txnItr.next().getRevision() );
+                    }
+                }
+
+                List<RevisionOffset> copiedRevisions = getRevisions( name );
+
+                for ( RevisionOffset ro : copiedRevisions )
+                {
+                    long rv = ro.getRevision();
+                    if ( inUseRevisions.contains( rv ) )
+                    {
+                        //System.out.println( "Revision " + rv + " of BTree " + name + " is in use, not reclaiming pages" );
+                        break;
+                    }
+
+                    long[] offsets = ro.getOffsets();
+
+                    //System.out.println( "Reclaiming " + Arrays.toString( offsets ) + "( " + offsets.length + " ) pages of the revision " + rv + " of BTree " + name );
+
+                    rm.free( offsets );
+
+                    RevisionName key = new RevisionName( rv, name );
+                    rm.copiedPageMap.remove( key );
+                }
+            }
+        }
+        catch ( Exception e )
+        {
+            e.printStackTrace();
+        }
+    }
+
+
+    /**
+     * gets a list of all the copied pages of a given B-Tree.
+     * 
+     * @param name the name of the B-Tree
+     * @return list of RevisionOffset
+     * @throws Exception
+     */
+    private List<RevisionOffset> getRevisions( String name ) throws Exception
+    {
+        long nbElems = rm.copiedPageMap.size();
+        //System.out.println( "Total number of entries in CPB " + nbElems );
+
+        if ( nbElems == 0 )
+        {
+            return Collections.EMPTY_LIST;
+        }
+
+        Iterator<Map.Entry<RevisionName, long[]>> cursor = rm.copiedPageMap.entrySet().iterator();
+
+        List<RevisionOffset> lst = new ArrayList<RevisionOffset>();
+
+        while ( cursor.hasNext() )
+        {
+            Map.Entry<RevisionName, long[]> t = cursor.next();
+            RevisionName rn = t.getKey();
+            if ( name.equals( rn.getName() ) )
+            {
+                //System.out.println( t.getValue() );
+                lst.add( new RevisionOffset( rn.getRevision(), t.getValue() ) );
+            }
+        }
+
+        return lst;
+    }
+}

Modified: directory/mavibot/trunk/mavibot/src/test/java/org/apache/directory/mavibot/btree/PersistedBTreeBrowseTest.java
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/test/java/org/apache/directory/mavibot/btree/PersistedBTreeBrowseTest.java?rev=1613792&r1=1613791&r2=1613792&view=diff
==============================================================================
--- directory/mavibot/trunk/mavibot/src/test/java/org/apache/directory/mavibot/btree/PersistedBTreeBrowseTest.java (original)
+++ directory/mavibot/trunk/mavibot/src/test/java/org/apache/directory/mavibot/btree/PersistedBTreeBrowseTest.java Sun Jul 27 13:41:02 2014
@@ -28,6 +28,7 @@ import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.UUID;
 
@@ -1106,7 +1107,7 @@ public class PersistedBTreeBrowseTest
     }
     
     
-    /*
+    @Ignore("test used for debugging")
     @Test
     public void testAdd20Random() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException
     {
@@ -1116,6 +1117,7 @@ public class PersistedBTreeBrowseTest
                 40, 33, 21, 18,  9, 30, 45, 36, 12,  8
             };
     
+        btree.setPageSize( 4 );
         // Inject some data
         for ( long value : values )
         {
@@ -1124,7 +1126,7 @@ public class PersistedBTreeBrowseTest
         }
 
 
-        BTree copiedPagesBtree = recordManager1.copiedPageBtree;
+        Map copiedPagesBtree = recordManager1.copiedPageMap;
         
         System.out.println( copiedPagesBtree );
         
@@ -1135,5 +1137,4 @@ public class PersistedBTreeBrowseTest
             System.out.println( cursor.nextKey() );
         }
     }
-    */
 }
\ No newline at end of file

Modified: directory/mavibot/trunk/mavibot/src/test/java/org/apache/directory/mavibot/btree/RecordManagerTest.java
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/test/java/org/apache/directory/mavibot/btree/RecordManagerTest.java?rev=1613792&r1=1613791&r2=1613792&view=diff
==============================================================================
--- directory/mavibot/trunk/mavibot/src/test/java/org/apache/directory/mavibot/btree/RecordManagerTest.java (original)
+++ directory/mavibot/trunk/mavibot/src/test/java/org/apache/directory/mavibot/btree/RecordManagerTest.java Sun Jul 27 13:41:02 2014
@@ -30,7 +30,9 @@ import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 
@@ -893,37 +895,4 @@ public class RecordManagerTest
         btree.insert( 1L, "V1" );
         btree.insert( 2L, "V2" );
     }
-    
-    
-    @Test
-    public void testOffsetsInCopiedPageBTree() throws Exception
-    {
-        btree.insert( 1L, "V1" );
-
-        checkCpbOffsets();
-        
-        openRecordManagerAndBtree();
-        
-        checkCpbOffsets();
-    }
-
-    
-    private void checkCpbOffsets() throws Exception
-    {
-        TupleCursor<RevisionName, long[]> cursor = recordManager.copiedPageBtree.browse();
-        
-        while( cursor.hasNext() )
-        {
-            Tuple<RevisionName, long[]> t = cursor.next();
-            long[] offsets = t.getValue();
-            
-            for( long o : offsets )
-            {
-                recordManager.checkOffset( o );
-            }
-        }
-      
-        cursor.close();
-    }
-    
 }