Return-Path: X-Original-To: apmail-directory-commits-archive@www.apache.org Delivered-To: apmail-directory-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 12E1C108D7 for ; Tue, 29 Apr 2014 05:17:47 +0000 (UTC) Received: (qmail 56886 invoked by uid 500); 29 Apr 2014 05:17:01 -0000 Delivered-To: apmail-directory-commits-archive@directory.apache.org Received: (qmail 56686 invoked by uid 500); 29 Apr 2014 05:16:58 -0000 Mailing-List: contact commits-help@directory.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@directory.apache.org Delivered-To: mailing list commits@directory.apache.org Received: (qmail 56641 invoked by uid 99); 29 Apr 2014 05:16:56 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 29 Apr 2014 05:16:56 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 29 Apr 2014 05:16:44 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 5E8962388A9B; Tue, 29 Apr 2014 05:16:19 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1590872 [4/5] - in /directory/mavibot/branches/with-txns/mavibot: img/ src/main/java/org/apache/directory/mavibot/btree/ src/test/java/org/apache/directory/mavibot/btree/ Date: Tue, 29 Apr 2014 05:16:17 -0000 To: commits@directory.apache.org From: elecharny@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20140429051619.5E8962388A9B@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Added: directory/mavibot/branches/with-txns/mavibot/img/transok.graphml URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/img/transok.graphml?rev=1590872&view=auto ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/img/transok.graphml (added) +++ directory/mavibot/branches/with-txns/mavibot/img/transok.graphml Tue Apr 29 05:16:16 2014 @@ -0,0 +1,374 @@ + + + + + + + + + + + + + + + + + + + + + + + r 7 + + + + + + + + + + + + + + + + B-tree A + + + + + + + + + + + + + + + + r 14 + + + + + + + + + + + + + + + + B-tree B + + + + + + + + + + + + + + + + r 3 + + + + + + + + + + + + + + + + B-tree C + + + + + + + + + + + + + + + + r 25 + + + + + + + + + + + + + + + + BOB + + + + + + + + + + + + + + + + RMHeader + + + + + + + + + + + + + + + + currentBOB + + + + + + + + + + + + + + + + r 8 + + + + + + + + + + + + + + + + B-tree A + + + + + + + + + + + + + + + + r 15 + + + + + + + + + + + + + + + + B-tree B + + + + + + + + + + + + + + + + r 27 + + + + + + + + + + + + + + + + BOB + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Added: directory/mavibot/branches/with-txns/mavibot/img/transok.png URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/img/transok.png?rev=1590872&view=auto ============================================================================== Binary file - no diff available. Propchange: directory/mavibot/branches/with-txns/mavibot/img/transok.png ------------------------------------------------------------------------------ svn:mime-type = application/octet-stream Modified: directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractBTree.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractBTree.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractBTree.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractBTree.java Tue Apr 29 05:16:16 2014 @@ -91,6 +91,9 @@ import org.apache.directory.mavibot.btre /** The current revision */ protected AtomicLong currentRevision = new AtomicLong( 0L ); + + /** The TransactionManager used for this BTree */ + protected TransactionManager transactionManager; /** * Starts a Read Only transaction. If the transaction is not closed, it will be @@ -98,16 +101,7 @@ import org.apache.directory.mavibot.btre * * @return The created transaction */ - protected ReadTransaction beginReadTransaction() - { - BTreeHeader btreeHeader = getBtreeHeader(); - - ReadTransaction readTransaction = new ReadTransaction( btreeHeader ); - - readTransactions.add( readTransaction ); - - return readTransaction; - } + protected abstract ReadTransaction beginReadTransaction(); /** @@ -116,29 +110,13 @@ import org.apache.directory.mavibot.btre * * @return The created transaction */ - protected ReadTransaction beginReadTransaction( long revision ) - { - BTreeHeader btreeHeader = getBtreeHeader( revision ); - - if ( btreeHeader != null ) - { - ReadTransaction readTransaction = new ReadTransaction( btreeHeader ); - - readTransactions.add( readTransaction ); - - return readTransaction; - } - else - { - return null; - } - } + protected abstract ReadTransaction beginReadTransaction( long revision ); /** * {@inheritDoc} */ - public TupleCursor browse() throws IOException + public TupleCursor browse() throws IOException, KeyNotFoundException { ReadTransaction transaction = beginReadTransaction(); @@ -150,7 +128,7 @@ import org.apache.directory.mavibot.btre { ParentPos[] stack = (ParentPos[]) Array.newInstance( ParentPos.class, 32 ); - TupleCursor cursor = transaction.getRootPage().browse( transaction, stack, 0 ); + TupleCursor cursor = getRootPage().browse( transaction, stack, 0 ); // Set the position before the first element cursor.beforeFirst(); @@ -176,7 +154,7 @@ import org.apache.directory.mavibot.btre ParentPos[] stack = (ParentPos[]) Array.newInstance( ParentPos.class, 32 ); // And get the cursor - TupleCursor cursor = transaction.getRootPage().browse( transaction, stack, 0 ); + TupleCursor cursor = getRootPage( transaction.getRevision() ).browse( transaction, stack, 0 ); return cursor; } @@ -192,9 +170,17 @@ import org.apache.directory.mavibot.btre ParentPos[] stack = (ParentPos[]) Array.newInstance( ParentPos.class, 32 ); - TupleCursor cursor = transaction.getRootPage().browse( key, transaction, stack, 0 ); - - return cursor; + TupleCursor cursor; + try + { + cursor = getRootPage( transaction.getRevision() ).browse( key, transaction, stack, 0 ); + + return cursor; + } + catch ( KeyNotFoundException e ) + { + throw new IOException( e.getMessage() ); + } } @@ -214,7 +200,7 @@ import org.apache.directory.mavibot.btre ParentPos[] stack = (ParentPos[]) Array.newInstance( ParentPos.class, 32 ); // And get the cursor - TupleCursor cursor = transaction.getRootPage().browse( key, transaction, stack, 0 ); + TupleCursor cursor = getRootPage( transaction.getRevision() ).browse( key, transaction, stack, 0 ); return cursor; } @@ -236,7 +222,11 @@ import org.apache.directory.mavibot.btre { try { - return transaction.getRootPage().contains( key, value ); + return getRootPage( transaction.getRevision() ).contains( key, value ); + } + catch ( KeyNotFoundException knfe ) + { + throw new IOException( knfe.getMessage() ); } finally { @@ -262,7 +252,7 @@ import org.apache.directory.mavibot.btre { try { - return transaction.getRootPage().contains( key, value ); + return getRootPage( transaction.getRevision() ).contains( key, value ); } finally { @@ -303,9 +293,22 @@ import org.apache.directory.mavibot.btre throw new IllegalArgumentException( "Value must not be null" ); } - Tuple deleted = delete( key, value, currentRevision.get() + 1 ); + transactionManager.beginTransaction(); - return deleted; + try + { + Tuple deleted = delete( key, value, currentRevision.get() + 1 ); + + transactionManager.commit(); + + return deleted; + } + catch ( IOException ioe ) + { + transactionManager.rollback(); + + throw ioe; + } } @@ -332,14 +335,35 @@ import org.apache.directory.mavibot.btre { V existingValue = null; - InsertResult result = insert( key, value, -1L ); - - if ( result instanceof ModifyResult ) + if ( key == null ) { - existingValue = ( ( ModifyResult ) result ).getModifiedValue(); + throw new IllegalArgumentException( "Key must not be null" ); } - return existingValue; + // Take the lock if it's not already taken by another thread + transactionManager.beginTransaction(); + + try + { + InsertResult result = insert( key, value, -1L ); + + if ( result instanceof ModifyResult ) + { + existingValue = ( ( ModifyResult ) result ).getModifiedValue(); + } + + // Commit now + transactionManager.commit(); + + return existingValue; + } + catch ( IOException ioe ) + { + // We have had an exception, we must rollback the transaction + transactionManager.rollback(); + + return null; + } } @@ -373,7 +397,7 @@ import org.apache.directory.mavibot.btre { try { - return transaction.getRootPage().get( key ); + return getRootPage( transaction.getRevision() ).get( key ); } finally { @@ -398,7 +422,7 @@ import org.apache.directory.mavibot.btre { try { - return transaction.getRootPage().get( key ); + return getRootPage( transaction.getRevision() ).get( key ); } finally { @@ -435,7 +459,7 @@ import org.apache.directory.mavibot.btre { try { - return transaction.getRootPage().getValues( key ); + return getRootPage( transaction.getRevision() ).getValues( key ); } finally { @@ -448,7 +472,7 @@ import org.apache.directory.mavibot.btre /** * {@inheritDoc} */ - public boolean hasKey( K key ) throws IOException + public boolean hasKey( K key ) throws IOException, KeyNotFoundException { if ( key == null ) { @@ -465,7 +489,7 @@ import org.apache.directory.mavibot.btre { try { - return transaction.getRootPage().hasKey( key ); + return getRootPage( transaction.getRevision() ).hasKey( key ); } finally { @@ -495,7 +519,7 @@ import org.apache.directory.mavibot.btre { try { - return transaction.getRootPage().hasKey( key ); + return getRootPage( transaction.getRevision() ).hasKey( key ); } finally { Modified: directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractPage.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractPage.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractPage.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractPage.java Tue Apr 29 05:16:16 2014 @@ -214,7 +214,6 @@ import org.apache.directory.mavibot.btre /** * {@inheritDoc} */ - @SuppressWarnings("unchecked") public DeleteResult delete( K key, V value, long revision ) throws IOException { return delete( key, value, revision, null, -1 ); Added: directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractTransactionManager.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractTransactionManager.java?rev=1590872&view=auto ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractTransactionManager.java (added) +++ directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractTransactionManager.java Tue Apr 29 05:16:16 2014 @@ -0,0 +1,35 @@ +/* + * 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.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * An abstract class implementing the TransactionManager interface. + * + * @author Apache Directory Project + */ +public abstract class AbstractTransactionManager implements TransactionManager +{ + /** A lock to protect the transaction handling */ + private ReadWriteLock transactionLock = new ReentrantReadWriteLock(); + +} Modified: directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractValueHolder.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractValueHolder.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractValueHolder.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/AbstractValueHolder.java Tue Apr 29 05:16:16 2014 @@ -24,6 +24,7 @@ import java.io.IOException; import java.lang.reflect.Array; import java.util.Comparator; +import org.apache.directory.mavibot.btree.exception.KeyNotFoundException; import org.apache.directory.mavibot.btree.serializer.ElementSerializer; @@ -244,6 +245,10 @@ import org.apache.directory.mavibot.btre e.printStackTrace(); return false; } + catch ( KeyNotFoundException knfe ) + { + knfe.printStackTrace();return false; + } } Modified: directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/BTree.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/BTree.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/BTree.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/BTree.java Tue Apr 29 05:16:16 2014 @@ -173,8 +173,9 @@ public interface BTree * @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 + * @throws KeyNotFoundException If the key is not found in the B-tree */ - boolean hasKey( K key ) throws IOException; + boolean hasKey( K key ) throws IOException, KeyNotFoundException; /** @@ -217,7 +218,7 @@ public interface BTree * @return A cursor on the B-tree * @throws IOException */ - TupleCursor browse() throws IOException; + TupleCursor browse() throws IOException, KeyNotFoundException; /** @@ -368,22 +369,4 @@ public interface BTree * @return the type */ BTreeTypeEnum getType(); - - - /** - * Starts a transaction - */ - void beginTransaction(); - - - /** - * Commits a transaction - */ - void commit(); - - - /** - * Rollback a transaction - */ - void rollback(); } Modified: directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryBTree.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryBTree.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryBTree.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryBTree.java Tue Apr 29 05:16:16 2014 @@ -29,8 +29,6 @@ import java.io.RandomAccessFile; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.directory.mavibot.btree.exception.InitializationException; import org.apache.directory.mavibot.btree.exception.KeyNotFoundException; @@ -49,7 +47,7 @@ import org.slf4j.LoggerFactory; * * @author Apache Directory Project */ -/* No qualifier */class InMemoryBTree extends AbstractBTree implements TransactionManager, Closeable +/* No qualifier */class InMemoryBTree extends AbstractBTree implements Closeable { /** The LoggerFactory used by this class */ protected static final Logger LOG = LoggerFactory.getLogger( InMemoryBTree.class ); @@ -79,9 +77,6 @@ import org.slf4j.LoggerFactory; /** The Journal channel */ private FileChannel journalChannel = null; - /** A lock to protect the transaction handling */ - private ReadWriteLock transactionLock = new ReentrantReadWriteLock(); - /** * Creates a new BTree, with no initialization. */ @@ -184,6 +179,9 @@ import org.slf4j.LoggerFactory; // Create the queue containing the pending read transactions readTransactions = new ConcurrentLinkedQueue>(); + + // Create the transaction manager + transactionManager = new InMemoryTransactionManager(); // Check the files and create them if missing // Create the queue containing the modifications, if it's not a in-memory btree @@ -224,6 +222,43 @@ import org.slf4j.LoggerFactory; /** + * {@inheritDoc} + */ + protected ReadTransaction beginReadTransaction() + { + BTreeHeader btreeHeader = getBtreeHeader(); + + ReadTransaction readTransaction = new ReadTransaction( btreeHeader, readTransactions ); + + readTransactions.add( readTransaction ); + + return readTransaction; + } + + + /** + * {@inheritDoc} + */ + protected ReadTransaction beginReadTransaction( long revision ) + { + BTreeHeader btreeHeader = getBtreeHeader( revision ); + + if ( btreeHeader != null ) + { + ReadTransaction readTransaction = new ReadTransaction( btreeHeader, readTransactions ); + + readTransactions.add( readTransaction ); + + return readTransaction; + } + else + { + return null; + } + } + + + /** * Close the BTree, cleaning up all the data structure */ public void close() throws IOException @@ -255,8 +290,6 @@ import org.slf4j.LoggerFactory; */ protected Tuple delete( K key, V value, long revision ) throws IOException { - beginTransaction( revision ); - if ( revision == -1L ) { revision = currentRevision.get() + 1; @@ -276,8 +309,6 @@ import org.slf4j.LoggerFactory; if ( result instanceof NotPresentResult ) { // Key not found. - commit(); - return null; } @@ -311,8 +342,6 @@ import org.slf4j.LoggerFactory; storeRevision( newBtreeHeader ); // Return the value we have found if it was modified - commit(); - if ( oldBtreeHeader.getNbUsers() == 0 ) { btreeRevisions.remove( oldBtreeHeader.getRevision() ); @@ -337,15 +366,8 @@ import org.slf4j.LoggerFactory; */ /* no qualifier */InsertResult insert( K key, V value, long revision ) throws IOException { - if ( key == null ) - { - throw new IllegalArgumentException( "Key must not be null" ); - } - // We have to start a new transaction, which will be committed or rollbacked // locally. This will duplicate the current BtreeHeader during this phase. - beginTransaction( revision ); - if ( revision == -1L ) { revision = currentRevision.get() + 1; @@ -404,8 +426,6 @@ import org.slf4j.LoggerFactory; storeRevision( newBtreeHeader ); - commit(); - if ( oldBtreeHeader.getNbUsers() == 0 ) { long oldRevision = oldBtreeHeader.getRevision(); @@ -488,45 +508,53 @@ import org.slf4j.LoggerFactory; // Create a buffer containing 200 4Kb pages (around 1Mb) ByteBuffer bb = ByteBuffer.allocateDirect( writeBufferSize ); - TupleCursor cursor = browse(); - - if ( keySerializer == null ) - { - throw new MissingSerializerException( "Cannot flush the btree without a Key serializer" ); - } - - if ( valueSerializer == null ) + try { - throw new MissingSerializerException( "Cannot flush the btree without a Value serializer" ); + TupleCursor cursor = browse(); + + if ( keySerializer == null ) + { + throw new MissingSerializerException( "Cannot flush the btree without a Key serializer" ); + } + + if ( valueSerializer == null ) + { + throw new MissingSerializerException( "Cannot flush the btree without a Value serializer" ); + } + + // Write the number of elements first + bb.putLong( getBtreeHeader().getNbElems() ); + + while ( cursor.hasNext() ) + { + Tuple tuple = cursor.next(); + + byte[] keyBuffer = keySerializer.serialize( tuple.getKey() ); + + writeBuffer( ch, bb, keyBuffer ); + + byte[] valueBuffer = valueSerializer.serialize( tuple.getValue() ); + + writeBuffer( ch, bb, valueBuffer ); + } + + // Write the buffer if needed + if ( bb.position() > 0 ) + { + bb.flip(); + ch.write( bb ); + } + + // Flush to the disk for real + ch.force( true ); + ch.close(); } - - // Write the number of elements first - bb.putLong( getBtreeHeader().getNbElems() ); - - while ( cursor.hasNext() ) + catch ( KeyNotFoundException knfe ) { - Tuple tuple = cursor.next(); - - byte[] keyBuffer = keySerializer.serialize( tuple.getKey() ); - - writeBuffer( ch, bb, keyBuffer ); - - byte[] valueBuffer = valueSerializer.serialize( tuple.getValue() ); - - writeBuffer( ch, bb, valueBuffer ); + knfe.printStackTrace(); + throw new IOException( knfe.getMessage() ); } - // Write the buffer if needed - if ( bb.position() > 0 ) - { - bb.flip(); - ch.write( bb ); - } - - // Flush to the disk for real - ch.force( true ); - ch.close(); - // Rename the current file to save a backup File backupFile = File.createTempFile( "mavibot", null, baseDirectory ); file.renameTo( backupFile ); @@ -767,63 +795,6 @@ import org.slf4j.LoggerFactory; /** - * Starts a transaction - */ - public void beginTransaction() - { - beginTransaction( getRevision() + 1 ); - } - - - /** - * Starts a transaction - */ - private void beginTransaction( long revision ) - { - transactionLock.writeLock().lock(); - - /* - if ( transactionStarted.get() ) - { - try - { - // A transaction has already been started, just wait on the write list - transactionLock.writeLock().lock(); - } - finally - { - // Done, we can release the readLock - transactionLock.readLock().unlock(); - } - } - else - { - // We have to start a transaction - // First let's release the read lock - transactionLock.readLock().unlock(); - - // Get the write lock now - transactionLock.writeLock().lock(); - - try - { - // Check the condition again - if ( !transactionStarted.get() ) - { - // Start a new transaction - transactionStarted.set( true ); - } - } - finally - { - transactionLock.readLock().unlock(); - } - } - */ - } - - - /** * Create a new B-tree header to be used for update operations * @param revision The reclaimed revision */ @@ -841,24 +812,6 @@ import org.slf4j.LoggerFactory; /** - * Commits a transaction - */ - public void commit() - { - transactionLock.writeLock().unlock(); - } - - - /** - * Rollback a transaction - */ - public void rollback() - { - transactionLock.writeLock().unlock(); - } - - - /** * @see Object#toString() */ public String toString() Modified: directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryTransactionManager.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryTransactionManager.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryTransactionManager.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryTransactionManager.java Tue Apr 29 05:16:16 2014 @@ -24,7 +24,7 @@ package org.apache.directory.mavibot.btr * * @author Apache Directory Project */ -public class InMemoryTransactionManager implements TransactionManager +public class InMemoryTransactionManager extends AbstractTransactionManager { /** * {@inheritDoc} Modified: directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryValueHolder.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryValueHolder.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryValueHolder.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/InMemoryValueHolder.java Tue Apr 29 05:16:16 2014 @@ -27,6 +27,7 @@ import java.util.UUID; import org.apache.directory.mavibot.btree.exception.BTreeOperationException; import org.apache.directory.mavibot.btree.exception.EndOfFileExceededException; +import org.apache.directory.mavibot.btree.exception.KeyNotFoundException; /** @@ -201,6 +202,10 @@ import org.apache.directory.mavibot.btre { throw new BTreeOperationException( e ); } + catch ( KeyNotFoundException knfe ) + { + throw new BTreeOperationException( knfe ); + } } return returnedValue; @@ -249,6 +254,12 @@ import org.apache.directory.mavibot.btre e.printStackTrace(); return false; } + catch ( KeyNotFoundException knfe ) + { + // TODO Auto-generated catch block + knfe.printStackTrace(); + return false; + } } else { Modified: directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedBTree.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedBTree.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedBTree.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedBTree.java Tue Apr 29 05:16:16 2014 @@ -43,16 +43,13 @@ import org.slf4j.LoggerFactory; * * @author Apache Directory Project */ -public class PersistedBTree extends AbstractBTree implements TransactionManager, Closeable +public class PersistedBTree extends AbstractBTree implements Closeable { /** The LoggerFactory used by this class */ protected static final Logger LOG = LoggerFactory.getLogger( PersistedBTree.class ); protected static final Logger LOG_PAGES = LoggerFactory.getLogger( "LOG_PAGES" ); - /** The RecordManager if the B-tree is managed */ - private RecordManager recordManager; - /** The cache associated with this B-tree */ protected Cache cache; @@ -74,6 +71,9 @@ public class PersistedBTree extend /** The BtreeInfo offset */ private long btreeInfoOffset; + + /** The internal recordManager */ + private RecordManager recordManager; /** * Creates a new BTree, with no initialization. @@ -261,6 +261,8 @@ public class PersistedBTree extend */ /* No qualifier */void setRecordManager( RecordManager recordManager ) { + // The RecordManager is also the TransactionManager + transactionManager = recordManager; this.recordManager = recordManager; } @@ -286,8 +288,6 @@ public class PersistedBTree extend revision = currentRevision.get() + 1; } - recordManager.beginTransaction(); - try { // Try to delete the entry starting from the root page. Here, the root @@ -299,7 +299,6 @@ public class PersistedBTree extend { // We haven't found the element in the B-tree, just get out // without updating the recordManager - rollback(); return null; } @@ -311,7 +310,6 @@ public class PersistedBTree extend // If the B-tree is managed, we have to update the rootPage on disk // Update the RecordManager header - commit(); // Return the value we have found if it was modified return tuple; @@ -319,8 +317,6 @@ public class PersistedBTree extend catch ( IOException ioe ) { // if we've got an error, we have to rollback - rollback(); - throw ioe; } } @@ -442,11 +438,6 @@ public class PersistedBTree extend */ /* no qualifier */InsertResult insert( K key, V value, long revision ) throws IOException { - if ( key == null ) - { - throw new IllegalArgumentException( "Key must not be null" ); - } - // We have to start a new transaction, which will be committed or rollbacked // locally. This will duplicate the current BtreeHeader during this phase. if ( revision == -1L ) @@ -454,8 +445,6 @@ public class PersistedBTree extend revision = currentRevision.get() + 1; } - recordManager.beginTransaction(); - try { // Try to insert the new value in the tree at the right place, @@ -463,17 +452,11 @@ public class PersistedBTree extend // a Node or a Leaf InsertResult result = processInsert( key, value, revision ); - // Done ! we can commit now - commit(); - // Return the value we have found if it was modified return result; } catch ( IOException ioe ) { - // if we've got an error, we have to rollback - rollback(); - throw ioe; } } @@ -713,39 +696,41 @@ public class PersistedBTree extend /** - * Starts a transaction + * {@inheritDoc} */ - public void beginTransaction() + protected ReadTransaction beginReadTransaction() { - beginTransaction( getRevision() + 1 ); - } + BTreeHeader btreeHeader = getBtreeHeader(); + ReadTransaction readTransaction = new ReadTransaction( recordManager, btreeHeader, readTransactions ); - /** - * Starts a transaction - */ - private void beginTransaction( long revision ) - { - } - + readTransactions.add( readTransaction ); + return readTransaction; + } + + /** - * Commits a transaction + * {@inheritDoc} */ - public void commit() + protected ReadTransaction beginReadTransaction( long revision ) { - recordManager.commit(); - } + BTreeHeader btreeHeader = getBtreeHeader( revision ); + if ( btreeHeader != null ) + { + ReadTransaction readTransaction = new ReadTransaction( recordManager, btreeHeader, readTransactions ); - /** - * Rollback a transaction - */ - public void rollback() - { - recordManager.rollback(); - } + readTransactions.add( readTransaction ); + return readTransaction; + } + else + { + return null; + } + } + /** * @see Object#toString() Modified: directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedValueHolder.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedValueHolder.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedValueHolder.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/PersistedValueHolder.java Tue Apr 29 05:16:16 2014 @@ -28,6 +28,7 @@ import java.util.UUID; import org.apache.directory.mavibot.btree.exception.BTreeAlreadyCreatedException; import org.apache.directory.mavibot.btree.exception.BTreeAlreadyManagedException; import org.apache.directory.mavibot.btree.exception.BTreeCreationException; +import org.apache.directory.mavibot.btree.exception.KeyNotFoundException; import org.apache.directory.mavibot.btree.serializer.IntSerializer; import org.apache.directory.mavibot.btree.serializer.LongSerializer; @@ -419,6 +420,12 @@ import org.apache.directory.mavibot.btre e.printStackTrace(); return null; } + catch ( KeyNotFoundException knfe ) + { + // TODO Auto-generated catch block + knfe.printStackTrace(); + return null; + } } else { Modified: directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/ReadTransaction.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/ReadTransaction.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/ReadTransaction.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/ReadTransaction.java Tue Apr 29 05:16:16 2014 @@ -21,6 +21,7 @@ package org.apache.directory.mavibot.btr import java.util.Date; +import java.util.concurrent.ConcurrentLinkedQueue; /** @@ -49,49 +50,63 @@ public class ReadTransaction /** The date of creation */ private long creationDate; - /** The revision on which we are having a transaction */ - private volatile Page rootPage; - /** The associated B-tree header */ private BTreeHeader btreeHeader; /** A flag used to tell if a transaction is closed or not */ private volatile boolean closed; - - + + /** The list of read transactions being executed */ + private ConcurrentLinkedQueue> readTransactions; + + /** The reference to the recordManager, if any */ + private RecordManager recordManager; + /** * Creates a new transaction instance * * @param btreeHeader The BtreeHeader we will use for this read transaction */ - public ReadTransaction( BTreeHeader btreeHeader ) + public ReadTransaction( RecordManager recordManager, BTreeHeader btreeHeader, ConcurrentLinkedQueue> readTransactions ) { if ( btreeHeader != null ) { this.revision = btreeHeader.getRevision(); this.creationDate = System.currentTimeMillis(); - this.rootPage = btreeHeader.getRootPage(); this.btreeHeader = btreeHeader; + this.recordManager = recordManager; closed = false; } + + this.readTransactions = readTransactions; } - - + + /** - * @return the associated revision + * Creates a new transaction instance + * + * @param btreeHeader The BtreeHeader we will use for this read transaction */ - public long getRevision() + public ReadTransaction( BTreeHeader btreeHeader, ConcurrentLinkedQueue> readTransactions ) { - return revision; + if ( btreeHeader != null ) + { + this.revision = btreeHeader.getRevision(); + this.creationDate = System.currentTimeMillis(); + this.btreeHeader = btreeHeader; + closed = false; + } + + this.readTransactions = readTransactions; } /** - * @return the associated rootPage + * @return the associated revision */ - public Page getRootPage() + public long getRevision() { - return rootPage; + return revision; } @@ -118,8 +133,18 @@ public class ReadTransaction */ public void close() { - rootPage = null; closed = true; + + // Remove the transaction from the list of opened transactions + readTransactions.remove( this ); + + // and push the + if ( recordManager != null ) + { + recordManager.releaseTransaction( this ); + } + + // Now, get back the copied pages } Modified: directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/RecordManager.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/RecordManager.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/RecordManager.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/RecordManager.java Tue Apr 29 05:16:16 2014 @@ -31,14 +31,13 @@ import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Queue; import java.util.Set; -import java.util.UUID; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; -import net.sf.ehcache.Cache; - import org.apache.directory.mavibot.btree.exception.BTreeAlreadyManagedException; import org.apache.directory.mavibot.btree.exception.BTreeCreationException; import org.apache.directory.mavibot.btree.exception.EndOfFileExceededException; @@ -66,7 +65,7 @@ import org.slf4j.LoggerFactory; * * @author Apache Directory Project */ -public class RecordManager +public class RecordManager extends AbstractTransactionManager { /** The LoggerFactory used by this class */ protected static final Logger LOG = LoggerFactory.getLogger( RecordManager.class ); @@ -147,6 +146,9 @@ public class RecordManager /** The set of managed B-trees */ private Map> managedBtrees; + + /** The queue of recently closed transactions */ + private Queue closedTransactionsQueue = new LinkedBlockingQueue(); /** The default file name */ private static final String DEFAULT_FILE_NAME = "mavibot.db"; @@ -183,9 +185,9 @@ public class RecordManager /** A lock to protect the transaction handling */ private Lock transactionLock = new ReentrantLock(); - - /** A counter used to remember how many times a transaction has been started */ - private int lockHeldCounter = 0; + + /** A ThreadLocalStorage used to store the current transaction */ + private static final ThreadLocal context = new ThreadLocal(); /** The list of PageIO that can be freed after a commit */ List freedPages = new ArrayList(); @@ -408,7 +410,7 @@ public class RecordManager * @throws IllegalArgumentException */ private void loadRecordManager() throws IOException, ClassNotFoundException, IllegalAccessException, - InstantiationException, IllegalArgumentException, SecurityException, NoSuchFieldException + InstantiationException, IllegalArgumentException, SecurityException, NoSuchFieldException, KeyNotFoundException { if ( fileChannel.size() != 0 ) { @@ -535,11 +537,23 @@ public class RecordManager /** * Starts a transaction */ - /*No Qualifier*/void beginTransaction() + public void beginTransaction() { + // First, take the lock transactionLock.lock(); - - lockHeldCounter++; + + // Now, check the TLS state + Integer nbTxnLevel = context.get(); + + if ( nbTxnLevel == null ) + { + context.set( 1 ); + } + else + { + // And increment the counter of inner txn. + context.set( nbTxnLevel + 1 ); + } } @@ -548,21 +562,26 @@ public class RecordManager */ public void commit() { - lockHeldCounter--; - - if ( !fileChannel.isOpen() ) + int nbTxnStarted = context.get(); + + if ( nbTxnStarted == 0 ) { - // The file has been closed, nothing remains to commit, let's get out + // The transaction was rollbacked, quit immediatelly transactionLock.unlock(); - return; } - - if ( lockHeldCounter == 0 ) + else { + if ( !fileChannel.isOpen() ) + { + // The file has been closed, nothing remains to commit, let's get out + transactionLock.unlock(); + return; + } + // We are done with the transaction // First update the RMHeader to be sure that we have a way to restore from a crash updateRecordManagerHeader(); - + // We can now free pages for ( PageIO pageIo : freedPages ) { @@ -575,16 +594,20 @@ public class RecordManager throw new RecordManagerException( ioe.getMessage() ); } } - + // Release the allocated and freed pages list freedPages.clear(); allocatedPages.clear(); - + // And update the RMHeader again, removing the old references to BOB and CPB b-tree headers // here, we have to erase the old references to keep only the new ones. updateRecordManagerHeader(); + + // And decrement the number of started transactions + context.set( nbTxnStarted - 1 ); } + // Finally, release the global lock transactionLock.unlock(); } @@ -594,30 +617,28 @@ public class RecordManager */ public void rollback() { - lockHeldCounter--; + // Reset the counter + context.set( 0 ); - if ( lockHeldCounter == 0 ) + // We can now free allocated pages, this is the end of the transaction + for ( PageIO pageIo : allocatedPages ) { - // We can now free allocated pages, this is the end of the transaction - for ( PageIO pageIo : allocatedPages ) + try { - try - { - free( pageIo ); - } - catch ( IOException ioe ) - { - throw new RecordManagerException( ioe.getMessage() ); - } + free( pageIo ); } + catch ( IOException ioe ) + { + throw new RecordManagerException( ioe.getMessage() ); + } + } - // Release the allocated and freed pages list - freedPages.clear(); - allocatedPages.clear(); + // Release the allocated and freed pages list + freedPages.clear(); + allocatedPages.clear(); - // And update the RMHeader - updateRecordManagerHeader(); - } + // And update the RMHeader + updateRecordManagerHeader(); transactionLock.unlock(); } @@ -1835,7 +1856,7 @@ public class RecordManager btreeOfBtrees.insert( nameRevision, btreeHeaderOffset ); // Update the B-tree of B-trees - currentBtreeOfBtreesOffset = ((AbstractBTree)btreeOfBtrees).getBtreeHeader().getBTreeHeaderOffset(); + currentBtreeOfBtreesOffset = ((AbstractBTree)btreeOfBtrees).getBtreeHeader().getBTreeHeaderOffset(); } @@ -1861,8 +1882,8 @@ public class RecordManager copiedPageBtree.insert( revisionName, pageOffsets ); - // Update the B-tree of B-trees - currentCopiedPagesBtreeOffset = ((AbstractBTree)copiedPageBtree).getBtreeHeader().getBTreeHeaderOffset(); + // Update the CopiedPageBtree offset + currentCopiedPagesBtreeOffset = ((AbstractBTree)copiedPageBtree).getBtreeHeader().getBTreeHeaderOffset(); } @@ -2221,7 +2242,6 @@ public class RecordManager for ( PageIO pageIo : pageIos ) { - ByteBuffer data = pageIo.getData(); pageIo.getData().rewind(); if ( fileChannel.size() < ( pageIo.getOffset() + pageSize ) ) @@ -3515,6 +3535,17 @@ public class RecordManager return btree; } + + /** + * Add a newly closd transaction into the closed transaction queue + */ + /* no qualifier */ void releaseTransaction( ReadTransaction readTransaction ) + { + RevisionName revisionName = new RevisionName( + readTransaction.getRevision(), + readTransaction.getBtreeHeader().getBtree().getName() ); + closedTransactionsQueue.add( revisionName ); + } private void setCheckedPage( long[] checkedPages, long offset, int pageSize ) { Modified: directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/ValueBTreeCursor.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/ValueBTreeCursor.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/ValueBTreeCursor.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/main/java/org/apache/directory/mavibot/btree/ValueBTreeCursor.java Tue Apr 29 05:16:16 2014 @@ -4,6 +4,7 @@ package org.apache.directory.mavibot.btr import java.io.IOException; import org.apache.directory.mavibot.btree.exception.EndOfFileExceededException; +import org.apache.directory.mavibot.btree.exception.KeyNotFoundException; import org.apache.directory.mavibot.btree.BTree; @@ -39,6 +40,11 @@ import org.apache.directory.mavibot.btre // TODO Auto-generated catch block e.printStackTrace(); } + catch ( KeyNotFoundException knfe ) + { + // TODO Auto-generated catch block + knfe.printStackTrace(); + } } Modified: directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/InMemoryBTreeBuilderTest.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/InMemoryBTreeBuilderTest.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/InMemoryBTreeBuilderTest.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/InMemoryBTreeBuilderTest.java Tue Apr 29 05:16:16 2014 @@ -27,6 +27,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import org.apache.directory.mavibot.btree.exception.KeyNotFoundException; import org.apache.directory.mavibot.btree.serializer.IntSerializer; import org.junit.Test; @@ -39,7 +40,7 @@ import org.junit.Test; public class InMemoryBTreeBuilderTest { @Test - public void testIntegerTree() throws IOException + public void testIntegerTree() throws IOException, KeyNotFoundException { List> sortedTuple = new ArrayList>(); Modified: directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/InMemoryBTreeDuplicateKeyTest.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/InMemoryBTreeDuplicateKeyTest.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/InMemoryBTreeDuplicateKeyTest.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/InMemoryBTreeDuplicateKeyTest.java Tue Apr 29 05:16:16 2014 @@ -33,6 +33,7 @@ import java.util.UUID; import org.apache.directory.mavibot.btree.exception.BTreeAlreadyManagedException; import org.apache.directory.mavibot.btree.exception.DuplicateValueNotAllowedException; +import org.apache.directory.mavibot.btree.exception.KeyNotFoundException; import org.apache.directory.mavibot.btree.serializer.IntSerializer; import org.apache.directory.mavibot.btree.serializer.LongSerializer; import org.apache.directory.mavibot.btree.serializer.StringSerializer; @@ -47,7 +48,7 @@ import org.junit.Test; public class InMemoryBTreeDuplicateKeyTest { @Test - public void testInsertNullValue() throws IOException + public void testInsertNullValue() throws IOException, KeyNotFoundException { IntSerializer serializer = IntSerializer.INSTANCE; @@ -69,7 +70,7 @@ public class InMemoryBTreeDuplicateKeyTe @Test - public void testBrowseEmptyTree() throws IOException + public void testBrowseEmptyTree() throws IOException, KeyNotFoundException { IntSerializer serializer = IntSerializer.INSTANCE; @@ -105,7 +106,7 @@ public class InMemoryBTreeDuplicateKeyTe @Test - public void testDuplicateKey() throws IOException + public void testDuplicateKey() throws IOException, KeyNotFoundException { IntSerializer serializer = IntSerializer.INSTANCE; Modified: directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/InMemoryBTreeTest.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/InMemoryBTreeTest.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/InMemoryBTreeTest.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/InMemoryBTreeTest.java Tue Apr 29 05:16:16 2014 @@ -911,7 +911,7 @@ public class InMemoryBTreeTest * Test the exist() method */ @Test - public void testExist() throws IOException + public void testExist() throws IOException, KeyNotFoundException { // Create a BTree with pages containing 4 elements BTree btree = createTwoLevelBTreeFullLeaves(); @@ -1193,7 +1193,7 @@ public class InMemoryBTreeTest * @param element The removed element * @param expected The expected set of elements */ - private void checkRemoval( BTree btree, int element, Set expected ) throws IOException + private void checkRemoval( BTree btree, int element, Set expected ) throws IOException, KeyNotFoundException { Tuple removed = btree.delete( element ); assertEquals( element, removed.getKey().intValue() ); @@ -1213,7 +1213,7 @@ public class InMemoryBTreeTest * @param btree The tree to check * @param expected The set with the expected elements */ - private void checkTree( BTree btree, Set expected ) + private void checkTree( BTree btree, Set expected ) throws KeyNotFoundException { try { @@ -1723,7 +1723,7 @@ public class InMemoryBTreeTest * Test the addition of elements with null values */ @Test - public void testAdditionNullValues() throws IOException + public void testAdditionNullValues() throws IOException, KeyNotFoundException { BTree btree = createMultiLevelBTreeLeavesHalfFull(); Modified: directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/MultiThreadedInMemoryBtreeTest.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/MultiThreadedInMemoryBtreeTest.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/MultiThreadedInMemoryBtreeTest.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/MultiThreadedInMemoryBtreeTest.java Tue Apr 29 05:16:16 2014 @@ -33,6 +33,7 @@ import org.apache.directory.mavibot.btre import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; +import org.junit.Ignore; /** @@ -110,7 +111,7 @@ public class MultiThreadedInMemoryBtreeT * @return The number of read elements * @throws IOException If the browse failed */ - private int testBrowse() throws IOException + private int testBrowse() throws IOException, KeyNotFoundException { TupleCursor cursor = btree.browse(); @@ -209,6 +210,7 @@ public class MultiThreadedInMemoryBtreeT * Test that we can use many threads inserting data in a BTree * @throws InterruptedException */ + @Ignore @Test public void testInsertMultiThreads() throws InterruptedException, IOException { Modified: directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/PersistedBTreeBrowseTest.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/PersistedBTreeBrowseTest.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/PersistedBTreeBrowseTest.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/PersistedBTreeBrowseTest.java Tue Apr 29 05:16:16 2014 @@ -34,6 +34,7 @@ import java.util.UUID; import org.apache.commons.io.FileUtils; import org.apache.directory.mavibot.btree.exception.BTreeAlreadyManagedException; import org.apache.directory.mavibot.btree.exception.EndOfFileExceededException; +import org.apache.directory.mavibot.btree.exception.KeyNotFoundException; import org.apache.directory.mavibot.btree.serializer.LongSerializer; import org.apache.directory.mavibot.btree.serializer.StringSerializer; import org.junit.After; @@ -194,9 +195,10 @@ public class PersistedBTreeBrowseTest //---------------------------------------------------------------------------------------- /** * Test the browse methods on an empty btree + * @throws KeyNotFoundException */ @Test - public void testBrowseEmptyBTree() throws IOException, BTreeAlreadyManagedException + public void testBrowseEmptyBTree() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { TupleCursor cursor = btree.browse(); @@ -231,7 +233,7 @@ public class PersistedBTreeBrowseTest * Test the browse methods on a btree containing just a leaf */ @Test - public void testBrowseBTreeLeafNext() throws IOException, BTreeAlreadyManagedException + public void testBrowseBTreeLeafNext() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some data btree.insert( 1L, "1" ); @@ -261,7 +263,7 @@ public class PersistedBTreeBrowseTest * Test the browse methods on a btree containing just a leaf */ @Test - public void testBrowseBTreeLeafPrev() throws IOException, BTreeAlreadyManagedException + public void testBrowseBTreeLeafPrev() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some data btree.insert( 1L, "1" ); @@ -289,7 +291,7 @@ public class PersistedBTreeBrowseTest * move at the end or at the beginning */ @Test - public void testBrowseBTreeLeafFirstLast() throws IOException, BTreeAlreadyManagedException + public void testBrowseBTreeLeafFirstLast() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some data btree.insert( 1L, "1" ); @@ -371,7 +373,7 @@ public class PersistedBTreeBrowseTest * move back and forth */ @Test - public void testBrowseBTreeLeafNextPrev() throws IOException, BTreeAlreadyManagedException + public void testBrowseBTreeLeafNextPrev() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some data btree.insert( 1L, "1" ); @@ -423,7 +425,7 @@ public class PersistedBTreeBrowseTest * Test the browse methods on a btree containing many nodes */ @Test - public void testBrowseBTreeNodesNext() throws IOException, BTreeAlreadyManagedException + public void testBrowseBTreeNodesNext() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some data for ( long i = 1; i < 1000L; i++ ) @@ -455,7 +457,7 @@ public class PersistedBTreeBrowseTest * Test the browse methods on a btree containing many nodes */ @Test - public void testBrowseBTreeNodesPrev() throws IOException, BTreeAlreadyManagedException + public void testBrowseBTreeNodesPrev() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some data for ( long i = 1; i < 1000L; i++ ) @@ -487,7 +489,7 @@ public class PersistedBTreeBrowseTest * Test the browse methods on a btree containing just a leaf with duplicate values */ @Test - public void testBrowseBTreeLeafNextDups1() throws IOException, BTreeAlreadyManagedException + public void testBrowseBTreeLeafNextDups1() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some duplicate data btree.insert( 1L, "1" ); @@ -517,7 +519,7 @@ public class PersistedBTreeBrowseTest * Test the browse methods on a btree containing just a leaf with duplicate values */ @Test - public void testBrowseBTreeLeafNextDupsN() throws IOException, BTreeAlreadyManagedException + public void testBrowseBTreeLeafNextDupsN() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some duplicate data btree.insert( 1L, "1" ); @@ -551,7 +553,7 @@ public class PersistedBTreeBrowseTest * Test the browse methods on a btree containing just a leaf with duplicate values */ @Test - public void testBrowseBTreeLeafPrevDups1() throws IOException, BTreeAlreadyManagedException + public void testBrowseBTreeLeafPrevDups1() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some duplicate data btree.insert( 1L, "1" ); @@ -581,7 +583,7 @@ public class PersistedBTreeBrowseTest * Test the browse methods on a btree containing just a leaf with duplicate values */ @Test - public void testBrowseBTreeLeafPrevDupsN() throws IOException, BTreeAlreadyManagedException + public void testBrowseBTreeLeafPrevDupsN() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some duplicate data btree.insert( 1L, "1" ); @@ -615,7 +617,7 @@ public class PersistedBTreeBrowseTest * Test the browse methods on a btree containing nodes with duplicate values */ @Test - public void testBrowseBTreeNodesNextDupsN() throws IOException, BTreeAlreadyManagedException + public void testBrowseBTreeNodesNextDupsN() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some data for ( long i = 1; i < 1000L; i++ ) @@ -661,7 +663,7 @@ public class PersistedBTreeBrowseTest * Test the browse methods on a btree containing nodes with duplicate values */ @Test - public void testBrowseBTreeNodesPrevDupsN() throws IOException, BTreeAlreadyManagedException + public void testBrowseBTreeNodesPrevDupsN() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some data for ( long i = 1; i < 1000L; i++ ) @@ -708,7 +710,7 @@ public class PersistedBTreeBrowseTest * stored into a sub btree */ @Test - public void testBrowseBTreeLeafNextDupsSubBTree1() throws IOException, BTreeAlreadyManagedException + public void testBrowseBTreeLeafNextDupsSubBTree1() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some duplicate data which will be stored into a sub btree for ( long i = 1L; i < 32L; i++ ) @@ -740,7 +742,7 @@ public class PersistedBTreeBrowseTest * Test the browse methods on a btree containing just a leaf with duplicate values */ @Test - public void testBrowseBTreeLeafPrevDupsSubBTree1() throws IOException, BTreeAlreadyManagedException + public void testBrowseBTreeLeafPrevDupsSubBTree1() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some duplicate data which will be stored into a sub btree for ( long i = 1L; i < 32L; i++ ) @@ -937,7 +939,7 @@ public class PersistedBTreeBrowseTest * with duplicate values. */ @Test - public void testNextKey() throws IOException, BTreeAlreadyManagedException + public void testNextKey() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some data for ( long i = 1; i < 1000L; i++ ) @@ -987,7 +989,7 @@ public class PersistedBTreeBrowseTest */ @Test @Ignore - public void testNextKeyDups() throws IOException, BTreeAlreadyManagedException + public void testNextKeyDups() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some data //for ( long i = 1; i < 3; i++ ) @@ -1036,7 +1038,7 @@ public class PersistedBTreeBrowseTest * with duplicate values. */ @Test - public void testPrevKey() throws IOException, BTreeAlreadyManagedException + public void testPrevKey() throws IOException, BTreeAlreadyManagedException, KeyNotFoundException { // Inject some data for ( long i = 1; i < 1000L; i++ ) Modified: directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/PersistedBTreeDuplicateKeyTest.java URL: http://svn.apache.org/viewvc/directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/PersistedBTreeDuplicateKeyTest.java?rev=1590872&r1=1590871&r2=1590872&view=diff ============================================================================== --- directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/PersistedBTreeDuplicateKeyTest.java (original) +++ directory/mavibot/branches/with-txns/mavibot/src/test/java/org/apache/directory/mavibot/btree/PersistedBTreeDuplicateKeyTest.java Tue Apr 29 05:16:16 2014 @@ -35,6 +35,7 @@ import java.util.UUID; import org.apache.commons.io.FileUtils; import org.apache.directory.mavibot.btree.exception.BTreeAlreadyManagedException; import org.apache.directory.mavibot.btree.exception.DuplicateValueNotAllowedException; +import org.apache.directory.mavibot.btree.exception.KeyNotFoundException; import org.apache.directory.mavibot.btree.serializer.IntSerializer; import org.apache.directory.mavibot.btree.serializer.LongSerializer; import org.apache.directory.mavibot.btree.serializer.StringSerializer; @@ -122,7 +123,7 @@ public class PersistedBTreeDuplicateKeyT @Test - public void testInsertNullValue() throws IOException + public void testInsertNullValue() throws IOException, KeyNotFoundException { btree.insert( 1L, null ); @@ -141,7 +142,7 @@ public class PersistedBTreeDuplicateKeyT @Test - public void testBrowseEmptyTree() throws IOException + public void testBrowseEmptyTree() throws IOException, KeyNotFoundException { IntSerializer serializer = IntSerializer.INSTANCE; @@ -177,7 +178,7 @@ public class PersistedBTreeDuplicateKeyT @Test - public void testDuplicateKey() throws IOException + public void testDuplicateKey() throws IOException, KeyNotFoundException { btree.insert( 1L, "1" ); btree.insert( 1L, "2" );