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/03 19:38:54 UTC

svn commit: r1452096 - in /labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree: ./ exception/ store/

Author: elecharny
Date: Sun Mar  3 18:38:53 2013
New Revision: 1452096

URL: http://svn.apache.org/r1452096
Log:
o Added a very first version of the RecordManager (not finished yet)
o Moved some exceptions in a dedicated package

Added:
    labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/exception/BTreeAlreadyManagedException.java
    labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/exception/PageSizeAlreadySetException.java
    labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/
    labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/BtreeHeader.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
Removed:
    labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/KeyNotFoundException.java
Modified:
    labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/BTree.java

Modified: labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/BTree.java
URL: http://svn.apache.org/viewvc/labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/BTree.java?rev=1452096&r1=1452095&r2=1452096&view=diff
==============================================================================
--- labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/BTree.java (original)
+++ labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/BTree.java Sun Mar  3 18:38:53 2013
@@ -1376,6 +1376,24 @@ public class BTree<K, V>
 
 
     /**
+     * @return the keySerializer
+     */
+    public ElementSerializer<K> getKeySerializer()
+    {
+        return keySerializer;
+    }
+
+
+    /**
+     * @return the valueSerializer
+     */
+    public ElementSerializer<V> getValueSerializer()
+    {
+        return valueSerializer;
+    }
+
+
+    /**
      * @see Object#toString()
      */
     public String toString()

Added: labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/exception/BTreeAlreadyManagedException.java
URL: http://svn.apache.org/viewvc/labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/exception/BTreeAlreadyManagedException.java?rev=1452096&view=auto
==============================================================================
--- labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/exception/BTreeAlreadyManagedException.java (added)
+++ labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/exception/BTreeAlreadyManagedException.java Sun Mar  3 18:38:53 2013
@@ -0,0 +1,73 @@
+/*
+ *  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.mavibot.btree.exception;
+
+
+/**
+ * An exception thrown when we try to manage a BTree which name is
+ * already managed by the RecordManager
+ * 
+ * @author <a href="mailto:labs@labs.apache.org">Mavibot labs Project</a>
+ */
+public class BTreeAlreadyManagedException extends Exception
+{
+    /** The serial version UUID */
+    private static final long serialVersionUID = 1L;
+
+
+    /**
+     * Creates a new instance of BtreeAlreadyManagedException.
+     */
+    public BTreeAlreadyManagedException()
+    {
+    }
+
+
+    /**
+     * Creates a new instance of BtreeAlreadyManagedException.
+     *
+     * @param explanation The message associated with the exception
+     */
+    public BTreeAlreadyManagedException( String explanation )
+    {
+        super( explanation );
+    }
+
+
+    /**
+     * Creates a new instance of BtreeAlreadyManagedException.
+     */
+    public BTreeAlreadyManagedException( Throwable cause )
+    {
+        super( cause );
+    }
+
+
+    /**
+     * Creates a new instance of KeyNotFoundException.
+     *
+     * @param explanation The message associated with the exception
+     * @param cause The root cause for this exception
+     */
+    public BTreeAlreadyManagedException( String explanation, Throwable cause )
+    {
+        super( explanation, cause );
+    }
+}

Added: labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/exception/PageSizeAlreadySetException.java
URL: http://svn.apache.org/viewvc/labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/exception/PageSizeAlreadySetException.java?rev=1452096&view=auto
==============================================================================
--- labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/exception/PageSizeAlreadySetException.java (added)
+++ labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/exception/PageSizeAlreadySetException.java Sun Mar  3 18:38:53 2013
@@ -0,0 +1,74 @@
+/*
+ *  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.mavibot.btree.exception;
+
+
+/**
+ * An exception thrown when we try to change the RecordManager Page Size
+ * when it's already set.
+ * already managed by the RecordManager
+ * 
+ * @author <a href="mailto:labs@labs.apache.org">Mavibot labs Project</a>
+ */
+public class PageSizeAlreadySetException extends Exception
+{
+    /** The serial version UUID */
+    private static final long serialVersionUID = 1L;
+
+
+    /**
+     * Creates a new instance of PageSizeAlreadySetException.
+     */
+    public PageSizeAlreadySetException()
+    {
+    }
+
+
+    /**
+     * Creates a new instance of PageSizeAlreadySetException.
+     *
+     * @param explanation The message associated with the exception
+     */
+    public PageSizeAlreadySetException( String explanation )
+    {
+        super( explanation );
+    }
+
+
+    /**
+     * Creates a new instance of PageSizeAlreadySetException.
+     */
+    public PageSizeAlreadySetException( Throwable cause )
+    {
+        super( cause );
+    }
+
+
+    /**
+     * Creates a new instance of PageSizeAlreadySetException.
+     *
+     * @param explanation The message associated with the exception
+     * @param cause The root cause for this exception
+     */
+    public PageSizeAlreadySetException( String explanation, Throwable cause )
+    {
+        super( explanation, cause );
+    }
+}

Added: labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/BtreeHeader.java
URL: http://svn.apache.org/viewvc/labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/BtreeHeader.java?rev=1452096&view=auto
==============================================================================
--- labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/BtreeHeader.java (added)
+++ labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/BtreeHeader.java Sun Mar  3 18:38:53 2013
@@ -0,0 +1,175 @@
+/*
+ *  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.mavibot.btree.store;
+
+
+/**
+ * 
+ * @author <a href="mailto:labs@labs.apache.org">Mavibot labs Project</a>
+ */
+public class BtreeHeader
+{
+    /** The BTree name */
+    private String name;
+
+    /** The current version */
+    private long version;
+
+    /** The existing versions */
+    private long[] versions;
+
+    /** The position in the file */
+    private long offset;
+
+    /** The FQCN of the Key serializer */
+    private String keySerializerFQCN;
+
+    /** The FQCN of the Key comparator */
+    private String keyComparatorFQCN;
+
+    /** The FQCN of the Value serializer */
+    private String valueSerializerFQCN;
+
+    /** The FQCN of the Value serializer */
+    private String valueComparatorFQCN;
+
+
+    public BtreeHeader( String name )
+    {
+        this.name = name;
+    }
+
+
+    /**
+     * @return the name
+     */
+    public String getName()
+    {
+        return name;
+    }
+
+
+    /**
+     * @param name the name to set
+     */
+    public void setName( String name )
+    {
+        this.name = name;
+    }
+
+
+    /**
+     * @return the version
+     */
+    public long getVersion()
+    {
+        return version;
+    }
+
+
+    /**
+     * @param version the version to set
+     */
+    public void setVersion( long version )
+    {
+        this.version = version;
+    }
+
+
+    /**
+     * @return the versions
+     */
+    public long[] getVersions()
+    {
+        return versions;
+    }
+
+
+    /**
+     * @param versions the versions to set
+     */
+    public void setVersions( long[] versions )
+    {
+        this.versions = versions;
+    }
+
+
+    /**
+     * @return the offset
+     */
+    public long getOffset()
+    {
+        return offset;
+    }
+
+
+    /**
+     * @param offset the offset to set
+     */
+    public void setOffset( long offset )
+    {
+        this.offset = offset;
+    }
+
+
+    /**
+     * @see Object#toString()
+     */
+    public String toString()
+    {
+        StringBuilder sb = new StringBuilder();
+
+        sb.append( "Btree '" ).append( name ).append( "'" );
+        sb.append( ", V[" ).append( version ).append( "]" );
+        sb.append( ", O[" ).append( offset ).append( "]\n" );
+        sb.append( "{\n" );
+        sb.append( "    Key serializer   : " ).append( keySerializerFQCN ).append( "\n" );
+        sb.append( "    Value serializer : " ).append( valueSerializerFQCN ).append( "\n" );
+        sb.append( "    Key comparator   : " ).append( keyComparatorFQCN ).append( "\n" );
+        sb.append( "    Value comparator : " ).append( valueComparatorFQCN ).append( "\n" );
+        sb.append( "}\n" );
+
+        if ( ( versions != null ) && ( versions.length != 0 ) )
+        {
+            sb.append( "Versions : \n" );
+            sb.append( "{\n" );
+
+            boolean isFirst = true;
+
+            for ( long version : versions )
+            {
+                if ( isFirst )
+                {
+                    isFirst = false;
+                }
+                else
+                {
+                    sb.append( ",\n" );
+                }
+
+                sb.append( "    " ).append( version );
+            }
+
+            sb.append( "}\n" );
+        }
+
+        return sb.toString();
+    }
+}

Added: 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=1452096&view=auto
==============================================================================
--- labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/PageIO.java (added)
+++ labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/PageIO.java Sun Mar  3 18:38:53 2013
@@ -0,0 +1,190 @@
+/*
+ *  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.mavibot.btree.store;
+
+
+/**
+ * A structure containing a Page on disk. It's a byte[PageSize] plus a few informations like
+ * the page offset on disk and a link to the next page.</br>
+ * As we may need more than one Page to store some data, the PageIO are linked so that
+ * the list of all the PageIO contain the full data.</br>
+ * The first PageIO contains the size of the data.</br>
+ * Here is the logical structure of a PageIO :
+ * <pre>
+ * For a first page :
+ * 
+ * +----------+------+----------------------+
+ * | nextPage | size | XXXXXXXXXXXXXXXXXXXX |
+ * +----------+------+----------------------+
+ * 
+ * for any page but the first :
+ * 
+ * +----------+-----------------------------+
+ * | nextPage | XXXXXXXXXXXXXXXXXXXXXXXXXXX |
+ * +----------+-----------------------------+
+ * 
+ * for the last page :
+ * +----------+-----------------------------+
+ * |    -1    | XXXXXXXXXXXXXXXXXXXXXXXXXXX |
+ * +----------+-----------------------------+
+ * 
+ * In any case, the page length is always PageSize.
+ * </pre>
+ *  
+ * @author <a href="mailto:labs@labs.apache.org">Mavibot labs Project</a>
+ */
+public class PageIO
+{
+    /** The contain data */
+    private byte[] data;
+
+    /** A pointer to the next pageIO */
+    private long nextPage;
+
+    /** The offset on disk */
+    private long size;
+
+    /** The position of the page on disk */
+    private long offset;
+
+
+    /** 
+     * A default constructor for a PageIO
+     */
+    public PageIO()
+    {
+        nextPage = -2L;
+        size = -1;
+        offset = -1L;
+    }
+
+
+    /** 
+     * A constructor for a PageIO when we know the offset of this page on disk
+     */
+    public PageIO( long offset )
+    {
+        nextPage = -2L;
+        size = -1;
+        this.offset = offset;
+    }
+
+
+    /**
+     * @return the data
+     */
+    public byte[] getData()
+    {
+        return data;
+    }
+
+
+    /**
+     * @param data the data to set
+     */
+    public void setData( byte[] data )
+    {
+        this.data = data;
+    }
+
+
+    /**
+     * Get the NextPage value from the PageIO. If it's -1, there is no next page<br/>
+     * @return the nextPage
+     */
+    public long getNextPage()
+    {
+        // read the nextPage from the PageIO
+        nextPage = ( ( long ) data[0] << 56 ) +
+            ( ( data[1] & 0xFFL ) << 48 ) +
+            ( ( data[2] & 0xFFL ) << 40 ) +
+            ( ( data[3] & 0xFFL ) << 32 ) +
+            ( ( data[4] & 0xFFL ) << 24 ) +
+            ( ( data[5] & 0xFFL ) << 16 ) +
+            ( ( data[6] & 0xFFL ) << 8 ) +
+            ( data[7] & 0xFFL );
+
+        return nextPage;
+    }
+
+
+    /**
+     * @param nextPage the nextPage to set
+     */
+    public void setNextPage( long nextPage )
+    {
+        this.nextPage = nextPage;
+
+        data[0] = ( byte ) ( nextPage >>> 56 );
+        data[1] = ( byte ) ( nextPage >>> 48 );
+        data[2] = ( byte ) ( nextPage >>> 40 );
+        data[3] = ( byte ) ( nextPage >>> 32 );
+        data[4] = ( byte ) ( nextPage >>> 24 );
+        data[5] = ( byte ) ( nextPage >>> 16 );
+        data[6] = ( byte ) ( nextPage >>> 8 );
+        data[7] = ( byte ) ( nextPage );
+    }
+
+
+    /**
+     * @return the size
+     */
+    public long getSize()
+    {
+        size = ( data[8] << 24 ) +
+            ( ( data[9] & 0xFF ) << 16 ) +
+            ( ( data[10] & 0xFF ) << 8 ) +
+            ( data[11] & 0xFF );
+
+        return size;
+    }
+
+
+    /**
+     * @param size the size to set
+     */
+    public void setSize( long size )
+    {
+        data[8] = ( byte ) ( size >>> 24 );
+        data[9] = ( byte ) ( size >>> 16 );
+        data[10] = ( byte ) ( size >>> 8 );
+        data[11] = ( byte ) ( size );
+
+        this.size = size;
+    }
+
+
+    /**
+     * @return the offset
+     */
+    public long getOffset()
+    {
+        return offset;
+    }
+
+
+    /**
+     * @param offset the offset to set
+     */
+    public void setOffset( long offset )
+    {
+        this.offset = offset;
+    }
+}

Added: 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=1452096&view=auto
==============================================================================
--- labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/RecordManager.java (added)
+++ labs/mavibot/trunk/mavibot/src/main/java/org/apache/mavibot/btree/store/RecordManager.java Sun Mar  3 18:38:53 2013
@@ -0,0 +1,446 @@
+/*
+ *  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.mavibot.btree.store;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.Map;
+
+import org.apache.mavibot.btree.BTree;
+import org.apache.mavibot.btree.exception.BTreeAlreadyManagedException;
+import org.apache.mavibot.btree.serializer.IntSerializer;
+import org.apache.mavibot.btree.serializer.LongArraySerializer;
+import org.apache.mavinot.btree.utils.Strings;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The RecordManager is used to manage the file in which we will store the BTrees. 
+ * A RecordManager will manage more than one BTree.<br/>
+ * 
+ * It stores data in fixed size pages (default size is 4Kb), which may be linked one to 
+ * the other if the data we want to store is too bug for a page.
+ * if 
+ * @author <a href="mailto:labs@labs.apache.org">Mavibot labs Project</a>
+ */
+public class RecordManager
+{
+    /** The LoggerFactory used by this class */
+    protected static final Logger LOG = LoggerFactory.getLogger( RecordManager.class );
+
+    /** The file name */
+    private String fileName;
+
+    /** The associated file */
+    private File file;
+
+    /** The channel used to read and write data */
+    private FileChannel fileChannel;
+
+    /** The number of stored BTrees */
+    private int nbBtree;
+
+    /** The first and last free page */
+    private long firstFreePage;
+    private long lastFreePage;
+
+    /** 
+     * A Btree used to manage the page that has been copied in a new version.
+     * Those page can be reclaimed when the associated version is dead. 
+     **/
+    private BTree<Integer, long[]> copiedPageBTree;
+
+    /** A constant for an offset on a non existing page */
+    private static final long NO_PAGE = -1L;
+
+    /** The number of stored BTrees */
+    private static final int NB_TREE_SIZE = 4;
+
+    /** The header page size */
+    private static final int PAGE_SIZE = 4;
+
+    /** The size of the data size in a page */
+    private static final int DATA_SIZE = 4;
+
+    /** The size of the link to next page */
+    private static final int LINK_SIZE = 8;
+
+    /** The size of the link to the first and last free page */
+    private static final int FIRST_FREE_PAGE_SIZE = 8;
+    private static final int LAST_FREE_PAGE_SIZE = 8;
+
+    private static final int HEADER_SIZE = NB_TREE_SIZE + PAGE_SIZE + FIRST_FREE_PAGE_SIZE + LAST_FREE_PAGE_SIZE;
+    private static final ByteBuffer HEADER_BUFFER = ByteBuffer.allocate( HEADER_SIZE );
+
+    /** The default page size */
+    private static final int DEFAULT_PAGE_SIZE = 4 * 1024;
+
+    /** The RecordManager underlying page size. */
+    private int pageSize = -1;
+
+    /** A buffer used to read a page */
+    private ByteBuffer blockBuffer;
+
+    /** The set of managed BTrees */
+    private Map<String, BTree<?, ?>> managedBTrees;
+
+    /** The default file name */
+    private static final String DEFAULT_FILE_NAME = "mavibot.db";
+
+
+    /**
+     * Create a Record manager which will either create the underlying file
+     * or load an existing one. If a folder is provider, then we will create
+     * a file with a default name : mavibot.db
+     * 
+     * @param name The file name, or a folder name
+     */
+    public RecordManager( String fileName )
+    {
+        this.fileName = fileName;
+
+        // Open the file or create it
+        File tmpFile = new File( fileName );
+        boolean isNewFile = false;
+
+        if ( tmpFile.isDirectory() )
+        {
+            // It's a directory. Check that we don't have an existing mavibot file
+            File mavibotFile = new File( tmpFile, DEFAULT_FILE_NAME );
+
+            if ( !mavibotFile.exists() )
+            {
+                // We have to create a new file
+                try
+                {
+                    mavibotFile.createNewFile();
+                }
+                catch ( IOException ioe )
+                {
+                    LOG.error( "Cannot create the file {}", mavibotFile.getName() );
+                    return;
+                }
+            }
+
+            file = mavibotFile;
+        }
+        else
+        {
+            // It's a file. Let's see if it exists, otherwise create it
+            if ( !tmpFile.exists() )
+            {
+                isNewFile = true;
+
+                try
+                {
+                    tmpFile.createNewFile();
+                }
+                catch ( IOException ioe )
+                {
+                    LOG.error( "Cannot create the file {}", tmpFile.getName() );
+                    return;
+                }
+            }
+
+            file = tmpFile;
+        }
+
+        try
+        {
+            RandomAccessFile randomFile = new RandomAccessFile( file, "rw" );
+            fileChannel = randomFile.getChannel();
+
+            if ( isNewFile )
+            {
+                initRecordManager();
+            }
+            else
+            {
+                loadRecordManager();
+            }
+        }
+        catch ( IOException ioe )
+        {
+            LOG.error( "Error while initializing the RecordManager : {}", ioe.getMessage() );
+        }
+    }
+
+
+    /**
+     * We will create a brand new RecordManager file, containing nothing
+     */
+    public void initRecordManager() throws IOException
+    {
+        // Create a new Header
+        // The page size
+        HEADER_BUFFER.putInt( pageSize );
+
+        // The number of managed BTree (currently we have only one : the discardedPage BTree
+        HEADER_BUFFER.putInt( 1 );
+
+        // The first free page
+        HEADER_BUFFER.putLong( NO_PAGE );
+        firstFreePage = NO_PAGE;
+
+        // The last free page
+        HEADER_BUFFER.putLong( NO_PAGE );
+        lastFreePage = NO_PAGE;
+
+        // Now, initialize the Discarded Page BTree
+        copiedPageBTree = new BTree<Integer, long[]>( "copiedPageBTree", new IntSerializer(), new LongArraySerializer() );
+
+        // Inject this BTree into the RecordManager
+        try
+        {
+            manage( copiedPageBTree );
+        }
+        catch ( BTreeAlreadyManagedException btame )
+        {
+            // Can't happen here.
+        }
+
+        // We are all set !
+    }
+
+
+    /**
+     * We will load all the existing BTrees in this record manager.
+     */
+    public void loadRecordManager() throws IOException
+    {
+        if ( fileChannel.size() != 0 )
+        {
+            // The file exists, we have to load the data now 
+            fileChannel.read( HEADER_BUFFER );
+
+            // The page size
+            pageSize = HEADER_BUFFER.getInt();
+
+            blockBuffer = ByteBuffer.allocate( pageSize );
+
+            // The number of stored BTrees
+            nbBtree = HEADER_BUFFER.getInt();
+
+            // The first free page
+            firstFreePage = HEADER_BUFFER.getLong();
+            lastFreePage = HEADER_BUFFER.getLong();
+
+            // Read the meta-data header
+            ByteBuffer header = readHeader();
+
+            // Read the btrees
+            for ( int i = 0; i < nbBtree; i++ )
+            {
+                //----
+            }
+        }
+    }
+
+
+    /**
+     * Manage a BTree. The btree will be stored and managed by this RecordManager.
+     *  
+     * @param btree The new BTree to manage.
+     */
+    public synchronized void manage( BTree<?, ?> btree ) throws BTreeAlreadyManagedException, IOException
+    {
+        String name = btree.getName();
+
+        if ( managedBTrees.containsKey( name ) )
+        {
+            // There is already a BTree with this name in the recordManager...
+            LOG.error( "There is already a BTree named '{}' managed by this recordManager", name );
+            throw new BTreeAlreadyManagedException( name );
+        }
+
+        managedBTrees.put( name, btree );
+
+        // We will add the newly managed BTree at the end of the header.
+        byte[] btreeNameBytes = Strings.getBytesUtf8( name );
+        String keySerializerFqcn = btree.getKeySerializer().getClass().getName();
+        byte[] keySerializerBytes = Strings.getBytesUtf8( keySerializerFqcn );
+        String valueSerializerFqcn = btree.getKeySerializer().getClass().getName();
+        byte[] valueSerializerBytes = Strings.getBytesUtf8( valueSerializerFqcn );
+
+        int bufferSize = btreeNameBytes.length + keySerializerBytes.length + valueSerializerBytes.length;
+
+        // Get the pageIOs we need to store the data. We may need more than one.
+        PageIO[] pageIos = fetchPageIOs( bufferSize );
+
+        // Now store the data in the pages.
+        //storeData( pageIos, btreeNameBytes, keySerializerBytes, valueSerializerBytes );
+
+        // And flush the pages to disk now
+        //flushPages( pageIos );
+    }
+
+
+    /**
+     * Get as many pages as needed to store the data which size is provided
+     *  
+     * @param dataSize The data size
+     * @return An array of pages, enough to store the full data
+     */
+    private PageIO[] fetchPageIOs( int dataSize ) throws IOException
+    {
+        // Compute the number of pages needed.
+        // Considering that each page coan contain PageSize bytes,
+        // but that the first 8 bytes are used for links and we 
+        // use 4 bytes to store the data size, the number of needed
+        // pages is :
+        // NbPages = ( (dataSize - (PageSize - 8 - 4 )) / (PageSize - 8) ) + 1 
+        // NbPages += ( if (dataSize - (PageSize - 8 - 4 )) % (PageSize - 8) > 0 : 1 : 0 )
+        int availableSize = ( pageSize - 8 );
+        int nbNeededPages = 1;
+
+        // Compute the number of pages that will be full but the first page
+        if ( dataSize > availableSize + 4 )
+        {
+            int remainingSize = dataSize - ( availableSize + 4 );
+            nbNeededPages += remainingSize / availableSize;
+
+            if ( remainingSize % availableSize > 0 )
+            {
+                nbNeededPages++;
+            }
+        }
+
+        PageIO[] pageIOs = new PageIO[nbNeededPages];
+
+        for ( int i = 0; i < nbNeededPages; i++ )
+        {
+            pageIOs[i] = fetchNewPage();
+        }
+
+        return pageIOs;
+    }
+
+
+    /**
+     * Return a new Page 
+     * @return
+     */
+    private PageIO fetchNewPage() throws IOException
+    {
+        if ( firstFreePage == NO_PAGE )
+        {
+            // We don't have any free page. Reclaim some new page at the end
+            // of the file
+            long offset = fileChannel.size();
+            PageIO newPage = new PageIO( offset );
+
+            byte[] data = new byte[pageSize];
+
+            newPage.setData( data );
+            newPage.setNextPage( NO_PAGE );
+            newPage.setSize( -1 );
+        }
+        else
+        {
+            // We have some existing free page. Fetch one from there.
+        }
+
+        return null;
+    }
+
+
+    public void setPageSize( int pageSize )
+    {
+        if ( this.pageSize != -1 )
+        {
+        }
+        else
+        {
+            this.pageSize = pageSize;
+        }
+    }
+
+
+    /** 
+     * Read the header. It will contain all the BTree headers. The header is stored in
+     * the first block, which may be linked to some other blocks. The first block contains 
+     * the header's size on 4 bytes, then the data, and if we have a linked block, the last
+     * 8 bytes contain the reference on the next page.
+     * <br/>
+     * <pre>
+     * +----+--------+-----...---+    +--------+---------...----------+
+     * |Size|NextPage|    data   | -->|NextPage|        data          | --> ...
+     * +----+--------+-----...---+    +--------+---------...----------+
+     *   ^      ^
+     *   |      |
+     *   |      +--- Offset of the next page, or -1 if no linked page
+     *   |
+     *   +----------------------- Size of the header
+     * </pre>
+     * @return
+     */
+    private ByteBuffer readHeader() throws IOException
+    {
+        ByteBuffer dataBuffer = null;
+
+        // Read the first block
+        fileChannel.read( blockBuffer );
+
+        // Now, get its size, and check if we have more pages to read
+        int dataSize = blockBuffer.getInt();
+
+        dataBuffer = ByteBuffer.allocate( dataSize );
+
+        if ( dataSize + DATA_SIZE + LINK_SIZE > pageSize )
+        {
+            // We have more than one page to read
+            long nextPage = blockBuffer.getLong();
+
+            dataBuffer.put( blockBuffer );
+
+            dataSize -= pageSize - DATA_SIZE - LINK_SIZE;
+
+            // Loop on pages
+            while ( dataSize > 0 )
+            {
+                blockBuffer.clear();
+                fileChannel.read( blockBuffer, nextPage );
+
+                nextPage = blockBuffer.getLong();
+                dataSize -= pageSize - LINK_SIZE;
+
+                if ( nextPage == -1L )
+                {
+                    dataBuffer.put( blockBuffer.array(), LINK_SIZE, dataSize );
+                }
+                else
+                {
+                    dataBuffer.put( blockBuffer );
+                }
+            }
+        }
+        else
+        {
+            fileChannel.read( dataBuffer );
+        }
+
+        return dataBuffer;
+    }
+}



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