You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@labs.apache.org by el...@apache.org on 2013/03/11 17:56:26 UTC

svn commit: r1455232 - in /labs/mavibot/trunk/mavibot/src: main/java/org/apache/mavibot/btree/ main/java/org/apache/mavibot/btree/store/ test/java/org/apache/mavibot/btree/ test/java/org/apache/mavibot/btree/store/

Author: elecharny
Date: Mon Mar 11 16:56:25 2013
New Revision: 1455232

URL: http://svn.apache.org/r1455232
Log:
o Added e getRoot() method to the BTreeFactory class
o The nextPage field is now updated when we inject data in the PageIO instances
o Added a setSize() that compute the size of teh data stored in PageIOs
o Many fixes in RM in order to be able to write and read back many btrees. It currently works for empty BTrees.
o Removed the reference on id in some test
o Added a dump() method in the RM (not complete yet)

Modified:
    labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/BTreeFactory.java
    labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/PageIO.java
    labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/RecordManager.java
    labs/mavibot/trunk/mavibot/src/test/java/org/apache/mavibot/btree/InMemoryBTreeTest.java
    labs/mavibot/trunk/mavibot/src/test/java/org/apache/mavibot/btree/store/RecordManagerTest.java

Modified: labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/BTreeFactory.java
URL: http://svn.apache.org/viewvc/labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/BTreeFactory.java?rev=1455232&r1=1455231&r2=1455232&view=diff
==============================================================================
--- labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/BTreeFactory.java (original)
+++ labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/BTreeFactory.java Mon Mar 11 16:56:25 2013
@@ -91,6 +91,18 @@ public class BTreeFactory
 
 
     /**
+     * Return the BTree root page
+     * 
+     * @param btree The Btree we want to root page from
+     * @return The root page
+     */
+    public static Page getRoot( BTree<?, ?> btree )
+    {
+        return btree.rootPage;
+    }
+
+
+    /**
      * @param nbElems the nbElems to set
      */
     public static void setNbElems( BTree<?, ?> btree, long nbElems )

Modified: labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/PageIO.java
URL: http://svn.apache.org/viewvc/labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/PageIO.java?rev=1455232&r1=1455231&r2=1455232&view=diff
==============================================================================
--- labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/PageIO.java (original)
+++ labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/PageIO.java Mon Mar 11 16:56:25 2013
@@ -107,6 +107,7 @@ public class PageIO
     public void setData( ByteBuffer data )
     {
         this.data = data;
+        nextPage = data.getLong( 0 );
     }
 
 
@@ -116,9 +117,6 @@ public class PageIO
      */
     public long getNextPage()
     {
-        // read the nextPage from the PageIO
-        nextPage = data.getLong( 0 );
-
         return nextPage;
     }
 
@@ -139,8 +137,6 @@ public class PageIO
      */
     public long getSize()
     {
-        size = data.getInt( 8 );
-
         return size;
     }
 
@@ -157,6 +153,15 @@ public class PageIO
 
 
     /**
+     * @param size the size to set
+     */
+    public void setSize()
+    {
+        size = data.getInt( 8 );
+    }
+
+
+    /**
      * @return the offset
      */
     public long getOffset()

Modified: labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/RecordManager.java
URL: http://svn.apache.org/viewvc/labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/RecordManager.java?rev=1455232&r1=1455231&r2=1455232&view=diff
==============================================================================
--- labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/RecordManager.java (original)
+++ labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/RecordManager.java Mon Mar 11 16:56:25 2013
@@ -32,6 +32,7 @@ import java.util.Map;
 
 import org.apache.mavibot.btree.BTree;
 import org.apache.mavibot.btree.BTreeFactory;
+import org.apache.mavibot.btree.Node;
 import org.apache.mavibot.btree.Page;
 import org.apache.mavibot.btree.exception.BTreeAlreadyManagedException;
 import org.apache.mavibot.btree.exception.EndOfFileExceededException;
@@ -116,7 +117,7 @@ public class RecordManager
     private ByteBuffer blockBuffer;
 
     /** The set of managed BTrees */
-    private Map<String, BTree<?, ?>> managedBTrees;
+    private Map<String, BTree<?, ?>> managedBTrees = new HashMap<String, BTree<?, ?>>();
 
     /** The default file name */
     private static final String DEFAULT_FILE_NAME = "mavibot.db";
@@ -146,7 +147,6 @@ public class RecordManager
     public RecordManager( String fileName, int pageSize )
     {
         this.fileName = fileName;
-        this.pageSize = pageSize;
 
         // Open the file or create it
         File tmpFile = new File( fileName );
@@ -204,6 +204,7 @@ public class RecordManager
 
             if ( isNewFile )
             {
+                this.pageSize = pageSize;
                 initRecordManager();
             }
             else
@@ -255,8 +256,6 @@ public class RecordManager
         // Inject this BTree into the RecordManager
         try
         {
-            managedBTrees = new HashMap<String, BTree<?, ?>>();
-
             manage( copiedPageBTree );
         }
         catch ( BTreeAlreadyManagedException btame )
@@ -269,7 +268,8 @@ public class RecordManager
 
 
     /**
-     * We will load the header and all the existing BTrees in this record manager.
+     * Load the BTrees from the disk. 
+     * 
      * @throws InstantiationException 
      * @throws IllegalAccessException 
      * @throws ClassNotFoundException 
@@ -300,17 +300,25 @@ public class RecordManager
             // manage the modified pages. Once read, we can discard all
             // the pages that are stored in it, as we have restarted 
             // the RecordManager.
-            PageIO[] pageIos = readPages( 0L );
+            PageIO[] pageIos = readPages( HEADER_SIZE );
+            long position = pageIos.length * pageSize + HEADER_SIZE;
 
             // Create the BTree
             copiedPageBTree = BTreeFactory.createBTree();
 
-            long position = loadBTree( pageIos, 0L, copiedPageBTree );
+            position = loadBTree( pageIos, position, copiedPageBTree );
 
             // Then process the next ones
-            for ( int i = 0; i < nbBtree; i++ )
+            for ( int i = 1; i < nbBtree; i++ )
             {
-                BTree<?, ?> btree = new BTree();
+                // Create the BTree
+                BTree<?, ?> btree = BTreeFactory.createBTree();
+
+                // Read the associated pages
+                pageIos = readPages( position );
+                position = pageIos.length * pageSize;
+
+                // Load the BTree
                 position = loadBTree( pageIos, position, btree );
 
                 // Store it into the managedBtrees map
@@ -333,15 +341,18 @@ public class RecordManager
     private PageIO[] readPages( long position ) throws IOException, EndOfFileExceededException
     {
         PageIO firstPage = fetchPage( position );
-        long nextPage = 0L;
+        firstPage.setSize();
         List<PageIO> listPages = new ArrayList<PageIO>();
         listPages.add( firstPage );
 
         // Iterate on the pages
-        while ( ( nextPage = firstPage.getNextPage() ) != NO_PAGE )
+        long nextPage = firstPage.getNextPage();
+
+        while ( nextPage != NO_PAGE )
         {
             PageIO page = fetchPage( nextPage );
             listPages.add( page );
+            nextPage = page.getNextPage();
         }
 
         // Return 
@@ -364,14 +375,16 @@ public class RecordManager
     private long loadBTree( PageIO[] pageIos, long position, BTree<?, ?> btree ) throws EndOfFileExceededException,
         IOException, ClassNotFoundException, IllegalAccessException, InstantiationException
     {
+        long dataPos = 0L;
+
         // The tree name
-        byte[] btreeNameBytes = readBytes( pageIos, position );
+        byte[] btreeNameBytes = readBytes( pageIos, dataPos );
 
-        position += INT_SIZE;
+        dataPos += INT_SIZE;
 
         if ( btreeNameBytes != null )
         {
-            position += btreeNameBytes.length;
+            dataPos += btreeNameBytes.length;
             String btreeName = Strings.utf8ToString( btreeNameBytes );
             BTreeFactory.setName( btree, btreeName );
         }
@@ -381,14 +394,14 @@ public class RecordManager
         }
 
         // The keySerializer FQCN
-        byte[] keySerializerBytes = readBytes( pageIos, position );
+        byte[] keySerializerBytes = readBytes( pageIos, dataPos );
 
         String keySerializerFqcn = null;
-        position += INT_SIZE;
+        dataPos += INT_SIZE;
 
         if ( keySerializerBytes != null )
         {
-            position += keySerializerBytes.length;
+            dataPos += keySerializerBytes.length;
             keySerializerFqcn = Strings.utf8ToString( keySerializerBytes );
         }
         else
@@ -399,14 +412,14 @@ public class RecordManager
         BTreeFactory.setKeySerializer( btree, keySerializerFqcn );
 
         // The valueSerialier FQCN
-        byte[] valueSerializerBytes = readBytes( pageIos, position );
+        byte[] valueSerializerBytes = readBytes( pageIos, dataPos );
 
         String valueSerializerFqcn = null;
-        position += INT_SIZE;
+        dataPos += INT_SIZE;
 
         if ( valueSerializerBytes != null )
         {
-            position += valueSerializerBytes.length;
+            dataPos += valueSerializerBytes.length;
             valueSerializerFqcn = Strings.utf8ToString( valueSerializerBytes );
         }
         else
@@ -417,45 +430,74 @@ public class RecordManager
         BTreeFactory.setValueSerializer( btree, valueSerializerFqcn );
 
         // The BTree page size
-        int btreePageSize = readInt( pageIos, position );
+        int btreePageSize = readInt( pageIos, dataPos );
         BTreeFactory.setPageSize( btree, btreePageSize );
-        position += INT_SIZE;
+        dataPos += INT_SIZE;
 
         // The BTree current revision
-        long revision = readLong( pageIos, position );
+        long revision = readLong( pageIos, dataPos );
         BTreeFactory.setRevision( btree, revision );
-        position += LONG_SIZE;
+        dataPos += LONG_SIZE;
 
         // The nb elems in the tree
-        int nbElems = readInt( pageIos, position );
+        int nbElems = readInt( pageIos, dataPos );
         BTreeFactory.setNbElems( btree, nbElems );
-        position += LONG_SIZE;
+        dataPos += LONG_SIZE;
 
-        // The BTree rootPage offset
-        long rootPageOffset = readLong( pageIos, position );
+        // Now, int the BTree
+        btree.init();
 
-        PageIO[] rootPage = readPages( rootPageOffset );
-        position += LONG_SIZE;
+        // The BTree rootPage offset
+        long rootPageOffset = readLong( pageIos, dataPos );
 
         // Now, load the rootPage, which can be a Leaf or a Node, depending 
         // on the number of elements in the tree : if it's above the pageSize,
         // it's a Node, otherwise it's a Leaf
-        Page btreeRoot = null;
 
-        if ( nbElems > btreePageSize )
+        // Read the rootPage pages on disk
+        PageIO[] rootPageIos = readPages( rootPageOffset );
+        position += rootPageIos.length * pageSize;
+
+        Page btreeRoot = readPage( btree, revision, rootPageIos );
+
+        BTreeFactory.setRoot( btree, btreeRoot );
+
+        return position;
+    }
+
+
+    private Page readNode( BTree btree, long offset, long revision, int nbElems ) throws IOException
+    {
+        Page node = BTreeFactory.createNode( btree, revision, nbElems );
+
+        // Read the rootPage pages on disk
+        PageIO[] pageIos = readPages( offset );
+
+        return node;
+    }
+
+
+    private Page readPage( BTree btree, long revision, PageIO[] pageIos ) throws IOException
+    {
+        // Deserialize the rootPage now
+        long position = 0L;
+
+        int nbElems = readInt( pageIos, position );
+
+        Page page = null;
+
+        if ( nbElems >= 0 )
         {
-            // It's a Node
-            btreeRoot = BTreeFactory.createNode( btree, revision, nbElems );
+            // Its a leaf
+            page = BTreeFactory.createLeaf( btree, revision, nbElems );
         }
         else
         {
-            // it's a leaf
-            btreeRoot = BTreeFactory.createLeaf( btree, revision, nbElems );
+            // It's a node
+            page = BTreeFactory.createNode( btree, revision, nbElems );
         }
 
-        BTreeFactory.setRoot( btree, btreeRoot );
-
-        return position;
+        return page;
     }
 
 
@@ -723,14 +765,6 @@ public class RecordManager
         // Get the pageIOs we need to store the data. We may need more than one.
         PageIO[] pageIos = getFreePageIOs( bufferSize );
 
-        // Get a free page to store the RootPage
-        PageIO rootPageIo = fetchNewPage();
-
-        // Update the number of elements to 0, as it's a new page
-        // We have to do that as the page might contain garbage
-        store( 0L, 0L, rootPageIo );
-        rootPageIo.setSize( INT_SIZE );
-
         // Now store the BTree data in the pages :
         // - the BTree name
         // - the keySerializer FQCN
@@ -760,12 +794,20 @@ public class RecordManager
         // The nb elems in the tree
         position = store( position, btree.getNbElems(), pageIos );
 
-        // The BTree rootPage offset
+        // Serialize the BTree root page
+        Page rootPage = BTreeFactory.getRoot( btree );
+
+        PageIO[] rootPageIos = createSerializedPage( btree.getRevision(), rootPage );
+
+        // Get the reference on the first page
+        PageIO rootPageIo = rootPageIos[0];
+
+        // Now, we can inject the BTree rootPage offset into the BTree header
         position = store( position, rootPageIo.getOffset(), pageIos );
 
         // And flush the pages to disk now
         flushPages( pageIos );
-        flushPages( rootPageIo );
+        flushPages( rootPageIos );
 
         nbBtree++;
 
@@ -775,6 +817,67 @@ public class RecordManager
 
 
     /**
+     * Serialize a new Page. It will contain the following data :<br/>
+     * <ul>
+     * <li>the revision : a long</li>
+     * <li>the number of elements : an int (if <= 0, it's a Node, otherwise it's a Leaf)</li>
+     * <li>the keys : an array of serialized keys</li>
+     * <li>the values : an array of references to the children pageIO offset (stored as long)
+     * if it's a Node, or a list of values if it's a Leaf</li>
+     * <li></li>
+     * </ul>
+     * 
+     * @param revision The node revision
+     * @param keys The keys to serialize
+     * @param children The references to the children
+     * @return An array of pages containing the serialized node
+     * @throws IOException
+     */
+    private PageIO[] createSerializedPage( long revision, Page page ) throws IOException
+    {
+        int nbElems = page.getNbElems();
+
+        // Make it a negative value if it's a Node
+        if ( page instanceof Node )
+        {
+            nbElems = -nbElems;
+        }
+
+        if ( nbElems == 0 )
+        {
+            // We will have 1 single page if we have no elements
+            PageIO[] pageIos = new PageIO[1];
+
+            // This is either a new root page or a new page that will be filled later
+            PageIO newPage = fetchNewPage();
+
+            // We need first to create a byte[] that will contain all the data
+            // For the root page, this is easy, as we only have to store the revision, 
+            // and the number of elements, which is 0.
+            long position = 0L;
+
+            position = store( position, revision, newPage );
+            position = store( position, nbElems, newPage );
+
+            // Update the page size now
+            newPage.setSize( ( int ) position );
+
+            // Insert the result into the array of PageIO
+            pageIos[0] = newPage;
+
+            return pageIos;
+        }
+        else
+        {
+            // Allocate the array to store the result
+            PageIO[] pageIos = new PageIO[1];
+
+            return pageIos;
+        }
+    }
+
+
+    /**
      * Update the header, injecting the nbBtree, firstFreePage and lastFreePage
      */
     private void updateHeader() throws IOException
@@ -1202,7 +1305,7 @@ public class RecordManager
      */
     private PageIO fetchPage( long offset ) throws IOException, EndOfFileExceededException
     {
-        if ( fileChannel.size() <= offset + pageSize )
+        if ( fileChannel.size() < offset + pageSize )
         {
             // Error : we are past the end of the file
             throw new EndOfFileExceededException( "We are fetching a page on " + offset +
@@ -1215,6 +1318,7 @@ public class RecordManager
 
             ByteBuffer data = ByteBuffer.allocate( pageSize );
             fileChannel.read( data );
+            data.rewind();
 
             PageIO readPage = new PageIO( offset );
             readPage.setData( data );
@@ -1310,4 +1414,70 @@ public class RecordManager
 
         return dataBuffer;
     }
+
+
+    /**
+     * Close the RecordManager and flush everything on disk
+     */
+    public void close() throws IOException
+    {
+        // TODO : we must wait for the last write to finish
+
+        // Write the data
+        fileChannel.force( true );
+
+        // And close the channel
+        fileChannel.close();
+    }
+
+
+    /**
+     * Dump the RecordManager file
+     * @throws IOException 
+     */
+    public void dump() throws IOException
+    {
+        RandomAccessFile randomFile = new RandomAccessFile( file, "r" );
+        FileChannel fileChannel = randomFile.getChannel();
+
+        ByteBuffer header = ByteBuffer.allocate( HEADER_SIZE );
+
+        // load the header 
+        fileChannel.read( header );
+
+        header.rewind();
+
+        // The page size
+        int pageSize = header.getInt();
+
+        // The number of managed BTrees
+        int nbBTree = header.getInt();
+
+        // The first and last free page
+        long firstFreePage = header.getLong();
+        long lastFreePage = header.getLong();
+
+        System.out.println( "RecordManager" );
+        System.out.println( "-------------" );
+        System.out.println( "  Header " );
+        System.out.println( "    '" + Strings.dumpBytes( header.array() ) + "'" );
+        System.out.println( "    page size : " + pageSize );
+        System.out.println( "    nbTree : " + nbBTree );
+        System.out.println( "    firstFreePage : " + firstFreePage );
+        System.out.println( "    lastFreePage : " + lastFreePage );
+
+        long position = HEADER_SIZE;
+
+        // Dump the BTrees
+        for ( int i = 0; i < nbBTree; i++ )
+        {
+            System.out.println( "  Btree[" + i + "]" );
+            PageIO[] pageIos = readPages( position );
+
+            for ( PageIO pageIo : pageIos )
+            {
+                System.out.println( "    " + pageIo );
+            }
+        }
+    }
 }

Modified: labs/mavibot/trunk/mavibot/src/test/java/org/apache/mavibot/btree/InMemoryBTreeTest.java
URL: http://svn.apache.org/viewvc/labs/mavibot/trunk/mavibot/src/test/java/org/apache/mavibot/btree/InMemoryBTreeTest.java?rev=1455232&r1=1455231&r2=1455232&view=diff
==============================================================================
--- labs/mavibot/trunk/mavibot/src/test/java/org/apache/mavibot/btree/InMemoryBTreeTest.java (original)
+++ labs/mavibot/trunk/mavibot/src/test/java/org/apache/mavibot/btree/InMemoryBTreeTest.java Mon Mar 11 16:56:25 2013
@@ -1007,7 +1007,6 @@ public class InMemoryBTreeTest
         Leaf<Integer, String> leaf = new Leaf<Integer, String>( btree );
         int pos = 0;
         leaf.revision = revision;
-        //leaf.id = revision;
         leaf.nbElems = tuples.length;
         leaf.keys = new Integer[leaf.nbElems];
         leaf.values = ( MemoryValueHolder<Integer, String>[] ) Array

Modified: labs/mavibot/trunk/mavibot/src/test/java/org/apache/mavibot/btree/store/RecordManagerTest.java
URL: http://svn.apache.org/viewvc/labs/mavibot/trunk/mavibot/src/test/java/org/apache/mavibot/btree/store/RecordManagerTest.java?rev=1455232&r1=1455231&r2=1455232&view=diff
==============================================================================
--- labs/mavibot/trunk/mavibot/src/test/java/org/apache/mavibot/btree/store/RecordManagerTest.java (original)
+++ labs/mavibot/trunk/mavibot/src/test/java/org/apache/mavibot/btree/store/RecordManagerTest.java Mon Mar 11 16:56:25 2013
@@ -50,15 +50,20 @@ public class RecordManagerTest
 
         assertNotNull( recordManager );
 
+        //recordManager.dump();
+
         // Create a new BTree
         BTree<Long, String> btree = new BTree<Long, String>( "test", new LongSerializer(), new StringSerializer() );
 
         // And make it managed by the RM
         recordManager.manage( btree );
 
-        // Inject an element into the btree
-        btree.insert( 1L, "V1" );
+        //recordManager.dump();
 
-    }
+        // Close the recordManager
+        recordManager.close();
 
+        // Now, try to reload the file back
+        RecordManager recordManager1 = new RecordManager( tempFileName );
+    }
 }



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@labs.apache.org
For additional commands, e-mail: commits-help@labs.apache.org