You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@directory.apache.org by el...@apache.org on 2013/12/18 16:12:15 UTC

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

Author: elecharny
Date: Wed Dec 18 15:12:15 2013
New Revision: 1551974

URL: http://svn.apache.org/r1551974
Log:
o Added the WriteTransaction class
o Renamed the Transaction class to ReadTransaction 
o Modified the PersistentBTree code to leverage the WriteTransaction, if created

Added:
    directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/ReadTransaction.java
      - copied, changed from r1551878, directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/Transaction.java
    directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/WriteTransaction.java
    directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/exception/BadTransactionStateException.java
Modified:
    directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractBTree.java
    directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractPage.java
    directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/BTree.java
    directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryBTree.java
    directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryLeaf.java
    directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/Page.java
    directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedBTree.java
    directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedLeaf.java
    directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/TupleCursor.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/AbstractBTree.java
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractBTree.java?rev=1551974&r1=1551973&r2=1551974&view=diff
==============================================================================
--- directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractBTree.java (original)
+++ directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractBTree.java Wed Dec 18 15:12:15 2013
@@ -35,7 +35,7 @@ import org.apache.directory.mavibot.btre
  *
  * @param <K> The Key type
  * @param <V> The Value type
- * 
+ *
  * @author <a href="mailto:dev@directory.apache.org">Apache Directory Project</a>
  */
 /* No qualifier*/abstract class AbstractBTree<K, V> implements BTree<K, V>
@@ -56,7 +56,7 @@ import org.apache.directory.mavibot.btre
     protected ElementSerializer<V> valueSerializer;
 
     /** The list of read transactions being executed */
-    protected ConcurrentLinkedQueue<Transaction<K, V>> readTransactions;
+    protected ConcurrentLinkedQueue<ReadTransaction<K, V>> readTransactions;
 
     /** The size of the buffer used to write data in disk */
     protected int writeBufferSize;
@@ -73,16 +73,19 @@ import org.apache.directory.mavibot.btre
     /** The BTree type : either in-memory, disk backed or persisted */
     private BTreeTypeEnum type;
 
+    /** The current transaction */
+    protected WriteTransaction writeTransaction;
+
 
     /**
-     * Starts a Read Only transaction. If the transaction is not closed, it will be 
+     * Starts a Read Only transaction. If the transaction is not closed, it will be
      * automatically closed after the timeout
-     * 
+     *
      * @return The created transaction
      */
-    protected Transaction<K, V> beginReadTransaction()
+    protected ReadTransaction<K, V> beginReadTransaction()
     {
-        Transaction<K, V> readTransaction = new Transaction<K, V>( rootPage, btreeHeader.getRevision() - 1,
+        ReadTransaction<K, V> readTransaction = new ReadTransaction<K, V>( rootPage, btreeHeader.getRevision() - 1,
             System.currentTimeMillis() );
 
         readTransactions.add( readTransaction );
@@ -96,7 +99,7 @@ import org.apache.directory.mavibot.btre
      */
     public TupleCursor<K, V> browse() throws IOException
     {
-        Transaction<K, V> transaction = beginReadTransaction();
+        ReadTransaction<K, V> transaction = beginReadTransaction();
 
         // Fetch the root page for this revision
         ParentPos<K, V>[] stack = new ParentPos[32];
@@ -115,7 +118,7 @@ import org.apache.directory.mavibot.btre
      */
     public TupleCursor<K, V> browse( long revision ) throws IOException, KeyNotFoundException
     {
-        Transaction<K, V> transaction = beginReadTransaction();
+        ReadTransaction<K, V> transaction = beginReadTransaction();
 
         // Fetch the root page for this revision
         Page<K, V> revisionRootPage = getRootPage( revision );
@@ -134,7 +137,7 @@ import org.apache.directory.mavibot.btre
      */
     public TupleCursor<K, V> browseFrom( K key ) throws IOException
     {
-        Transaction<K, V> transaction = beginReadTransaction();
+        ReadTransaction<K, V> transaction = beginReadTransaction();
 
         // Fetch the root page for this revision
         ParentPos<K, V>[] stack = new ParentPos[32];
@@ -150,7 +153,7 @@ import org.apache.directory.mavibot.btre
      */
     public TupleCursor<K, V> browseFrom( long revision, K key ) throws IOException, KeyNotFoundException
     {
-        Transaction<K, V> transaction = beginReadTransaction();
+        ReadTransaction<K, V> transaction = beginReadTransaction();
 
         // Fetch the rootPage for this revision
         Page<K, V> revisionRootPage = getRootPage( revision );
@@ -229,7 +232,7 @@ import org.apache.directory.mavibot.btre
     /**
      * Delete the entry which key is given as a parameter. If the entry exists, it will
      * be removed from the tree, the old tuple will be returned. Otherwise, null is returned.
-     * 
+     *
      * @param key The key for the entry we try to remove
      * @return A Tuple<K, V> containing the removed entry, or null if it's not found.
      */
@@ -253,8 +256,10 @@ import org.apache.directory.mavibot.btre
 
         try
         {
-            // Commented atm, we will have to play around the idea of transactions later
-            writeLock.lock();
+            if ( writeTransaction == null )
+            {
+                writeLock.lock();
+            }
 
             InsertResult<K, V> result = insert( key, value, revision );
 
@@ -266,7 +271,10 @@ import org.apache.directory.mavibot.btre
         finally
         {
             // See above
-            writeLock.unlock();
+            if ( writeTransaction == null )
+            {
+                writeLock.unlock();
+            }
         }
 
         return existingValue;
@@ -280,6 +288,15 @@ import org.apache.directory.mavibot.btre
 
 
     /**
+     * Flush the latest revision to disk. We will replace the current file by the new one, as
+     * we flush in a temporary file.
+     */
+    public void flush() throws IOException
+    {
+    }
+
+
+    /**
      * {@inheritDoc}
      */
     public V get( K key ) throws IOException, KeyNotFoundException
@@ -414,7 +431,7 @@ import org.apache.directory.mavibot.btre
     }
 
 
-    /** 
+    /**
      * {@inheritDoc}
      */
     public long getRevision()
@@ -434,7 +451,7 @@ import org.apache.directory.mavibot.btre
 
     /**
      * Generates a new revision number. It's only used by the Page instances.
-     * 
+     *
      * @return a new incremental revision number
      */
     /* no qualifier */long generateRevision()
@@ -461,7 +478,7 @@ import org.apache.directory.mavibot.btre
     }
 
 
-    /** 
+    /**
      * {@inheritDoc}
      */
     public long getNbElems()
@@ -614,7 +631,7 @@ import org.apache.directory.mavibot.btre
             {
                 try
                 {
-                    Transaction<K, V> transaction = null;
+                    ReadTransaction<K, V> transaction = null;
 
                     while ( !Thread.currentThread().isInterrupted() )
                     {

Modified: directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractPage.java
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractPage.java?rev=1551974&r1=1551973&r2=1551974&view=diff
==============================================================================
--- directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractPage.java (original)
+++ directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractPage.java Wed Dec 18 15:12:15 2013
@@ -173,7 +173,7 @@ import org.apache.directory.mavibot.btre
     /**
      * {@inheritDoc}
      */
-    public TupleCursor<K, V> browse( K key, Transaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
+    public TupleCursor<K, V> browse( K key, ReadTransaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
         throws IOException
     {
         int pos = findPos( key );
@@ -285,7 +285,7 @@ import org.apache.directory.mavibot.btre
     /**
      * {@inheritDoc}
      */
-    public TupleCursor<K, V> browse( Transaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
+    public TupleCursor<K, V> browse( ReadTransaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
         throws IOException
     {
         stack[depth++] = new ParentPos<K, V>( this, 0 );

Modified: directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/BTree.java
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/BTree.java?rev=1551974&r1=1551973&r2=1551974&view=diff
==============================================================================
--- directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/BTree.java (original)
+++ directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/BTree.java Wed Dec 18 15:12:15 2013
@@ -32,7 +32,7 @@ import org.apache.directory.mavibot.btre
  *
  * @param <K> The Key type
  * @param <V> The Value type
- * 
+ *
  * @author <a href="mailto:dev@directory.apache.org">Apache Directory Project</a>
  */
 public interface BTree<K, V>
@@ -55,7 +55,7 @@ public interface BTree<K, V>
 
     /**
      * Initialize the BTree.
-     * 
+     *
      * @throws IOException If we get some exception while initializing the BTree
      */
     void init() throws IOException;
@@ -74,7 +74,7 @@ public interface BTree<K, V>
      * If the provided size is below 2, we will default to DEFAULT_PAGE_SIZE.<br/>
      * If the provided size is not a power of 2, we will select the closest power of 2
      * higher than the given number<br/>
-     * 
+     *
      * @param pageSize The requested page size
      */
     void setPageSize( int pageSize );
@@ -103,7 +103,7 @@ public interface BTree<K, V>
     /**
      * Delete the entry which key is given as a parameter. If the entry exists, it will
      * be removed from the tree, the old tuple will be returned. Otherwise, null is returned.
-     * 
+     *
      * @param key The key for the entry we try to remove
      * @return A Tuple<K, V> containing the removed entry, or null if it's not found.
      */
@@ -112,10 +112,10 @@ public interface BTree<K, V>
 
     /**
      * Delete the value from an entry associated with the given key. If the value
-     * If the value is present, it will be deleted first, later if there are no other 
-     * values associated with this key(which can happen when duplicates are enabled), 
+     * If the value is present, it will be deleted first, later if there are no other
+     * values associated with this key(which can happen when duplicates are enabled),
      * we will remove the key from the tree.
-     * 
+     *
      * @param key The key for the entry we try to remove
      * @param value The value to delete (can be null)
      * @return A Tuple<K, V> containing the removed entry, or null if it's not found.
@@ -127,7 +127,7 @@ public interface BTree<K, V>
      * Find a value in the tree, given its key. If the key is not found,
      * it will throw a KeyNotFoundException. <br/>
      * Note that we can get a null value stored, or many values.
-     * 
+     *
      * @param key The key we are looking at
      * @return The found value, or null if the key is not present in the tree
      * @throws KeyNotFoundException If the key is not found in the BTree
@@ -138,7 +138,7 @@ public interface BTree<K, V>
 
     /**
      * Get the rootPage associated to a given revision.
-     * 
+     *
      * @param revision The revision we are looking for
      * @return The rootPage associated to this revision
      * @throws IOException If we had an issue while accessing the underlying file
@@ -149,7 +149,7 @@ public interface BTree<K, V>
 
     /**
      * Get the current rootPage
-     * 
+     *
      * @return The current rootPage
      */
     Page<K, V> getRootPage();
@@ -165,7 +165,7 @@ public interface BTree<K, V>
      * Find a value in the tree, given its key, at a specific revision. If the key is not found,
      * it will throw a KeyNotFoundException. <br/>
      * Note that we can get a null value stored, or many values.
-     * 
+     *
      * @param revision The revision for which we want to find a key
      * @param key The key we are looking at
      * @return The found value, or null if the key is not present in the tree
@@ -177,7 +177,7 @@ public interface BTree<K, V>
 
     /**
      * Checks if the given key exists.
-     *  
+     *
      * @param key The key we are looking at
      * @return true if the key is present, false otherwise
      * @throws IOException If we have an error while trying to access the page
@@ -187,7 +187,7 @@ public interface BTree<K, V>
 
     /**
      * Checks if the given key exists for a given revision.
-     *  
+     *
      * @param revision The revision for which we want to find a key
      * @param key The key we are looking at
      * @return true if the key is present, false otherwise
@@ -199,7 +199,7 @@ public interface BTree<K, V>
 
     /**
      * Checks if the BTree contains the given key with the given value.
-     * 
+     *
      * @param key The key we are looking for
      * @param value The value associated with the given key
      * @return true if the key and value are associated with each other, false otherwise
@@ -209,7 +209,7 @@ public interface BTree<K, V>
 
     /**
      * Checks if the BTree contains the given key with the given value for a given revision
-     * 
+     *
      * @param revision The revision we would like to browse
      * @param key The key we are looking for
      * @param value The value associated with the given key
@@ -221,7 +221,7 @@ public interface BTree<K, V>
 
     /**
      * Creates a cursor starting at the beginning of the tree
-     * 
+     *
      * @return A cursor on the btree
      * @throws IOException
      */
@@ -230,7 +230,7 @@ public interface BTree<K, V>
 
     /**
      * Creates a cursor starting at the beginning of the tree, for a given revision
-     * 
+     *
      * @param revision The revision we would like to browse
      * @return A cursor on the btree
      * @throws IOException If we had an issue while fetching data from the disk
@@ -241,7 +241,7 @@ public interface BTree<K, V>
 
     /**
      * Creates a cursor starting on the given key
-     * 
+     *
      * @param key The key which is the starting point. If the key is not found,
      * then the cursor will always return null.
      * @return A cursor on the btree
@@ -252,7 +252,7 @@ public interface BTree<K, V>
 
     /**
      * Creates a cursor starting on the given key at the given revision
-     * 
+     *
      * @param The revision we are looking for
      * @param key The key which is the starting point. If the key is not found,
      * then the cursor will always return null.
@@ -348,13 +348,13 @@ public interface BTree<K, V>
     String getValueSerializerFQCN();
 
 
-    /** 
+    /**
      * @return The current BTree revision
      */
     long getRevision();
 
 
-    /** 
+    /**
      * @return The current number of elements in the BTree
      */
     long getNbElems();
@@ -376,4 +376,22 @@ public interface BTree<K, V>
      * @return the type
      */
     BTreeTypeEnum getType();
+
+
+    /**
+     * Starts a transaction
+     */
+    void beginTransaction();
+
+
+    /**
+     * Commits a transaction
+     */
+    void commit();
+
+
+    /**
+     * Rollback a transaction
+     */
+    void rollback();
 }

Modified: directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryBTree.java
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryBTree.java?rev=1551974&r1=1551973&r2=1551974&view=diff
==============================================================================
--- directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryBTree.java (original)
+++ directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryBTree.java Wed Dec 18 15:12:15 2013
@@ -173,7 +173,7 @@ import org.slf4j.LoggerFactory;
         }
 
         // Create the queue containing the pending read transactions
-        readTransactions = new ConcurrentLinkedQueue<Transaction<K, V>>();
+        readTransactions = new ConcurrentLinkedQueue<ReadTransaction<K, V>>();
 
         writeLock = new ReentrantLock();
 
@@ -718,6 +718,33 @@ import org.slf4j.LoggerFactory;
 
 
     /**
+     * Starts a transaction
+     */
+    public void beginTransaction()
+    {
+        // Does nothing...
+    }
+
+
+    /**
+     * Commits a transaction
+     */
+    public void commit()
+    {
+        // Does nothing...
+    }
+
+
+    /**
+     * Rollback a transaction
+     */
+    public void rollback()
+    {
+        // Does nothing...
+    }
+
+
+    /**
      * @see Object#toString()
      */
     public String toString()

Modified: directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryLeaf.java
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryLeaf.java?rev=1551974&r1=1551973&r2=1551974&view=diff
==============================================================================
--- directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryLeaf.java (original)
+++ directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryLeaf.java Wed Dec 18 15:12:15 2013
@@ -581,7 +581,7 @@ import org.apache.directory.mavibot.btre
     /**
      * {@inheritDoc}
      */
-    public TupleCursor<K, V> browse( K key, Transaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
+    public TupleCursor<K, V> browse( K key, ReadTransaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
     {
         int pos = findPos( key );
         TupleCursor<K, V> cursor = null;
@@ -652,7 +652,7 @@ import org.apache.directory.mavibot.btre
     /**
      * {@inheritDoc}
      */
-    public TupleCursor<K, V> browse( Transaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
+    public TupleCursor<K, V> browse( ReadTransaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
     {
         int pos = 0;
         TupleCursor<K, V> cursor = null;

Modified: directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/Page.java
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/Page.java?rev=1551974&r1=1551973&r2=1551974&view=diff
==============================================================================
--- directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/Page.java (original)
+++ directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/Page.java Wed Dec 18 15:12:15 2013
@@ -132,7 +132,7 @@ import org.apache.directory.mavibot.btre
      * @return A Cursor to browse the next elements
      * @throws IOException If we have an error while trying to access the page
      */
-    TupleCursor<K, V> browse( K key, Transaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
+    TupleCursor<K, V> browse( K key, ReadTransaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
         throws IOException;
 
 
@@ -144,7 +144,7 @@ import org.apache.directory.mavibot.btre
      * @return A Cursor to browse the next elements
      * @throws IOException If we have an error while trying to access the page
      */
-    TupleCursor<K, V> browse( Transaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
+    TupleCursor<K, V> browse( ReadTransaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
         throws EndOfFileExceededException, IOException;
 
 

Modified: directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedBTree.java
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedBTree.java?rev=1551974&r1=1551973&r2=1551974&view=diff
==============================================================================
--- directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedBTree.java (original)
+++ directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedBTree.java Wed Dec 18 15:12:15 2013
@@ -74,6 +74,9 @@ public class PersistedBTree<K, V> extend
     /*No qualifier*/static int valueThresholdUp = DEFAULT_VALUE_THRESHOLD_UP;
     /*No qualifier*/static int valueThresholdLow = DEFAULT_VALUE_THRESHOLD_LOW;
 
+    /** A lock to protect the creation of the transaction */
+    protected ReentrantLock createTransaction = new ReentrantLock();
+
 
     /**
      * Creates a new BTree, with no initialization.
@@ -132,7 +135,7 @@ public class PersistedBTree<K, V> extend
             // The subBTree inherit its cache from its parent BTree
             this.cache = ( ( PersistedBTree<K, V> ) configuration.getParentBTree() ).getCache();
             this.writeLock = ( ( PersistedBTree<K, V> ) configuration.getParentBTree() ).getWriteLock();
-            readTransactions = new ConcurrentLinkedQueue<Transaction<K, V>>();
+            readTransactions = new ConcurrentLinkedQueue<ReadTransaction<K, V>>();
         }
 
         // Now, initialize the BTree
@@ -152,7 +155,7 @@ public class PersistedBTree<K, V> extend
             // This is not a subBtree, we have to initialize the cache
 
             // Create the queue containing the pending read transactions
-            readTransactions = new ConcurrentLinkedQueue<Transaction<K, V>>();
+            readTransactions = new ConcurrentLinkedQueue<ReadTransaction<K, V>>();
 
             writeLock = new ReentrantLock();
 
@@ -196,7 +199,7 @@ public class PersistedBTree<K, V> extend
     /**
      * Return the cache we use in this BTree
      */
-    /* No qualifier */ConcurrentLinkedQueue<Transaction<K, V>> getReadTransactions()
+    /* No qualifier */ConcurrentLinkedQueue<ReadTransaction<K, V>> getReadTransactions()
     {
         return readTransactions;
     }
@@ -544,14 +547,61 @@ public class PersistedBTree<K, V> extend
 
 
     /**
-     * Flush the latest revision to disk. We will replace the current file by the new one, as
-     * we flush in a temporary file.
+     * Starts a transaction
+     */
+    public void beginTransaction()
+    {
+        createTransaction.lock();
+
+        if ( writeTransaction == null )
+        {
+            writeTransaction = new WriteTransaction( recordManager );
+        }
+
+        createTransaction.unlock();
+
+        writeTransaction.start();
+    }
+
+
+    /**
+     * Commits a transaction
+     */
+    public void commit()
+    {
+        createTransaction.lock();
+
+        if ( writeTransaction == null )
+        {
+            writeTransaction = new WriteTransaction( recordManager );
+        }
+
+        createTransaction.unlock();
+
+        writeTransaction.commit();
+    }
+
+
+    /**
+     * Rollback a transaction
      */
-    public void flush() throws IOException
+    public void rollback()
     {
+        createTransaction.lock();
+
+        if ( writeTransaction == null )
+        {
+            writeTransaction = new WriteTransaction( recordManager );
+        }
+
+        createTransaction.unlock();
+
+        writeTransaction.rollback();
     }
 
 
+
+
     /**
      * @see Object#toString()
      */

Modified: directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedLeaf.java
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedLeaf.java?rev=1551974&r1=1551973&r2=1551974&view=diff
==============================================================================
--- directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedLeaf.java (original)
+++ directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedLeaf.java Wed Dec 18 15:12:15 2013
@@ -630,7 +630,7 @@ import org.apache.directory.mavibot.btre
     /**
      * {@inheritDoc}
      */
-    public TupleCursor<K, V> browse( K key, Transaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
+    public TupleCursor<K, V> browse( K key, ReadTransaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
     {
         int pos = findPos( key );
         TupleCursor<K, V> cursor = null;
@@ -701,7 +701,7 @@ import org.apache.directory.mavibot.btre
     /**
      * {@inheritDoc}
      */
-    public TupleCursor<K, V> browse( Transaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
+    public TupleCursor<K, V> browse( ReadTransaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
     {
         int pos = 0;
         TupleCursor<K, V> cursor = null;

Copied: directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/ReadTransaction.java (from r1551878, directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/Transaction.java)
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/ReadTransaction.java?p2=directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/ReadTransaction.java&p1=directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/Transaction.java&r1=1551878&r2=1551974&rev=1551974&view=diff
==============================================================================
--- directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/Transaction.java (original)
+++ directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/ReadTransaction.java Wed Dec 18 15:12:15 2013
@@ -41,7 +41,7 @@ import java.util.Date;
  * @param <K> The type for the Key
  * @param <V> The type for the stored value
  */
-public class Transaction<K, V>
+public class ReadTransaction<K, V>
 {
     /** The associated revision */
     private long revision;
@@ -63,7 +63,7 @@ public class Transaction<K, V>
      * @param revision The revision this transaction is using
      * @param creationDate The creation date for this transaction
      */
-    public Transaction( Page<K, V> root, long revision, long creationDate )
+    public ReadTransaction( Page<K, V> root, long revision, long creationDate )
     {
         this.revision = revision;
         this.creationDate = creationDate;

Modified: directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/TupleCursor.java
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/TupleCursor.java?rev=1551974&r1=1551973&r2=1551974&view=diff
==============================================================================
--- directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/TupleCursor.java (original)
+++ directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/TupleCursor.java Wed Dec 18 15:12:15 2013
@@ -52,7 +52,7 @@ public class TupleCursor<K, V>
     protected int depth = 0;
 
     /** The transaction used for this cursor */
-    protected Transaction<K, V> transaction;
+    protected ReadTransaction<K, V> transaction;
 
     /** The Tuple used to return the results */
     protected Tuple<K, V> tuple = new Tuple<K, V>();
@@ -64,7 +64,7 @@ public class TupleCursor<K, V>
      * @param transaction The transaction this operation is protected by
      * @param stack The stack of parent's from root to this page
      */
-    public TupleCursor( Transaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
+    public TupleCursor( ReadTransaction<K, V> transaction, ParentPos<K, V>[] stack, int depth )
     {
         this.transaction = transaction;
         this.stack = stack;

Added: directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/WriteTransaction.java
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/WriteTransaction.java?rev=1551974&view=auto
==============================================================================
--- directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/WriteTransaction.java (added)
+++ directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/WriteTransaction.java Wed Dec 18 15:12:15 2013
@@ -0,0 +1,78 @@
+/*
+ *  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.util.concurrent.locks.ReentrantLock;
+
+import org.apache.directory.mavibot.btree.exception.BadTransactionStateException;
+
+/**
+ * A data structure used to manage a write transaction
+ *
+ * @author <a href="mailto:dev@directory.apache.org">Apache Directory Project</a>
+ */
+/* no qualifier */ class WriteTransaction
+{
+    /** The recordManager on which this transaction is applied */
+    private RecordManager recordManager;
+
+    /** A lock used to protect the write operation against concurrent access */
+    protected ReentrantLock writeLock;
+
+    /* no qualifier */WriteTransaction( RecordManager recordManager )
+    {
+        System.out.println( "Creating the transaction oject" );
+        this.recordManager = recordManager;
+        writeLock = new ReentrantLock();
+    }
+
+
+    /* no qualifier */ void start()
+    {
+        if ( writeLock.isLocked() )
+        {
+            throw new BadTransactionStateException( "Cannot start a write transaction when it's already started" );
+        }
+
+        writeLock.lock();
+    }
+
+
+    /* no qualifier */ void commit()
+    {
+        if ( !writeLock.isLocked() )
+        {
+            throw new BadTransactionStateException( "Cannot commit a write transaction when it's not started" );
+        }
+
+        writeLock.unlock();
+    }
+
+
+    /* no qualifier */ void rollback()
+    {
+        if ( !writeLock.isLocked() )
+        {
+            throw new BadTransactionStateException( "Cannot commit a write transaction when it's not started" );
+        }
+
+        writeLock.unlock();
+    }
+}

Added: directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/exception/BadTransactionStateException.java
URL: http://svn.apache.org/viewvc/directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/exception/BadTransactionStateException.java?rev=1551974&view=auto
==============================================================================
--- directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/exception/BadTransactionStateException.java (added)
+++ directory/mavibot/trunk/mavibot/src/main/java/org/apache/directory/mavibot/btree/exception/BadTransactionStateException.java Wed Dec 18 15:12:15 2013
@@ -0,0 +1,72 @@
+/*
+ *  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.exception;
+
+
+/**
+ * An exception thrown when we try to use a transaction in a wrong state
+ *
+ * @author <a href="mailto:dev@directory.apache.org">Apache Directory Project</a>
+ */
+public class BadTransactionStateException extends RuntimeException
+{
+    /** The serial version UUID */
+    private static final long serialVersionUID = 1L;
+
+
+    /**
+     * Creates a new instance of BadTransactionStateException.
+     */
+    public BadTransactionStateException()
+    {
+    }
+
+
+    /**
+     * Creates a new instance of BadTransactionStateException.
+     *
+     * @param explanation The message associated with the exception
+     */
+    public BadTransactionStateException( String explanation )
+    {
+        super( explanation );
+    }
+
+
+    /**
+     * Creates a new instance of BadTransactionStateException.
+     */
+    public BadTransactionStateException( Throwable cause )
+    {
+        super( cause );
+    }
+
+
+    /**
+     * Creates a new instance of BadTransactionStateException.
+     *
+     * @param explanation The message associated with the exception
+     * @param cause The root cause for this exception
+     */
+    public BadTransactionStateException( String explanation, Throwable cause )
+    {
+        super( explanation, cause );
+    }
+}

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=1551974&r1=1551973&r2=1551974&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 Wed Dec 18 15:12:15 2013
@@ -61,7 +61,7 @@ public class RecordManagerTest
 {
     private BTree<Long, String> btree = null;
 
-    private RecordManager recordManager1 = null;
+    private RecordManager recordManager = null;
 
     @Rule
     public TemporaryFolder tempFolder = new TemporaryFolder();
@@ -79,7 +79,7 @@ public class RecordManagerTest
         try
         {
             // Create a new BTree
-            btree = recordManager1.addBTree( "test", new LongSerializer(), new StringSerializer(), false );
+            btree = recordManager.addBTree( "test", new LongSerializer(), new StringSerializer(), false );
         }
         catch ( Exception e )
         {
@@ -104,18 +104,18 @@ public class RecordManagerTest
     {
         try
         {
-            if ( recordManager1 != null )
+            if ( recordManager != null )
             {
-                recordManager1.close();
+                recordManager.close();
             }
 
             // Now, try to reload the file back
-            recordManager1 = new RecordManager( dataDir.getAbsolutePath() );
+            recordManager = new RecordManager( dataDir.getAbsolutePath() );
 
             // load the last created btree
             if ( btree != null )
             {
-                btree = recordManager1.getManagedTree( btree.getName() );
+                btree = recordManager.getManagedTree( btree.getName() );
             }
         }
         catch ( Exception e )
@@ -131,14 +131,14 @@ public class RecordManagerTest
     @Test
     public void testRecordManager() throws IOException, BTreeAlreadyManagedException
     {
-        assertEquals( 1, recordManager1.getNbManagedTrees() );
+        assertEquals( 1, recordManager.getNbManagedTrees() );
 
-        Set<String> managedBTrees = recordManager1.getManagedTrees();
+        Set<String> managedBTrees = recordManager.getManagedTrees();
 
         assertEquals( 1, managedBTrees.size() );
         assertTrue( managedBTrees.contains( "test" ) );
 
-        BTree<Long, String> btree1 = recordManager1.getManagedTree( "test" );
+        BTree<Long, String> btree1 = recordManager.getManagedTree( "test" );
 
         assertNotNull( btree1 );
         assertEquals( btree.getComparator().getClass().getName(), btree1.getComparator().getClass().getName() );
@@ -165,14 +165,14 @@ public class RecordManagerTest
         // Now, try to reload the file back
         openRecordManagerAndBtree();
 
-        assertEquals( 1, recordManager1.getNbManagedTrees() );
+        assertEquals( 1, recordManager.getNbManagedTrees() );
 
-        Set<String> managedBTrees = recordManager1.getManagedTrees();
+        Set<String> managedBTrees = recordManager.getManagedTrees();
 
         assertEquals( 1, managedBTrees.size() );
         assertTrue( managedBTrees.contains( "test" ) );
 
-        BTree<Long, String> btree1 = recordManager1.getManagedTree( "test" );
+        BTree<Long, String> btree1 = recordManager.getManagedTree( "test" );
 
         assertNotNull( btree1 );
         assertEquals( btree.getComparator().getClass().getName(), btree1.getComparator().getClass().getName() );
@@ -209,14 +209,14 @@ public class RecordManagerTest
         // Now, try to reload the file back
         openRecordManagerAndBtree();
 
-        assertEquals( 1, recordManager1.getNbManagedTrees() );
+        assertEquals( 1, recordManager.getNbManagedTrees() );
 
-        Set<String> managedBTrees = recordManager1.getManagedTrees();
+        Set<String> managedBTrees = recordManager.getManagedTrees();
 
         assertEquals( 1, managedBTrees.size() );
         assertTrue( managedBTrees.contains( "test" ) );
 
-        BTree<Long, String> btree1 = recordManager1.getManagedTree( "test" );
+        BTree<Long, String> btree1 = recordManager.getManagedTree( "test" );
 
         assertNotNull( btree1 );
         assertEquals( btree.getComparator().getClass().getName(), btree1.getComparator().getClass().getName() );
@@ -245,7 +245,7 @@ public class RecordManagerTest
         KeyNotFoundException
     {
         // Don't keep any revision
-        recordManager1.setKeepRevisions( false );
+        recordManager.setKeepRevisions( false );
 
         String fileName = dataDir.getAbsolutePath() + "/mavibot.db";
         File file = new File( fileName );
@@ -273,7 +273,7 @@ public class RecordManagerTest
             {
                 fileSize = file.length();
                 System.out.println( "----- Size after insertion of " + i + " = " + fileSize );
-                System.out.println( recordManager1 );
+                System.out.println( recordManager );
                 //System.out.println( btree );
             }
         }
@@ -283,22 +283,22 @@ public class RecordManagerTest
         System.out.println( "Size after insertion of 100 000 elements : " + fileSize );
         System.out.println( "Time taken to write 100 000 elements : " + ( t1 - t0 ) );
         System.out.println( "  Nb elem/s : " + ( ( nbElems * 1000 ) / ( t1 - t0 ) ) );
-        System.out.println( "Nb created page " + recordManager1.nbCreatedPages.get() );
-        System.out.println( "Nb allocated page " + recordManager1.nbReusedPages.get() );
-        System.out.println( "Nb page we have freed " + recordManager1.nbFreedPages.get() );
-        System.out.println( recordManager1 );
+        System.out.println( "Nb created page " + recordManager.nbCreatedPages.get() );
+        System.out.println( "Nb allocated page " + recordManager.nbReusedPages.get() );
+        System.out.println( "Nb page we have freed " + recordManager.nbFreedPages.get() );
+        System.out.println( recordManager );
 
         // Now, try to reload the file back
         openRecordManagerAndBtree();
 
-        assertEquals( 1, recordManager1.getNbManagedTrees() );
+        assertEquals( 1, recordManager.getNbManagedTrees() );
 
-        Set<String> managedBTrees = recordManager1.getManagedTrees();
+        Set<String> managedBTrees = recordManager.getManagedTrees();
 
         assertEquals( 1, managedBTrees.size() );
         assertTrue( managedBTrees.contains( "test" ) );
 
-        BTree<Long, String> btree1 = recordManager1.getManagedTree( "test" );
+        BTree<Long, String> btree1 = recordManager.getManagedTree( "test" );
 
         assertNotNull( btree1 );
         assertEquals( btree.getComparator().getClass().getName(), btree1.getComparator().getClass().getName() );
@@ -404,7 +404,7 @@ public class RecordManagerTest
     public void testRecordManagerBrowseWithKeepRevisions() throws IOException, BTreeAlreadyManagedException,
         KeyNotFoundException
     {
-        recordManager1.setKeepRevisions( true );
+        recordManager.setKeepRevisions( true );
 
         // Now, add some elements in the BTree
         btree.insert( 3L, "V3" );
@@ -429,14 +429,14 @@ public class RecordManagerTest
         // Now, try to reload the file back
         openRecordManagerAndBtree();
 
-        assertEquals( 1, recordManager1.getNbManagedTrees() );
+        assertEquals( 1, recordManager.getNbManagedTrees() );
 
-        Set<String> managedBTrees = recordManager1.getManagedTrees();
+        Set<String> managedBTrees = recordManager.getManagedTrees();
 
         assertEquals( 1, managedBTrees.size() );
         assertTrue( managedBTrees.contains( "test" ) );
 
-        BTree<Long, String> btree1 = recordManager1.getManagedTree( "test" );
+        BTree<Long, String> btree1 = recordManager.getManagedTree( "test" );
 
         assertNotNull( btree1 );
         assertEquals( btree.getComparator().getClass().getName(), btree1.getComparator().getClass().getName() );
@@ -475,7 +475,7 @@ public class RecordManagerTest
     public void testRecordManagerBrowseFromWithRevision() throws IOException, BTreeAlreadyManagedException,
         KeyNotFoundException
     {
-        recordManager1.setKeepRevisions( true );
+        recordManager.setKeepRevisions( true );
 
         // Now, add some elements in the BTree
         btree.insert( 3L, "V3" );
@@ -500,14 +500,14 @@ public class RecordManagerTest
         // Now, try to reload the file back
         openRecordManagerAndBtree();
 
-        assertEquals( 1, recordManager1.getNbManagedTrees() );
+        assertEquals( 1, recordManager.getNbManagedTrees() );
 
-        Set<String> managedBTrees = recordManager1.getManagedTrees();
+        Set<String> managedBTrees = recordManager.getManagedTrees();
 
         assertEquals( 1, managedBTrees.size() );
         assertTrue( managedBTrees.contains( "test" ) );
 
-        BTree<Long, String> btree1 = recordManager1.getManagedTree( "test" );
+        BTree<Long, String> btree1 = recordManager.getManagedTree( "test" );
 
         assertNotNull( btree1 );
         assertEquals( btree.getComparator().getClass().getName(), btree1.getComparator().getClass().getName() );
@@ -545,7 +545,7 @@ public class RecordManagerTest
     public void testGetWithRevision() throws IOException, BTreeAlreadyManagedException,
         KeyNotFoundException
     {
-        recordManager1.setKeepRevisions( true );
+        recordManager.setKeepRevisions( true );
 
         // Now, add some elements in the BTree
         btree.insert( 3L, "V3" );
@@ -591,14 +591,14 @@ public class RecordManagerTest
         // Now, try to reload the file back
         openRecordManagerAndBtree();
 
-        assertEquals( 1, recordManager1.getNbManagedTrees() );
+        assertEquals( 1, recordManager.getNbManagedTrees() );
 
-        Set<String> managedBTrees = recordManager1.getManagedTrees();
+        Set<String> managedBTrees = recordManager.getManagedTrees();
 
         assertEquals( 1, managedBTrees.size() );
         assertTrue( managedBTrees.contains( "test" ) );
 
-        BTree<Long, String> btree1 = recordManager1.getManagedTree( "test" );
+        BTree<Long, String> btree1 = recordManager.getManagedTree( "test" );
 
         assertNotNull( btree1 );
         assertEquals( btree.getComparator().getClass().getName(), btree1.getComparator().getClass().getName() );
@@ -652,7 +652,7 @@ public class RecordManagerTest
     public void testContainWithRevision() throws IOException, BTreeAlreadyManagedException,
         KeyNotFoundException
     {
-        recordManager1.setKeepRevisions( true );
+        recordManager.setKeepRevisions( true );
 
         // Now, add some elements in the BTree
         btree.insert( 3L, "V3" );
@@ -692,14 +692,14 @@ public class RecordManagerTest
         // Now, try to reload the file back
         openRecordManagerAndBtree();
 
-        assertEquals( 1, recordManager1.getNbManagedTrees() );
+        assertEquals( 1, recordManager.getNbManagedTrees() );
 
-        Set<String> managedBTrees = recordManager1.getManagedTrees();
+        Set<String> managedBTrees = recordManager.getManagedTrees();
 
         assertEquals( 1, managedBTrees.size() );
         assertTrue( managedBTrees.contains( "test" ) );
 
-        BTree<Long, String> btree1 = recordManager1.getManagedTree( "test" );
+        BTree<Long, String> btree1 = recordManager.getManagedTree( "test" );
 
         assertNotNull( btree1 );
         assertEquals( btree.getComparator().getClass().getName(), btree1.getComparator().getClass().getName() );
@@ -747,7 +747,7 @@ public class RecordManagerTest
     public void testHasKeyWithRevision() throws IOException, BTreeAlreadyManagedException,
         KeyNotFoundException
     {
-        recordManager1.setKeepRevisions( true );
+        recordManager.setKeepRevisions( true );
 
         // Now, add some elements in the BTree
         btree.insert( 3L, "V3" );
@@ -787,14 +787,14 @@ public class RecordManagerTest
         // Now, try to reload the file back
         openRecordManagerAndBtree();
 
-        assertEquals( 1, recordManager1.getNbManagedTrees() );
+        assertEquals( 1, recordManager.getNbManagedTrees() );
 
-        Set<String> managedBTrees = recordManager1.getManagedTrees();
+        Set<String> managedBTrees = recordManager.getManagedTrees();
 
         assertEquals( 1, managedBTrees.size() );
         assertTrue( managedBTrees.contains( "test" ) );
 
-        BTree<Long, String> btree1 = recordManager1.getManagedTree( "test" );
+        BTree<Long, String> btree1 = recordManager.getManagedTree( "test" );
 
         assertNotNull( btree1 );
         assertEquals( btree.getComparator().getClass().getName(), btree1.getComparator().getClass().getName() );
@@ -851,7 +851,7 @@ public class RecordManagerTest
         BTree<Long, String> dupsTree = BTreeFactory.createPersistedBTree( name, new LongSerializer(),
             new StringSerializer(), pageSize, true );
 
-        recordManager1.manage( dupsTree );
+        recordManager.manage( dupsTree );
 
         for ( long i = 0; i < numKeys; i++ )
         {
@@ -864,7 +864,7 @@ public class RecordManagerTest
         // Now, try to reload the file back
         openRecordManagerAndBtree();
 
-        dupsTree = recordManager1.getManagedTree( name );
+        dupsTree = recordManager.getManagedTree( name );
 
         for ( long i = 0; i < numKeys; i++ )
         {