From fb05a04c84aaf054c3863dca893c6fe5cbc86621 Mon Sep 17 00:00:00 2001
From: boli <boli@localhost>
Date: Wed, 04 Jun 2008 17:24:01 +0000
Subject: [PATCH] Fix for issues 3255, 3265, and 3269:

---
 opendj-sdk/opends/src/server/org/opends/server/backends/jeb/EntryContainer.java | 1802 +++++++++++++++++++--------------------------------------
 1 files changed, 594 insertions(+), 1,208 deletions(-)

diff --git a/opendj-sdk/opends/src/server/org/opends/server/backends/jeb/EntryContainer.java b/opendj-sdk/opends/src/server/org/opends/server/backends/jeb/EntryContainer.java
index ca6dc3e..ebba91b 100644
--- a/opendj-sdk/opends/src/server/org/opends/server/backends/jeb/EntryContainer.java
+++ b/opendj-sdk/opends/src/server/org/opends/server/backends/jeb/EntryContainer.java
@@ -191,14 +191,6 @@
    */
   private HashMap<String, VLVIndex> vlvIndexMap;
 
-  /**
-   * Cached value from config so they don't have to be retrieved per operation.
-   */
-
-  private int subtreeDeleteSizeLimit;
-
-  private int subtreeDeleteBatchSize;
-
   private String databasePrefix;
   /**
    * This class is responsible for managing the configuraiton for attribute
@@ -309,11 +301,9 @@
     public boolean isConfigurationAddAcceptable(
         LocalDBVLVIndexCfg cfg, List<Message> unacceptableReasons)
     {
-      SearchFilter filter;
       try
       {
-        filter =
-            SearchFilter.createFilterFromString(cfg.getFilter());
+        SearchFilter.createFilterFromString(cfg.getFilter());
       }
       catch(Exception e)
       {
@@ -500,9 +490,6 @@
     }
     this.databasePrefix = builder.toString();
 
-    this.subtreeDeleteSizeLimit = config.getSubtreeDeleteSizeLimit();
-    this.subtreeDeleteBatchSize = config.getSubtreeDeleteBatchSize();
-
     // Instantiate the attribute indexes.
     attrIndexMap = new HashMap<AttributeType, AttributeIndex>();
 
@@ -838,10 +825,10 @@
    *          If a problem occurs while processing the
    *          search.
    * @throws DatabaseException If an error occurs in the JE database.
-   * @throws JebException If an error occurs in the JE database.
+   * @throws CanceledOperationException if this operation should be cancelled.
    */
   public void search(SearchOperation searchOperation)
-       throws DirectoryException, DatabaseException, JebException
+       throws DirectoryException, DatabaseException, CanceledOperationException
   {
     DN baseDN = searchOperation.getBaseDN();
     SearchScope searchScope = searchOperation.getScope();
@@ -1213,7 +1200,7 @@
    */
   private void searchNotIndexed(SearchOperation searchOperation,
                                 PagedResultsControl pageRequest)
-       throws DirectoryException
+       throws DirectoryException, CanceledOperationException
   {
     EntryCache<?> entryCache = DirectoryServer.getEntryCache();
     DN baseDN = searchOperation.getBaseDN();
@@ -1385,8 +1372,8 @@
 
           if (isInScope)
           {
-            Entry entry = null;
-            Entry cacheEntry = null;
+            Entry entry;
+            Entry cacheEntry;
 
             // Try the entry cache first. Note no need to take a lock.
             lockList.clear();
@@ -1395,12 +1382,7 @@
 
             if (cacheEntry == null)
             {
-              GetEntryByIDOperation operation =
-                   new GetEntryByIDOperation(entryID);
-
-              // Fetch the candidate entry from the database.
-              this.invokeTransactedOperation(operation);
-              entry = operation.getEntry();
+              entry = id2entry.get(null, entryID, LockMode.DEFAULT);
             }
             else
             {
@@ -1443,6 +1425,8 @@
             }
           }
 
+          searchOperation.checkIfCanceled(false);
+
           // Move to the next record.
           status = cursor.getNext(key, data, LockMode.DEFAULT);
         }
@@ -1459,13 +1443,6 @@
         TRACER.debugCaught(DebugLogLevel.ERROR, e);
       }
     }
-    catch (JebException e)
-    {
-      if (debugEnabled())
-      {
-        TRACER.debugCaught(DebugLogLevel.ERROR, e);
-      }
-    }
 
     if (pageRequest != null)
     {
@@ -1504,7 +1481,7 @@
                              boolean candidatesAreInScope,
                              SearchOperation searchOperation,
                              PagedResultsControl pageRequest)
-       throws DirectoryException
+       throws DirectoryException, CanceledOperationException
   {
     EntryCache<?> entryCache = DirectoryServer.getEntryCache();
     SearchScope searchScope = searchOperation.getScope();
@@ -1562,8 +1539,8 @@
       while (iterator.hasNext())
       {
         EntryID id = iterator.next();
-        Entry entry = null;
-        Entry cacheEntry = null;
+        Entry entry;
+        Entry cacheEntry;
 
         // Try the entry cache first. Note no need to take a lock.
         lockList.clear();
@@ -1576,13 +1553,10 @@
         {
           if (cacheEntry == null)
           {
-            GetEntryByIDOperation operation = new GetEntryByIDOperation(id);
-
             // Fetch the candidate entry from the database.
             try
             {
-              this.invokeTransactedOperation(operation);
-              entry = operation.getEntry();
+              entry = id2entry.get(null, id, LockMode.DEFAULT);
             }
             catch (Exception e)
             {
@@ -1687,6 +1661,7 @@
             lock.unlock();
           }
         }
+        searchOperation.checkIfCanceled(false);
       }
     }
 
@@ -1752,171 +1727,15 @@
    * @throws DirectoryException If a problem occurs while trying to add the
    *                            entry.
    * @throws DatabaseException If an error occurs in the JE database.
-   * @throws JebException If an error occurs in the JE backend.
+   * @throws CanceledOperationException if this operation should be cancelled.
    */
   public void addEntry(Entry entry, AddOperation addOperation)
-      throws DatabaseException, DirectoryException, JebException
+      throws DatabaseException, DirectoryException, CanceledOperationException
   {
-    TransactedOperation operation =
-        new AddEntryTransaction(entry);
+    Transaction txn = beginTransaction();
+    DN parentDN = getParentWithinBase(entry.getDN());
 
-    invokeTransactedOperation(operation);
-  }
-
-  /**
-   * This method is common to all operations invoked under a database
-   * transaction. It retries the operation if the transaction is
-   * aborted due to a deadlock condition, up to a configured maximum
-   * number of retries.
-   *
-   * @param operation An object implementing the TransactedOperation interface.
-   * @throws DatabaseException If an error occurs in the JE database.
-   * @throws DirectoryException If a Directory Server error occurs.
-   * @throws JebException If an error occurs in the JE backend.
-   */
-  private void invokeTransactedOperation(TransactedOperation operation)
-      throws DatabaseException, DirectoryException, JebException
-  {
-    // Attempt the operation under a transaction until it fails or completes.
-    boolean completed = false;
-    while (!completed)
-    {
-      // Start a transaction.
-      Transaction txn = operation.beginOperationTransaction();
-
-      try
-      {
-        // Invoke the operation.
-        operation.invokeOperation(txn);
-
-        // Commit the transaction.
-        EntryContainer.transactionCommit(txn);
-        completed = true;
-      }
-      catch (DatabaseException databaseException)
-      {
-        EntryContainer.transactionAbort(txn);
-        throw databaseException;
-      }
-      catch (DirectoryException directoryException)
-      {
-        EntryContainer.transactionAbort(txn);
-        throw directoryException;
-      }
-      catch (JebException jebException)
-      {
-        EntryContainer.transactionAbort(txn);
-        throw jebException;
-      }
-      catch (Exception e)
-      {
-        EntryContainer.transactionAbort(txn);
-
-        String msg = e.getMessage();
-        if (msg == null)
-        {
-          msg = stackTraceToSingleLineString(e);
-        }
-        Message message = ERR_JEB_UNCHECKED_EXCEPTION.get(msg);
-        throw new JebException(message, e);
-      }
-    }
-
-    // Do any actions necessary after successful commit,
-    // usually to update the entry cache.
-    operation.postCommitAction();
-  }
-
-  /**
-   * This interface represents any kind of operation on the database
-   * that must be performed under a transaction. A class which implements
-   * this interface does not need to be concerned with creating the
-   * transaction nor retrying the transaction after deadlock.
-   */
-  private interface TransactedOperation
-  {
-    /**
-     * Begin a transaction for this operation.
-     *
-     * @return The transaction for the operation, or null if the operation
-     *         will not use a transaction.
-     * @throws DatabaseException If an error occurs in the JE database.
-     */
-    public abstract Transaction beginOperationTransaction()
-        throws DatabaseException;
-
-    /**
-     * Invoke the operation under the given transaction.
-     *
-     * @param txn The transaction to be used to perform the operation.
-     * @throws DatabaseException If an error occurs in the JE database.
-     * @throws DirectoryException If a Directory Server error occurs.
-     * @throws JebException If an error occurs in the JE backend.
-     */
-    public abstract void invokeOperation(Transaction txn)
-        throws DatabaseException, DirectoryException, JebException;
-
-    /**
-     * This method is called after the transaction has successfully
-     * committed.
-     */
-    public abstract void postCommitAction();
-  }
-
-  /**
-   * This inner class implements the Add Entry operation through
-   * the TransactedOperation interface.
-   */
-  private class AddEntryTransaction implements TransactedOperation
-  {
-    /**
-     * The entry to be added.
-     */
-    private Entry entry;
-
-    /**
-     * The DN of the superior entry of the entry to be added.  This can be
-     * null if the entry to be added is a base entry.
-     */
-    DN parentDN;
-
-    /**
-     * The ID of the entry once it has been assigned.
-     */
-    EntryID entryID = null;
-
-    /**
-     * Begin a transaction for this operation.
-     *
-     * @return The transaction for the operation, or null if the operation
-     *         will not use a transaction.
-     * @throws DatabaseException If an error occurs in the JE database.
-     */
-    public Transaction beginOperationTransaction() throws DatabaseException
-    {
-      return beginTransaction();
-    }
-
-    /**
-     * Create a new Add Entry Transaction.
-     * @param entry The entry to be added.
-     */
-    public AddEntryTransaction(Entry entry)
-    {
-      this.entry = entry;
-      this.parentDN = getParentWithinBase(entry.getDN());
-    }
-
-    /**
-     * Invoke the operation under the given transaction.
-     *
-     * @param txn The transaction to be used to perform the operation.
-     * @throws DatabaseException If an error occurs in the JE database.
-     * @throws DirectoryException If a Directory Server error occurs.
-     * @throws JebException If an error occurs in the JE backend.
-     */
-    public void invokeOperation(Transaction txn)
-        throws DatabaseException, DirectoryException, JebException
+    try
     {
       // Check whether the entry already exists.
       if (dn2id.get(txn, entry.getDN(), LockMode.DEFAULT) != null)
@@ -1924,7 +1743,7 @@
         Message message =
             ERR_JEB_ADD_ENTRY_ALREADY_EXISTS.get(entry.getDN().toString());
         throw new DirectoryException(ResultCode.ENTRY_ALREADY_EXISTS,
-                                     message);
+            message);
       }
 
       // Check that the parent entry exists.
@@ -1939,18 +1758,14 @@
         if (parentID == null)
         {
           Message message = ERR_JEB_ADD_NO_SUCH_OBJECT.get(
-                  entry.getDN().toString());
+              entry.getDN().toString());
           DN matchedDN = getMatchedDN(baseDN);
           throw new DirectoryException(ResultCode.NO_SUCH_OBJECT,
               message, matchedDN, null);
         }
       }
 
-      // First time through, assign the next entryID.
-      if (entryID == null)
-      {
-        entryID = rootContainer.getNextEntryID();
-      }
+      EntryID entryID = rootContainer.getNextEntryID();
 
       // Insert into dn2id.
       if (!dn2id.insert(txn, entry.getDN(), entryID))
@@ -1959,7 +1774,7 @@
         Message message =
             ERR_JEB_ADD_ENTRY_ALREADY_EXISTS.get(entry.getDN().toString());
         throw new DirectoryException(ResultCode.ENTRY_ALREADY_EXISTS,
-                                     message);
+            message);
       }
 
       // Update the referral database for referral entries.
@@ -1969,7 +1784,7 @@
         Message message =
             ERR_JEB_ADD_ENTRY_ALREADY_EXISTS.get(entry.getDN().toString());
         throw new DirectoryException(ResultCode.ENTRY_ALREADY_EXISTS,
-                                     message);
+            message);
       }
 
       // Insert into id2entry.
@@ -1979,7 +1794,7 @@
         Message message =
             ERR_JEB_ADD_ENTRY_ALREADY_EXISTS.get(entry.getDN().toString());
         throw new DirectoryException(ResultCode.ENTRY_ALREADY_EXISTS,
-                                     message);
+            message);
       }
 
       // Insert into the indexes, in index configuration order.
@@ -2015,14 +1830,15 @@
         }
       }
 
-    }
+      if(addOperation != null)
+      {
+        // One last check before committing
+        addOperation.checkIfCanceled(true);
+      }
 
-    /**
-     * This method is called after the transaction has successfully
-     * committed.
-     */
-    public void postCommitAction()
-    {
+      // Commit the transaction.
+      EntryContainer.transactionCommit(txn);
+
       // Update the entry cache.
       EntryCache entryCache = DirectoryServer.getEntryCache();
       if (entryCache != null)
@@ -2030,6 +1846,29 @@
         entryCache.putEntry(entry, backend, entryID.longValue());
       }
     }
+    catch (DatabaseException databaseException)
+    {
+      EntryContainer.transactionAbort(txn);
+      throw databaseException;
+    }
+    catch (DirectoryException directoryException)
+    {
+      EntryContainer.transactionAbort(txn);
+      throw directoryException;
+    }
+    catch (Exception e)
+    {
+      EntryContainer.transactionAbort(txn);
+
+      String msg = e.getMessage();
+      if (msg == null)
+      {
+        msg = stackTraceToSingleLineString(e);
+      }
+      Message message = ERR_JEB_UNCHECKED_EXCEPTION.get(msg);
+      throw new DirectoryException(DirectoryServer.getServerErrorResultCode(),
+          message, e);
+    }
   }
 
   /**
@@ -2046,194 +1885,56 @@
    * @throws DirectoryException If a problem occurs while trying to remove the
    *                            entry.
    * @throws DatabaseException If an error occurs in the JE database.
-   * @throws JebException If an error occurs in the JE backend.
+   * @throws CanceledOperationException if this operation should be cancelled.
    */
   public void deleteEntry(DN entryDN, DeleteOperation deleteOperation)
-      throws DirectoryException, DatabaseException, JebException
+      throws DirectoryException, DatabaseException, CanceledOperationException
   {
-    DeleteEntryTransaction operation =
-        new DeleteEntryTransaction(entryDN, deleteOperation);
-    boolean isComplete = false;
-    while(!isComplete)
-    {
-      invokeTransactedOperation(operation);
+    Transaction txn = beginTransaction();
+    IndexBuffer indexBuffer = null;
 
-      if (operation.adminSizeLimitExceeded())
-      {
-        Message message = NOTE_JEB_SUBTREE_DELETE_SIZE_LIMIT_EXCEEDED.get(
-                operation.getDeletedEntryCount());
-        throw new DirectoryException(
-          ResultCode.ADMIN_LIMIT_EXCEEDED,
-          message);
-      }
-      if(operation.batchSizeExceeded())
-      {
-        operation.resetBatchSize();
-        continue;
-      }
-      isComplete = true;
-      Message message =
-          NOTE_JEB_DELETED_ENTRY_COUNT.get(operation.getDeletedEntryCount());
-      MessageBuilder errorMessage = new MessageBuilder();
-      errorMessage.append(message);
-      deleteOperation.setErrorMessage(errorMessage);
-    }
-  }
-
-  /**
-   * This inner class implements the Delete Entry operation through
-   * the TransactedOperation interface.
-   */
-  private class DeleteEntryTransaction implements TransactedOperation
-  {
-    /**
-     * The DN of the entry or subtree to be deleted.
-     */
-    private DN entryDN;
-
-    /**
-     * The Delete operation.
-     */
-    private DeleteOperation deleteOperation;
-
-
-    /**
-     * Indicates whether the subtree delete size limit has been exceeded.
-     */
-    private boolean adminSizeLimitExceeded = false;
-
-
-    /**
-     * Indicates whether the subtree delete batch size has been exceeded.
-     */
-    private boolean batchSizeExceeded = false;
-
-
-    /**
-     * Indicates the total count of deleted DNs in the Delete Operation.
-     */
-    private int totalDeletedDN;
-
-    /**
-     * Indicates the batch count of deleted DNs in the Delete Operation.
-     */
-    private int batchDeletedDN;
-
-    /**
-     * The index buffer used to buffer up the index changes.
-     */
-    private IndexBuffer indexBuffer = null;
-
-    /**
-     * Create a new Delete Entry Transaction.
-     * @param entryDN The entry or subtree to be deleted.
-     * @param deleteOperation The Delete operation.
-     */
-    public DeleteEntryTransaction(DN entryDN, DeleteOperation deleteOperation)
-    {
-      this.entryDN = entryDN;
-      this.deleteOperation = deleteOperation;
-    }
-
-    /**
-     * Determine whether the subtree delete size limit has been exceeded.
-     * @return true if the size limit has been exceeded.
-     */
-    public boolean adminSizeLimitExceeded()
-    {
-      return adminSizeLimitExceeded;
-    }
-
-    /**
-     * Determine whether the subtree delete batch size has been exceeded.
-     * @return true if the batch size has been exceeded.
-     */
-    public boolean batchSizeExceeded()
-    {
-      return batchSizeExceeded;
-    }
-
-    /**
-     * Resets the batchSizeExceeded parameter to reuse the object
-     * for multiple batches.
-     */
-    public void resetBatchSize()
-    {
-      batchSizeExceeded=false;
-      batchDeletedDN = 0;
-    }
-
-    /**
-     * Get the number of entries deleted during the operation.
-     * @return The number of entries deleted.
-     */
-    public int getDeletedEntryCount()
-    {
-      return totalDeletedDN;
-    }
-
-    /**
-     * Begin a transaction for this operation.
-     *
-     * @return The transaction for the operation, or null if the operation
-     *         will not use a transaction.
-     * @throws DatabaseException If an error occurs in the JE database.
-     */
-    public Transaction beginOperationTransaction() throws DatabaseException
-    {
-      Transaction txn =  beginTransaction();
-      return txn;
-    }
-
-    /**
-     * Invoke the operation under the given transaction.
-     *
-     * @param txn The transaction to be used to perform the operation.
-     * @throws DatabaseException If an error occurs in the JE database.
-     * @throws DirectoryException If a Directory Server error occurs.
-     * @throws JebException If an error occurs in the JE backend.
-     */
-    public void invokeOperation(Transaction txn)
-        throws DatabaseException, DirectoryException, JebException
+    try
     {
       // Check for referral entries above the target entry.
       dn2uri.targetEntryReferrals(entryDN, null);
 
       // Determine whether this is a subtree delete.
       boolean isSubtreeDelete = false;
-      List<Control> controls = deleteOperation.getRequestControls();
-      if (controls != null)
+
+      if(deleteOperation != null)
       {
-        for (Control control : controls)
+        List<Control> controls = deleteOperation.getRequestControls();
+        if (controls != null)
         {
-          if (control.getOID().equals(OID_SUBTREE_DELETE_CONTROL))
+          for (Control control : controls)
           {
-            isSubtreeDelete = true;
+            if (control.getOID().equals(OID_SUBTREE_DELETE_CONTROL))
+            {
+              isSubtreeDelete = true;
+            }
           }
         }
       }
 
       /*
-       * We will iterate backwards through a range of the dn2id keys to
-       * find subordinates of the target entry from the bottom of the tree
-       * upwards. For example, any subordinates of "dc=example,dc=com" appear
-       * in dn2id with a key ending in ",dc=example,dc=com". The entry
-       * "cn=joe,ou=people,dc=example,dc=com" will appear after the entry
-       * "ou=people,dc=example,dc=com".
-       */
+      * We will iterate backwards through a range of the dn2id keys to
+      * find subordinates of the target entry from the bottom of the tree
+      * upwards. For example, any subordinates of "dc=example,dc=com" appear
+      * in dn2id with a key ending in ",dc=example,dc=com". The entry
+      * "cn=joe,ou=people,dc=example,dc=com" will appear after the entry
+      * "ou=people,dc=example,dc=com".
+      */
       byte[] suffix = StaticUtils.getBytes("," + entryDN.toNormalizedString());
 
       /*
-       * Set the starting value to a value of equal length but slightly
-       * greater than the target DN. Since keys are compared in
-       * reverse order we must set the first byte (the comma).
-       * No possibility of overflow here.
-       */
+      * Set the starting value to a value of equal length but slightly
+      * greater than the target DN. Since keys are compared in
+      * reverse order we must set the first byte (the comma).
+      * No possibility of overflow here.
+      */
       byte[] begin = suffix.clone();
       begin[0] = (byte) (begin[0] + 1);
-
-      // Set the ending value to the suffix.
-      byte[] end = suffix;
+      int subordinateEntriesDeleted = 0;
 
       DatabaseEntry data = new DatabaseEntry();
       DatabaseEntry key = new DatabaseEntry(begin);
@@ -2262,7 +1963,7 @@
         // Step back until we pass the ending value.
         while (status == OperationStatus.SUCCESS)
         {
-          int cmp = dn2id.getComparator().compare(key.getData(), end);
+          int cmp = dn2id.getComparator().compare(key.getData(), suffix);
           if (cmp < 0)
           {
             // We have gone past the ending value.
@@ -2275,29 +1976,14 @@
           {
             // The subtree delete control was not specified and
             // the target entry is not a leaf.
+
             Message message =
                 ERR_JEB_DELETE_NOT_ALLOWED_ON_NONLEAF.get(entryDN.toString());
             throw new DirectoryException(ResultCode.NOT_ALLOWED_ON_NONLEAF,
-                                         message);
+                message);
           }
 
-          // Enforce any subtree delete size limit.
-          if (subtreeDeleteSizeLimit > 0 &&
-              totalDeletedDN >= subtreeDeleteSizeLimit)
-          {
-            adminSizeLimitExceeded = true;
-            break;
-          }
-
-          // Enforce any subtree delete batch size.
-          if (subtreeDeleteBatchSize > 0 &&
-              batchDeletedDN >= subtreeDeleteBatchSize)
-          {
-            batchSizeExceeded = true;
-            break;
-          }
-
-          // This is a subtree delete so crate a index buffer
+          // This is a subtree delete so create a index buffer
           // if it there isn't one.
           if(indexBuffer == null)
           {
@@ -2305,15 +1991,19 @@
           }
 
           /*
-           * Delete this entry which by now must be a leaf because
-           * we have been deleting from the bottom of the tree upwards.
-           */
+          * Delete this entry which by now must be a leaf because
+          * we have been deleting from the bottom of the tree upwards.
+          */
           EntryID entryID = new EntryID(data);
           DN subordinateDN = DN.decode(new ASN1OctetString(key.getData()));
-          deleteEntry(txn, true, entryDN, subordinateDN, entryID);
+          deleteEntry(txn, indexBuffer, true, entryDN, subordinateDN, entryID);
+          subordinateEntriesDeleted++;
 
-          batchDeletedDN++;
-          totalDeletedDN++;
+          if(deleteOperation != null)
+          {
+            deleteOperation.checkIfCanceled(false);
+          }
+
           status = cursor.getPrev(key, data, LockMode.DEFAULT);
         }
       }
@@ -2322,192 +2012,193 @@
         cursor.close();
       }
 
-      // Finally delete the target entry as it was not included
-      // in the dn2id iteration.
-      if (!adminSizeLimitExceeded && !batchSizeExceeded)
-      {
-        // Enforce any subtree delete size limit.
-        if (subtreeDeleteSizeLimit > 0 &&
-            totalDeletedDN >= subtreeDeleteSizeLimit)
-        {
-          adminSizeLimitExceeded = true;
-        }
-        else if (subtreeDeleteBatchSize > 0 &&
-            batchDeletedDN >= subtreeDeleteBatchSize)
-        {
-          batchSizeExceeded = true;
-        }
-        else
-        {
-          // draft-armijo-ldap-treedelete, 4.1 Tree Delete Semantics:
-          // The server MUST NOT chase referrals stored in the tree.  If
-          // information about referrals is stored in this section of the
-          // tree, this pointer will be deleted.
-          deleteEntry(txn,
-              isSubtreeDelete || isManageDsaITOperation(deleteOperation),
-              entryDN, null, null);
+      // draft-armijo-ldap-treedelete, 4.1 Tree Delete Semantics:
+      // The server MUST NOT chase referrals stored in the tree.  If
+      // information about referrals is stored in this section of the
+      // tree, this pointer will be deleted.
+      deleteEntry(txn, indexBuffer,
+          isSubtreeDelete || isManageDsaITOperation(deleteOperation),
+          entryDN, null, null);
 
-          batchDeletedDN++;
-          totalDeletedDN++;
-        }
-      }
 
       if(indexBuffer != null)
       {
         indexBuffer.flush(txn);
       }
-    }
 
-    /**
-     * Delete an entry with appropriate handling of referral entries.
-     * The caller must be sure that the entry is indeed a leaf. We cannot
-     * rely on id2children to check for children since this entry may at
-     * one time have had enough children to exceed the index entry limit,
-     * after which the number of children IDs is unknown.
-     *
-     * @param txn    The database transaction.
-     * @param manageDsaIT Whether it is an manage DSA IT operation.
-     * @param targetDN The DN of the target entry.
-     * @param leafDN The DN of the leaf entry to be deleted.
-     * @param leafID The ID of the leaf entry.
-     * @throws DatabaseException If an error occurs in the JE database.
-     * @throws DirectoryException If a Directory Server error occurs.
-     * @throws JebException If an error occurs in the JE backend.
-     */
-    private void deleteEntry(Transaction txn,
-                             boolean manageDsaIT,
-                             DN targetDN,
-                             DN leafDN,
-                             EntryID leafID)
-        throws DatabaseException, DirectoryException, JebException
-    {
-      if(leafID == null || leafDN == null)
+
+      if(deleteOperation != null)
       {
-        // Read the entry ID from dn2id.
-        leafDN = targetDN;
-        leafID = dn2id.get(txn, leafDN, LockMode.RMW);
-        if (leafID == null)
-        {
-          Message message =
-              ERR_JEB_DELETE_NO_SUCH_OBJECT.get(leafDN.toString());
-          DN matchedDN = getMatchedDN(baseDN);
-          throw new DirectoryException(ResultCode.NO_SUCH_OBJECT,
-              message, matchedDN, null);
-        }
+        // One last check before committing
+        deleteOperation.checkIfCanceled(true);
       }
 
-      // Remove from dn2id.
-      if (!dn2id.remove(txn, leafDN))
+      // Commit the transaction.
+      EntryContainer.transactionCommit(txn);
+
+      if(isSubtreeDelete)
       {
-        // Do not expect to ever come through here.
-        Message message = ERR_JEB_DELETE_NO_SUCH_OBJECT.get(leafDN.toString());
+        deleteOperation.appendAdditionalLogMessage(
+            NOTE_JEB_DELETED_ENTRY_COUNT.get(subordinateEntriesDeleted + 1));
+      }
+    }
+    catch (DatabaseException databaseException)
+    {
+      EntryContainer.transactionAbort(txn);
+      throw databaseException;
+    }
+    catch (DirectoryException directoryException)
+    {
+      EntryContainer.transactionAbort(txn);
+      throw directoryException;
+    }
+    catch (CanceledOperationException coe)
+    {
+      EntryContainer.transactionAbort(txn);
+      throw coe;
+    }
+    catch (Exception e)
+    {
+      EntryContainer.transactionAbort(txn);
+
+      String msg = e.getMessage();
+      if (msg == null)
+      {
+        msg = stackTraceToSingleLineString(e);
+      }
+      Message message = ERR_JEB_UNCHECKED_EXCEPTION.get(msg);
+      throw new DirectoryException(DirectoryServer.getServerErrorResultCode(),
+          message, e);
+    }
+  }
+
+  private void deleteEntry(Transaction txn,
+                           IndexBuffer indexBuffer,
+                           boolean manageDsaIT,
+                           DN targetDN,
+                           DN leafDN,
+                           EntryID leafID)
+      throws DatabaseException, DirectoryException, JebException
+  {
+    if(leafID == null || leafDN == null)
+    {
+      // Read the entry ID from dn2id.
+      leafDN = targetDN;
+      leafID = dn2id.get(txn, leafDN, LockMode.RMW);
+      if (leafID == null)
+      {
+        Message message =
+            ERR_JEB_DELETE_NO_SUCH_OBJECT.get(leafDN.toString());
         DN matchedDN = getMatchedDN(baseDN);
         throw new DirectoryException(ResultCode.NO_SUCH_OBJECT,
             message, matchedDN, null);
       }
+    }
 
-      // Check that the entry exists in id2entry and read its contents.
-      Entry entry = id2entry.get(txn, leafID, LockMode.RMW);
-      if (entry == null)
+    // Remove from dn2id.
+    if (!dn2id.remove(txn, leafDN))
+    {
+      // Do not expect to ever come through here.
+      Message message = ERR_JEB_DELETE_NO_SUCH_OBJECT.get(leafDN.toString());
+      DN matchedDN = getMatchedDN(baseDN);
+      throw new DirectoryException(ResultCode.NO_SUCH_OBJECT,
+          message, matchedDN, null);
+    }
+
+    // Check that the entry exists in id2entry and read its contents.
+    Entry entry = id2entry.get(txn, leafID, LockMode.RMW);
+    if (entry == null)
+    {
+      Message msg = ERR_JEB_MISSING_ID2ENTRY_RECORD.get(leafID.toString());
+      throw new DirectoryException(DirectoryServer.getServerErrorResultCode(),
+          msg);
+    }
+
+    if (!manageDsaIT)
+    {
+      dn2uri.checkTargetForReferral(entry, null);
+    }
+
+    // Update the referral database.
+    dn2uri.deleteEntry(txn, entry);
+
+    // Remove from id2entry.
+    if (!id2entry.remove(txn, leafID))
+    {
+      Message msg = ERR_JEB_MISSING_ID2ENTRY_RECORD.get(leafID.toString());
+      throw new DirectoryException(DirectoryServer.getServerErrorResultCode(),
+          msg);
+    }
+
+    // Remove from the indexes, in index config order.
+    if(indexBuffer != null)
+    {
+      indexRemoveEntry(indexBuffer, entry, leafID);
+    }
+    else
+    {
+      indexRemoveEntry(txn, entry, leafID);
+    }
+
+    // Remove the id2c and id2s records for this entry.
+    if(indexBuffer != null)
+    {
+      byte[] leafIDKeyBytes =
+          JebFormat.entryIDToDatabase(leafID.longValue());
+      id2children.delete(indexBuffer, leafIDKeyBytes);
+      id2subtree.delete(indexBuffer, leafIDKeyBytes);
+    }
+    else
+    {
+      DatabaseEntry leafIDKey = leafID.getDatabaseEntry();
+      id2children.delete(txn, leafIDKey);
+      id2subtree.delete(txn, leafIDKey);
+    }
+
+    // Iterate up through the superior entries from the target entry.
+    boolean isParent = true;
+    for (DN parentDN = getParentWithinBase(targetDN); parentDN != null;
+         parentDN = getParentWithinBase(parentDN))
+    {
+      // Read the ID from dn2id.
+      EntryID parentID = dn2id.get(txn, parentDN, LockMode.DEFAULT);
+      if (parentID == null)
       {
-        Message msg = ERR_JEB_MISSING_ID2ENTRY_RECORD.get(leafID.toString());
+        Message msg =
+            ERR_JEB_MISSING_DN2ID_RECORD.get(parentDN.toNormalizedString());
         throw new JebException(msg);
       }
 
-      if (!manageDsaIT)
-      {
-        dn2uri.checkTargetForReferral(entry, null);
-      }
-
-      // Update the referral database.
-      dn2uri.deleteEntry(txn, entry);
-
-      // Remove from id2entry.
-      if (!id2entry.remove(txn, leafID))
-      {
-        Message msg = ERR_JEB_MISSING_ID2ENTRY_RECORD.get(leafID.toString());
-        throw new JebException(msg);
-      }
-
-      // Remove from the indexes, in index config order.
       if(indexBuffer != null)
       {
-       indexRemoveEntry(indexBuffer, entry, leafID);
+        byte[] parentIDBytes =
+            JebFormat.entryIDToDatabase(parentID.longValue());
+        // Remove from id2children.
+        if (isParent)
+        {
+          id2children.removeID(indexBuffer, parentIDBytes, leafID);
+          isParent = false;
+        }
+        id2subtree.removeID(indexBuffer, parentIDBytes, leafID);
       }
       else
       {
-        indexRemoveEntry(txn, entry, leafID);
-      }
-
-      // Remove the id2c and id2s records for this entry.
-      if(indexBuffer != null)
-      {
-        byte[] leafIDKeyBytes =
-            JebFormat.entryIDToDatabase(leafID.longValue());
-        id2children.delete(indexBuffer, leafIDKeyBytes);
-        id2subtree.delete(indexBuffer, leafIDKeyBytes);
-      }
-      else
-      {
-        DatabaseEntry leafIDKey = leafID.getDatabaseEntry();
-        id2children.delete(txn, leafIDKey);
-        id2subtree.delete(txn, leafIDKey);
-      }
-
-      // Iterate up through the superior entries from the target entry.
-      boolean isParent = true;
-      for (DN parentDN = getParentWithinBase(targetDN); parentDN != null;
-           parentDN = getParentWithinBase(parentDN))
-      {
-        // Read the ID from dn2id.
-        EntryID parentID = dn2id.get(txn, parentDN, LockMode.DEFAULT);
-        if (parentID == null)
+        DatabaseEntry nodeIDData = parentID.getDatabaseEntry();
+        // Remove from id2children.
+        if(isParent)
         {
-          Message msg =
-              ERR_JEB_MISSING_DN2ID_RECORD.get(parentDN.toNormalizedString());
-          throw new JebException(msg);
+          id2children.removeID(txn, nodeIDData, leafID);
+          isParent = false;
         }
-
-        if(indexBuffer != null)
-        {
-          byte[] parentIDBytes =
-              JebFormat.entryIDToDatabase(parentID.longValue());
-          // Remove from id2children.
-          if (isParent)
-          {
-            id2children.removeID(indexBuffer, parentIDBytes, leafID);
-            isParent = false;
-          }
-          id2subtree.removeID(indexBuffer, parentIDBytes, leafID);
-        }
-        else
-        {
-          DatabaseEntry nodeIDData = parentID.getDatabaseEntry();
-          // Remove from id2children.
-          if(isParent)
-          {
-            id2children.removeID(txn, nodeIDData, leafID);
-            isParent = false;
-          }
-          id2subtree.removeID(txn, nodeIDData, leafID);
-        }
-      }
-
-      // Remove the entry from the entry cache.
-      EntryCache entryCache = DirectoryServer.getEntryCache();
-      if (entryCache != null)
-      {
-        entryCache.removeEntry(leafDN);
+        id2subtree.removeID(txn, nodeIDData, leafID);
       }
     }
 
-    /**
-     * This method is called after the transaction has successfully
-     * committed.
-     */
-    public void postCommitAction()
+    // Remove the entry from the entry cache.
+    EntryCache entryCache = DirectoryServer.getEntryCache();
+    if (entryCache != null)
     {
-
+      entryCache.removeEntry(leafDN);
     }
   }
 
@@ -2564,11 +2255,10 @@
    *         exist.
    * @throws DirectoryException If a problem occurs while trying to retrieve
    *                            the entry.
-   * @throws JebException If an error occurs in the JE backend.
    * @throws DatabaseException An error occurred during a database operation.
    */
   public Entry getEntry(DN entryDN)
-      throws JebException, DatabaseException, DirectoryException
+      throws DatabaseException, DirectoryException
   {
     EntryCache entryCache = DirectoryServer.getEntryCache();
     Entry entry = null;
@@ -2581,103 +2271,8 @@
 
     if (entry == null)
     {
-      GetEntryByDNOperation operation = new GetEntryByDNOperation(entryDN);
-
-      // Fetch the entry from the database.
-      invokeTransactedOperation(operation);
-
-      entry = operation.getEntry();
-
-      // Put the entry in the cache making sure not to overwrite
-      // a newer copy that may have been inserted since the time
-      // we read the cache.
-      if (entry != null && entryCache != null)
-      {
-        entryCache.putEntryIfAbsent(entry, backend,
-                                    operation.getEntryID().longValue());
-      }
-    }
-
-    return entry;
-  }
-
-  /**
-   * This inner class gets an entry by DN through
-   * the TransactedOperation interface.
-   */
-  private class GetEntryByDNOperation implements TransactedOperation
-  {
-    /**
-     * The retrieved entry.
-     */
-    private Entry entry = null;
-
-    /**
-     * The ID of the retrieved entry.
-     */
-    private EntryID entryID = null;
-
-    /**
-     * The DN of the entry to be retrieved.
-     */
-    DN entryDN;
-
-    /**
-     * Create a new transacted operation to retrieve an entry by DN.
-     * @param entryDN The DN of the entry to be retrieved.
-     */
-    public GetEntryByDNOperation(DN entryDN)
-    {
-      this.entryDN = entryDN;
-    }
-
-    /**
-     * Get the retrieved entry.
-     * @return The retrieved entry.
-     */
-    public Entry getEntry()
-    {
-      return entry;
-    }
-
-    /**
-     * Get the ID of the retrieved entry.
-     * @return The ID of the retrieved entry.
-     */
-    public EntryID getEntryID()
-    {
-      return entryID;
-    }
-
-    /**
-     * Begin a transaction for this operation.
-     *
-     * @return The transaction for the operation, or null if the operation
-     *         will not use a transaction.
-     * @throws DatabaseException If an error occurs in the JE database.
-     */
-    public Transaction beginOperationTransaction() throws DatabaseException
-    {
-      // For best performance queries do not use a transaction.
-      // We permit temporary inconsistencies between the multiple
-      // records that make up a single entry.
-      return null;
-    }
-
-    /**
-     * Invoke the operation under the given transaction.
-     *
-     * @param txn The transaction to be used to perform the operation
-     * @throws DatabaseException If an error occurs in the JE database.
-     * @throws DirectoryException If a Directory Server error occurs.
-     * @throws JebException If an error occurs in the JE backend.
-     */
-    public void invokeOperation(Transaction txn) throws DatabaseException,
-                                                        DirectoryException,
-                                                        JebException
-    {
       // Read dn2id.
-      entryID = dn2id.get(txn, entryDN, LockMode.DEFAULT);
+      EntryID entryID = dn2id.get(null, entryDN, LockMode.DEFAULT);
       if (entryID == null)
       {
         // The entryDN does not exist.
@@ -2685,113 +2280,30 @@
         // Check for referral entries above the target entry.
         dn2uri.targetEntryReferrals(entryDN, null);
 
-        return;
+        return null;
       }
 
       // Read id2entry.
-      entry = id2entry.get(txn, entryID, LockMode.DEFAULT);
+      entry = id2entry.get(null, entryID, LockMode.DEFAULT);
 
       if (entry == null)
       {
         // The entryID does not exist.
         Message msg = ERR_JEB_MISSING_ID2ENTRY_RECORD.get(entryID.toString());
-        throw new JebException(msg);
+        throw new DirectoryException(DirectoryServer.getServerErrorResultCode(),
+          msg);
       }
 
+      // Put the entry in the cache making sure not to overwrite
+      // a newer copy that may have been inserted since the time
+      // we read the cache.
+      if (entryCache != null)
+      {
+        entryCache.putEntryIfAbsent(entry, backend, entryID.longValue());
+      }
     }
 
-    /**
-     * This method is called after the transaction has successfully
-     * committed.
-     */
-    public void postCommitAction()
-    {
-      // No implementation required.
-    }
-  }
-
-  /**
-   * This inner class gets an entry by ID through
-   * the TransactedOperation interface.
-   */
-  private class GetEntryByIDOperation implements TransactedOperation
-  {
-    /**
-     * The retrieved entry.
-     */
-    private Entry entry = null;
-
-    /**
-     * The ID of the entry to be retrieved.
-     */
-    private EntryID entryID;
-
-    /**
-     * Create a new transacted operation to retrieve an entry by ID.
-     * @param entryID The ID of the entry to be retrieved.
-     */
-    public GetEntryByIDOperation(EntryID entryID)
-    {
-      this.entryID = entryID;
-    }
-
-    /**
-     * Get the retrieved entry.
-     * @return The retrieved entry.
-     */
-    public Entry getEntry()
-    {
-      return entry;
-    }
-
-    /**
-     * Get the ID of the retrieved entry.
-     * @return the ID of the retrieved entry.
-     */
-    public EntryID getEntryID()
-    {
-      return entryID;
-    }
-
-    /**
-     * Begin a transaction for this operation.
-     *
-     * @return The transaction for the operation, or null if the operation
-     *         will not use a transaction.
-     * @throws DatabaseException If an error occurs in the JE database.
-     */
-    public Transaction beginOperationTransaction() throws DatabaseException
-    {
-      // For best performance queries do not use a transaction.
-      // We permit temporary inconsistencies between the multiple
-      // records that make up a single entry.
-      return null;
-    }
-
-    /**
-     * Invoke the operation under the given transaction.
-     *
-     * @param txn The transaction to be used to perform the operation.
-     * @throws DatabaseException If an error occurs in the JE database.
-     * @throws DirectoryException If a Directory Server error occurs.
-     * @throws JebException If an error occurs in the JE backend.
-     */
-    public void invokeOperation(Transaction txn) throws DatabaseException,
-                                                        DirectoryException,
-                                                        JebException
-    {
-      // Read id2entry.
-      entry = id2entry.get(txn, entryID, LockMode.DEFAULT);
-    }
-
-    /**
-     * This method is called after the transaction has successfully
-     * committed.
-     */
-    public void postCommitAction()
-    {
-      // No implementation required.
-    }
+    return entry;
   }
 
   /**
@@ -2804,83 +2316,22 @@
    *                        modifications performed internally.
    * @throws DatabaseException If an error occurs in the JE database.
    * @throws DirectoryException If a Directory Server error occurs.
-   * @throws JebException If an error occurs in the JE backend.
+   * @throws CanceledOperationException if this operation should be cancelled.
    */
   public void replaceEntry(Entry entry, ModifyOperation modifyOperation)
-       throws DatabaseException, DirectoryException, JebException
+      throws DatabaseException, DirectoryException, CanceledOperationException
   {
-    TransactedOperation operation =
-         new ReplaceEntryTransaction(entry, modifyOperation);
+    Transaction txn = beginTransaction();
 
-    invokeTransactedOperation(operation);
-  }
-
-  /**
-   * This inner class implements the Replace Entry operation through
-   * the TransactedOperation interface.
-   */
-  private class ReplaceEntryTransaction implements TransactedOperation
-  {
-    /**
-     * The new contents of the entry.
-     */
-    private Entry entry;
-
-    /**
-     * The Modify operation, or null if the replace is not due to a Modify
-     * operation.
-     */
-    private ModifyOperation modifyOperation;
-
-    /**
-     * The ID of the entry that was replaced.
-     */
-    private EntryID entryID = null;
-
-    /**
-     * Create a new transacted operation to replace an entry.
-     * @param entry The new contents of the entry.
-     * @param modifyOperation The Modify operation, or null if the replace is
-     * not due to a Modify operation.
-     */
-    public ReplaceEntryTransaction(Entry entry,
-                                   ModifyOperation modifyOperation)
-    {
-      this.entry = entry;
-      this.modifyOperation = modifyOperation;
-    }
-
-    /**
-     * Begin a transaction for this operation.
-     *
-     * @return The transaction for the operation, or null if the operation
-     *         will not use a transaction.
-     * @throws DatabaseException If an error occurs in the JE database.
-     */
-    public Transaction beginOperationTransaction() throws DatabaseException
-    {
-      return beginTransaction();
-    }
-
-    /**
-     * Invoke the operation under the given transaction.
-     *
-     * @param txn The transaction to be used to perform the operation.
-     * @throws DatabaseException If an error occurs in the JE database.
-     * @throws DirectoryException If a Directory Server error occurs.
-     * @throws JebException If an error occurs in the JE backend.
-     */
-    public void invokeOperation(Transaction txn) throws DatabaseException,
-                                                        DirectoryException,
-                                                        JebException
+    try
     {
       // Read dn2id.
-      entryID = dn2id.get(txn, entry.getDN(), LockMode.RMW);
+      EntryID entryID = dn2id.get(txn, entry.getDN(), LockMode.RMW);
       if (entryID == null)
       {
         // The entry does not exist.
         Message message =
-                ERR_JEB_MODIFY_NO_SUCH_OBJECT.get(entry.getDN().toString());
+            ERR_JEB_MODIFY_NO_SUCH_OBJECT.get(entry.getDN().toString());
         DN matchedDN = getMatchedDN(baseDN);
         throw new DirectoryException(ResultCode.NO_SUCH_OBJECT,
             message, matchedDN, null);
@@ -2892,7 +2343,8 @@
       {
         // The entry does not exist.
         Message msg = ERR_JEB_MISSING_ID2ENTRY_RECORD.get(entryID.toString());
-        throw new JebException(msg);
+        throw new DirectoryException(
+              DirectoryServer.getServerErrorResultCode(), msg);
       }
 
       if (!isManageDsaITOperation(modifyOperation))
@@ -2929,14 +2381,16 @@
         indexRemoveEntry(txn, originalEntry, entryID);
         indexInsertEntry(txn, entry, entryID);
       }
-    }
 
-    /**
-     * This method is called after the transaction has successfully
-     * committed.
-     */
-    public void postCommitAction()
-    {
+      if(modifyOperation != null)
+      {
+        // One last check before committing
+        modifyOperation.checkIfCanceled(true);
+      }
+
+      // Commit the transaction.
+      EntryContainer.transactionCommit(txn);
+
       // Update the entry cache.
       EntryCache entryCache = DirectoryServer.getEntryCache();
       if (entryCache != null)
@@ -2944,6 +2398,29 @@
         entryCache.putEntry(entry, backend, entryID.longValue());
       }
     }
+    catch (DatabaseException databaseException)
+    {
+      EntryContainer.transactionAbort(txn);
+      throw databaseException;
+    }
+    catch (DirectoryException directoryException)
+    {
+      EntryContainer.transactionAbort(txn);
+      throw directoryException;
+    }
+    catch (Exception e)
+    {
+      EntryContainer.transactionAbort(txn);
+
+      String msg = e.getMessage();
+      if (msg == null)
+      {
+        msg = stackTraceToSingleLineString(e);
+      }
+      Message message = ERR_JEB_UNCHECKED_EXCEPTION.get(msg);
+      throw new DirectoryException(DirectoryServer.getServerErrorResultCode(),
+          message, e);
+    }
   }
 
   /**
@@ -2966,115 +2443,50 @@
    *          to a request to cancel or abandon the
    *          modify DN operation.
    * @throws DatabaseException If an error occurs in the JE database.
-   * @throws JebException If an error occurs in the JE backend.
    */
   public void renameEntry(DN currentDN, Entry entry,
                           ModifyDNOperation modifyDNOperation)
-      throws DatabaseException, JebException, DirectoryException,
-      CanceledOperationException {
-    TransactedOperation operation =
-        new RenameEntryTransaction(currentDN, entry, modifyDNOperation);
-
-    invokeTransactedOperation(operation);
-  }
-
-  /**
-   * This inner class implements the Modify DN operation through
-   * the TransactedOperation interface.
-   */
-  private class RenameEntryTransaction implements TransactedOperation
+      throws DatabaseException, DirectoryException, CanceledOperationException
   {
-    /**
-     * The DN of the entry to be renamed.
-     */
-    private DN oldApexDN;
+    Transaction txn = beginTransaction();
+    DN oldSuperiorDN = getParentWithinBase(currentDN);
+    DN newSuperiorDN = getParentWithinBase(entry.getDN());
+    boolean isApexEntryMoved;
 
-    /**
-     * The DN of the superior entry of the entry to be renamed.
-     * This is null if the entry to be renamed is a base entry.
-     */
-    private DN oldSuperiorDN;
-
-    /**
-     * The DN of the new superior entry, which can be the same
-     * as the current superior entry.
-     */
-    private DN newSuperiorDN;
-
-    /**
-     * The new contents of the entry to be renamed.
-     */
-    private Entry newApexEntry;
-
-    /**
-     * The Modify DN operation.
-     */
-    private ModifyDNOperation modifyDNOperation;
-
-    /**
-     * Whether the apex entry moved under another parent.
-     */
-    private boolean isApexEntryMoved;
-
-    /**
-     * Create a new transacted operation for a Modify DN operation.
-     * @param currentDN The DN of the entry to be renamed.
-     * @param entry The new contents of the entry.
-     * @param modifyDNOperation The Modify DN operation to be performed.
-     */
-    public RenameEntryTransaction(DN currentDN, Entry entry,
-                                  ModifyDNOperation modifyDNOperation)
+    if(oldSuperiorDN != null)
     {
-      this.oldApexDN = currentDN;
-      this.oldSuperiorDN = getParentWithinBase(currentDN);
-      this.newSuperiorDN = getParentWithinBase(entry.getDN());
-      this.newApexEntry = entry;
-      this.modifyDNOperation = modifyDNOperation;
-
-      if(oldSuperiorDN != null)
-      {
-        this.isApexEntryMoved = ! oldSuperiorDN.equals(newSuperiorDN);
-      }
-      else if(newSuperiorDN != null)
-      {
-        this.isApexEntryMoved = ! newSuperiorDN.equals(oldSuperiorDN);
-      }
-      else
-      {
-        this.isApexEntryMoved = false;
-      }
+      isApexEntryMoved = ! oldSuperiorDN.equals(newSuperiorDN);
+    }
+    else if(newSuperiorDN != null)
+    {
+      isApexEntryMoved = ! newSuperiorDN.equals(oldSuperiorDN);
+    }
+    else
+    {
+      isApexEntryMoved = false;
     }
 
-    /**
-     * Invoke the operation under the given transaction.
-     *
-     * @param txn The transaction to be used to perform the operation.
-     * @throws DatabaseException If an error occurs in the JE database.
-     * @throws DirectoryException If a Directory Server error occurs.
-     * @throws JebException If an error occurs in the JE backend.
-     */
-    public void invokeOperation(Transaction txn)
-        throws DatabaseException, DirectoryException, JebException
-    {
-      IndexBuffer buffer = new IndexBuffer(EntryContainer.this);
+    IndexBuffer buffer = new IndexBuffer(EntryContainer.this);
 
+    try
+    {
       // Check whether the renamed entry already exists.
-      if (dn2id.get(txn, newApexEntry.getDN(), LockMode.DEFAULT) != null)
+      if (dn2id.get(txn, entry.getDN(), LockMode.DEFAULT) != null)
       {
         Message message = ERR_JEB_MODIFYDN_ALREADY_EXISTS.get(
-            newApexEntry.getDN().toString());
+            entry.getDN().toString());
         throw new DirectoryException(ResultCode.ENTRY_ALREADY_EXISTS,
                                      message);
       }
 
-      EntryID oldApexID = dn2id.get(txn, oldApexDN, LockMode.DEFAULT);
+      EntryID oldApexID = dn2id.get(txn, currentDN, LockMode.DEFAULT);
       if (oldApexID == null)
       {
         // Check for referral entries above the target entry.
-        dn2uri.targetEntryReferrals(oldApexDN, null);
+        dn2uri.targetEntryReferrals(currentDN, null);
 
         Message message =
-                ERR_JEB_MODIFYDN_NO_SUCH_OBJECT.get(oldApexDN.toString());
+                ERR_JEB_MODIFYDN_NO_SUCH_OBJECT.get(currentDN.toString());
         DN matchedDN = getMatchedDN(baseDN);
         throw new DirectoryException(ResultCode.NO_SUCH_OBJECT,
             message, matchedDN, null);
@@ -3084,7 +2496,8 @@
       if (oldApexEntry == null)
       {
         Message msg = ERR_JEB_MISSING_ID2ENTRY_RECORD.get(oldApexID.toString());
-        throw new JebException(msg);
+        throw new DirectoryException(
+              DirectoryServer.getServerErrorResultCode(), msg);
       }
 
       if (!isManageDsaITOperation(modifyDNOperation))
@@ -3128,7 +2541,7 @@
                 "Old entry ID: %d " +
                 "New entry ID: %d " +
                 "New Superior ID: %d" +
-                oldApexEntry.getDN(), newApexEntry.getDN(),
+                oldApexEntry.getDN(), entry.getDN(),
                 oldApexID.longValue(), newApexID.longValue(),
                 newSuperiorID.longValue());
           }
@@ -3136,8 +2549,8 @@
       }
 
       // Move or rename the apex entry.
-      renameApexEntry(txn, buffer, oldApexID, newApexID, oldApexEntry,
-            newApexEntry);
+      renameApexEntry(txn, buffer, oldSuperiorDN, newSuperiorDN, oldApexID,
+          newApexID, oldApexEntry, entry,isApexEntryMoved, modifyDNOperation);
 
       /*
        * We will iterate forwards through a range of the dn2id keys to
@@ -3145,7 +2558,7 @@
        * downwards.
        */
       byte[] suffix = StaticUtils.getBytes("," +
-          oldApexDN.toNormalizedString());
+          currentDN.toNormalizedString());
 
       /*
        * Set the ending value to a value of equal length but slightly
@@ -3154,12 +2567,8 @@
       byte[] end = suffix.clone();
       end[0] = (byte) (end[0] + 1);
 
-      // Set the starting value to the suffix.
-      byte[] begin = suffix;
-
       DatabaseEntry data = new DatabaseEntry();
-      DatabaseEntry key = new DatabaseEntry(begin);
-      int subordinateEntriesMoved = 0;
+      DatabaseEntry key = new DatabaseEntry(suffix);
 
       CursorConfig cursorConfig = new CursorConfig();
       cursorConfig.setReadCommitted(true);
@@ -3173,7 +2582,7 @@
 
         // Step forward until the key is greater than the starting value.
         while (status == OperationStatus.SUCCESS &&
-            dn2id.getComparator().compare(key.getData(), begin) <= 0)
+            dn2id.getComparator().compare(key.getData(), suffix) <= 0)
         {
           status = cursor.getNext(key, data, LockMode.DEFAULT);
         }
@@ -3195,8 +2604,8 @@
 
           // Construct the new DN of the entry.
           DN newDN = modDN(oldEntry.getDN(),
-                           oldApexDN.getNumComponents(),
-                           newApexEntry.getDN());
+                           currentDN.getNumComponents(),
+                           entry.getDN());
 
           // Assign a new entry ID if we are renumbering.
           EntryID newID = oldID;
@@ -3218,13 +2627,13 @@
           }
 
           // Move this entry.
-          renameSubordinateEntry(txn, buffer, oldID, newID, oldEntry, newDN);
-          subordinateEntriesMoved++;
+          renameSubordinateEntry(txn, buffer, oldSuperiorDN, newSuperiorDN,
+              oldID, newID, oldEntry, newDN, isApexEntryMoved,
+              modifyDNOperation);
 
-          if(subordinateEntriesMoved >= subtreeDeleteBatchSize)
+          if(modifyDNOperation != null)
           {
-            buffer.flush(txn);
-            subordinateEntriesMoved = 0;
+            modifyDNOperation.checkIfCanceled(false);
           }
 
           // Get the next DN.
@@ -3237,300 +2646,293 @@
       }
 
       buffer.flush(txn);
-    }
 
-    /**
-     * Begin a transaction for this operation.
-     *
-     * @return The transaction for the operation, or null if the operation
-     *         will not use a transaction.
-     * @throws DatabaseException If an error occurs in the JE database.
-     */
-    public Transaction beginOperationTransaction() throws DatabaseException
-    {
-      return beginTransaction();
-    }
-
-    /**
-     * Update the database for the target entry of a Modify DN operation
-     * not specifying a new superior.
-     *
-     * @param txn The database transaction to be used for the updates.
-     * @param buffer The index buffer used to buffer up the index changes.
-     * @param oldID The old ID of the target entry.
-     * @param newID The new ID of the target entry.
-     * @param oldEntry The original contents of the target entry.
-     * @param newEntry The new contents of the target entry.
-     * @throws DirectoryException If a Directory Server error occurs.
-     * @throws DatabaseException If an error occurs in the JE database.
-     * @throws JebException if an error occurs in the JE database.
-     */
-    private void renameApexEntry(Transaction txn, IndexBuffer buffer,
-                                 EntryID oldID, EntryID newID,
-                                 Entry oldEntry, Entry newEntry)
-        throws DirectoryException, DatabaseException, JebException
-    {
-      DN oldDN = oldEntry.getDN();
-      DN newDN = newEntry.getDN();
-
-      // Remove the old DN from dn2id.
-      dn2id.remove(txn, oldDN);
-
-      // Put the new DN in dn2id.
-      if (!dn2id.insert(txn, newDN, newID))
+      if(modifyDNOperation != null)
       {
-        Message message = ERR_JEB_MODIFYDN_ALREADY_EXISTS.get(newDN.toString());
-        throw new DirectoryException(ResultCode.ENTRY_ALREADY_EXISTS,
-            message);
+        // One last check before committing
+        modifyDNOperation.checkIfCanceled(true);
       }
 
-      // Remove old ID from id2entry and put the new entry
-      // (old entry with new DN) in id2entry.
-      if (!newID.equals(oldID))
+      // Commit the transaction.
+      EntryContainer.transactionCommit(txn);
+    }
+    catch (DatabaseException databaseException)
+    {
+      EntryContainer.transactionAbort(txn);
+      throw databaseException;
+    }
+    catch (DirectoryException directoryException)
+    {
+      EntryContainer.transactionAbort(txn);
+      throw directoryException;
+    }
+    catch (CanceledOperationException coe)
+    {
+      EntryContainer.transactionAbort(txn);
+      throw coe;
+    }
+    catch (Exception e)
+    {
+      EntryContainer.transactionAbort(txn);
+
+      String msg = e.getMessage();
+      if (msg == null)
       {
-        id2entry.remove(txn, oldID);
+        msg = stackTraceToSingleLineString(e);
       }
-      id2entry.put(txn, newID, newEntry);
+      Message message = ERR_JEB_UNCHECKED_EXCEPTION.get(msg);
+      throw new DirectoryException(DirectoryServer.getServerErrorResultCode(),
+          message, e);
+    }
+  }
 
-      // Update any referral records.
-      dn2uri.replaceEntry(txn, oldEntry, newEntry);
+  private void renameApexEntry(Transaction txn, IndexBuffer buffer,
+                               DN oldSuperiorDN, DN newSuperiorDN,
+                               EntryID oldID, EntryID newID,
+                               Entry oldEntry, Entry newEntry,
+                               boolean isApexEntryMoved,
+                               ModifyDNOperation modifyDNOperation)
+      throws DirectoryException, DatabaseException
+  {
+    DN oldDN = oldEntry.getDN();
+    DN newDN = newEntry.getDN();
 
-      // Remove the old ID from id2children and id2subtree of
-      // the old apex parent entry.
-      if(oldSuperiorDN != null && isApexEntryMoved)
+    // Remove the old DN from dn2id.
+    dn2id.remove(txn, oldDN);
+
+    // Put the new DN in dn2id.
+    if (!dn2id.insert(txn, newDN, newID))
+    {
+      Message message = ERR_JEB_MODIFYDN_ALREADY_EXISTS.get(newDN.toString());
+      throw new DirectoryException(ResultCode.ENTRY_ALREADY_EXISTS,
+          message);
+    }
+
+    // Remove old ID from id2entry and put the new entry
+    // (old entry with new DN) in id2entry.
+    if (!newID.equals(oldID))
+    {
+      id2entry.remove(txn, oldID);
+    }
+    id2entry.put(txn, newID, newEntry);
+
+    // Update any referral records.
+    dn2uri.replaceEntry(txn, oldEntry, newEntry);
+
+    // Remove the old ID from id2children and id2subtree of
+    // the old apex parent entry.
+    if(oldSuperiorDN != null && isApexEntryMoved)
+    {
+      EntryID parentID;
+      byte[] parentIDKeyBytes;
+      boolean isParent = true;
+      for (DN dn = oldSuperiorDN; dn != null; dn = getParentWithinBase(dn))
       {
-        EntryID parentID;
-        byte[] parentIDKeyBytes;
-        boolean isParent = true;
-        for (DN dn = oldSuperiorDN; dn != null; dn = getParentWithinBase(dn))
+        parentID = dn2id.get(txn, dn, LockMode.DEFAULT);
+        parentIDKeyBytes =
+            JebFormat.entryIDToDatabase(parentID.longValue());
+        if(isParent)
         {
-          parentID = dn2id.get(txn, dn, LockMode.DEFAULT);
-          parentIDKeyBytes =
-              JebFormat.entryIDToDatabase(parentID.longValue());
-          if(isParent)
-          {
-            id2children.removeID(buffer, parentIDKeyBytes, oldID);
-            isParent = false;
-          }
-          id2subtree.removeID(buffer, parentIDKeyBytes, oldID);
+          id2children.removeID(buffer, parentIDKeyBytes, oldID);
+          isParent = false;
         }
-      }
-
-      if (!newID.equals(oldID) || modifyDNOperation == null)
-      {
-        // All the subordinates will be renumbered so we have to rebuild
-        // id2c and id2s with the new ID.
-        byte[] oldIDKeyBytes = JebFormat.entryIDToDatabase(oldID.longValue());
-        id2children.delete(buffer, oldIDKeyBytes);
-        id2subtree.delete(buffer, oldIDKeyBytes);
-
-        // Reindex the entry with the new ID.
-        indexRemoveEntry(buffer, oldEntry, oldID);
-        indexInsertEntry(buffer, newEntry, newID);
-      }
-      else
-      {
-        // Update the indexes if needed.
-        indexModifications(buffer, oldEntry, newEntry, oldID,
-            modifyDNOperation.getModifications());
-      }
-
-      // Add the new ID to id2children and id2subtree of new apex parent entry.
-      if(newSuperiorDN != null && isApexEntryMoved)
-      {
-        EntryID parentID;
-        byte[] parentIDKeyBytes;
-        boolean isParent = true;
-        for (DN dn = newSuperiorDN; dn != null; dn = getParentWithinBase(dn))
-        {
-          parentID = dn2id.get(txn, dn, LockMode.DEFAULT);
-          parentIDKeyBytes =
-              JebFormat.entryIDToDatabase(parentID.longValue());
-          if(isParent)
-          {
-            id2children.insertID(buffer, parentIDKeyBytes, newID);
-            isParent = false;
-          }
-          id2subtree.insertID(buffer, parentIDKeyBytes, newID);
-        }
-      }
-
-      // Remove the entry from the entry cache.
-      EntryCache entryCache = DirectoryServer.getEntryCache();
-      if (entryCache != null)
-      {
-        entryCache.removeEntry(oldDN);
+        id2subtree.removeID(buffer, parentIDKeyBytes, oldID);
       }
     }
 
-    /**
-     * Update the database for a subordinate entry of the target entry
-     * of a Modify DN operation specifying a new superior.
-     *
-     * @param txn The database transaction to be used for the updates.
-     * @param buffer The index buffer used to buffer up the index changes.
-     * @param oldID The original ID of the subordinate entry.
-     * @param newID The new ID of the subordinate entry, or the original ID if
-     *              the ID has not changed.
-     * @param oldEntry The original contents of the subordinate entry.
-     * @param newDN The new DN of the subordinate entry.
-     * @throws JebException If an error occurs in the JE backend.
-     * @throws DirectoryException If a Directory Server error occurs.
-     * @throws DatabaseException If an error occurs in the JE database.
-     */
-    private void renameSubordinateEntry(Transaction txn, IndexBuffer buffer,
-                                        EntryID oldID, EntryID newID,
-                                        Entry oldEntry, DN newDN)
-        throws JebException, DirectoryException, DatabaseException
+    if (!newID.equals(oldID) || modifyDNOperation == null)
     {
-      DN oldDN = oldEntry.getDN();
-      Entry newEntry = oldEntry.duplicate(false);
-      newEntry.setDN(newDN);
-      List<Modification> modifications =
-          Collections.unmodifiableList(new ArrayList<Modification>(0));
+      // All the subordinates will be renumbered so we have to rebuild
+      // id2c and id2s with the new ID.
+      byte[] oldIDKeyBytes = JebFormat.entryIDToDatabase(oldID.longValue());
+      id2children.delete(buffer, oldIDKeyBytes);
+      id2subtree.delete(buffer, oldIDKeyBytes);
 
-      // Create a new entry that is a copy of the old entry but with the new DN.
-      // Also invoke any subordinate modify DN plugins on the entry.
-      // FIXME -- At the present time, we don't support subordinate modify DN
-      //          plugins that make changes to subordinate entries and therefore
-      //          provide an unmodifiable list for the modifications element.
-      // FIXME -- This will need to be updated appropriately if we decided that
-      //          these plugins should be invoked for synchronization
-      //          operations.
-      if (! modifyDNOperation.isSynchronizationOperation())
+      // Reindex the entry with the new ID.
+      indexRemoveEntry(buffer, oldEntry, oldID);
+      indexInsertEntry(buffer, newEntry, newID);
+    }
+    else
+    {
+      // Update the indexes if needed.
+      indexModifications(buffer, oldEntry, newEntry, oldID,
+          modifyDNOperation.getModifications());
+    }
+
+    // Add the new ID to id2children and id2subtree of new apex parent entry.
+    if(newSuperiorDN != null && isApexEntryMoved)
+    {
+      EntryID parentID;
+      byte[] parentIDKeyBytes;
+      boolean isParent = true;
+      for (DN dn = newSuperiorDN; dn != null; dn = getParentWithinBase(dn))
       {
-        PluginConfigManager pluginManager =
-            DirectoryServer.getPluginConfigManager();
-        PluginResult.SubordinateModifyDN pluginResult =
-            pluginManager.invokeSubordinateModifyDNPlugins(
-                modifyDNOperation, oldEntry, newEntry, modifications);
-
-        if (!pluginResult.continueProcessing())
+        parentID = dn2id.get(txn, dn, LockMode.DEFAULT);
+        parentIDKeyBytes =
+            JebFormat.entryIDToDatabase(parentID.longValue());
+        if(isParent)
         {
-          Message message = ERR_JEB_MODIFYDN_ABORTED_BY_SUBORDINATE_PLUGIN.get(
-              oldDN.toString(), newDN.toString());
+          id2children.insertID(buffer, parentIDKeyBytes, newID);
+          isParent = false;
+        }
+        id2subtree.insertID(buffer, parentIDKeyBytes, newID);
+      }
+    }
+
+    // Remove the entry from the entry cache.
+    EntryCache entryCache = DirectoryServer.getEntryCache();
+    if (entryCache != null)
+    {
+      entryCache.removeEntry(oldDN);
+    }
+  }
+
+  private void renameSubordinateEntry(Transaction txn, IndexBuffer buffer,
+                                      DN oldSuperiorDN, DN newSuperiorDN,
+                                      EntryID oldID, EntryID newID,
+                                      Entry oldEntry, DN newDN,
+                                      boolean isApexEntryMoved,
+                                      ModifyDNOperation modifyDNOperation)
+      throws DirectoryException, DatabaseException
+  {
+    DN oldDN = oldEntry.getDN();
+    Entry newEntry = oldEntry.duplicate(false);
+    newEntry.setDN(newDN);
+    List<Modification> modifications =
+        Collections.unmodifiableList(new ArrayList<Modification>(0));
+
+    // Create a new entry that is a copy of the old entry but with the new DN.
+    // Also invoke any subordinate modify DN plugins on the entry.
+    // FIXME -- At the present time, we don't support subordinate modify DN
+    //          plugins that make changes to subordinate entries and therefore
+    //          provide an unmodifiable list for the modifications element.
+    // FIXME -- This will need to be updated appropriately if we decided that
+    //          these plugins should be invoked for synchronization
+    //          operations.
+    if (! modifyDNOperation.isSynchronizationOperation())
+    {
+      PluginConfigManager pluginManager =
+          DirectoryServer.getPluginConfigManager();
+      PluginResult.SubordinateModifyDN pluginResult =
+          pluginManager.invokeSubordinateModifyDNPlugins(
+              modifyDNOperation, oldEntry, newEntry, modifications);
+
+      if (!pluginResult.continueProcessing())
+      {
+        Message message = ERR_JEB_MODIFYDN_ABORTED_BY_SUBORDINATE_PLUGIN.get(
+            oldDN.toString(), newDN.toString());
+        throw new DirectoryException(
+            DirectoryServer.getServerErrorResultCode(), message);
+      }
+
+      if (! modifications.isEmpty())
+      {
+        MessageBuilder invalidReason = new MessageBuilder();
+        if (! newEntry.conformsToSchema(null, false, false, false,
+            invalidReason))
+        {
+          Message message =
+              ERR_JEB_MODIFYDN_ABORTED_BY_SUBORDINATE_SCHEMA_ERROR.get(
+                  oldDN.toString(),
+                  newDN.toString(),
+                  invalidReason.toString());
           throw new DirectoryException(
               DirectoryServer.getServerErrorResultCode(), message);
         }
+      }
+    }
 
-        if (! modifications.isEmpty())
+    // Remove the old DN from dn2id.
+    dn2id.remove(txn, oldDN);
+
+    // Put the new DN in dn2id.
+    if (!dn2id.insert(txn, newDN, newID))
+    {
+      Message message = ERR_JEB_MODIFYDN_ALREADY_EXISTS.get(newDN.toString());
+      throw new DirectoryException(ResultCode.ENTRY_ALREADY_EXISTS,
+          message);
+    }
+
+    // Remove old ID from id2entry and put the new entry
+    // (old entry with new DN) in id2entry.
+    if (!newID.equals(oldID))
+    {
+      id2entry.remove(txn, oldID);
+    }
+    id2entry.put(txn, newID, newEntry);
+
+    // Update any referral records.
+    dn2uri.replaceEntry(txn, oldEntry, newEntry);
+
+    if(isApexEntryMoved)
+    {
+      // Remove the old ID from id2subtree of old apex superior entries.
+      for (DN dn = oldSuperiorDN; dn != null; dn = getParentWithinBase(dn))
+      {
+        EntryID parentID = dn2id.get(txn, dn, LockMode.DEFAULT);
+        byte[] parentIDKeyBytes =
+            JebFormat.entryIDToDatabase(parentID.longValue());
+        id2subtree.removeID(buffer, parentIDKeyBytes, oldID);
+      }
+    }
+
+    if (!newID.equals(oldID))
+    {
+      // All the subordinates will be renumbered so we have to rebuild
+      // id2c and id2s with the new ID.
+      byte[] oldIDKeyBytes = JebFormat.entryIDToDatabase(oldID.longValue());
+      id2children.delete(buffer, oldIDKeyBytes);
+      id2subtree.delete(buffer, oldIDKeyBytes);
+
+      // Add new ID to the id2c and id2s of our new parent and
+      // new ID to id2s up the tree.
+      EntryID newParentID;
+      byte[] parentIDKeyBytes;
+      boolean isParent = true;
+      for (DN superiorDN = newDN; superiorDN != null;
+           superiorDN = getParentWithinBase(superiorDN))
+      {
+        newParentID = dn2id.get(txn, superiorDN, LockMode.DEFAULT);
+        parentIDKeyBytes =
+            JebFormat.entryIDToDatabase(newParentID.longValue());
+        if(isParent)
         {
-          MessageBuilder invalidReason = new MessageBuilder();
-          if (! newEntry.conformsToSchema(null, false, false, false,
-              invalidReason))
-          {
-            Message message =
-                ERR_JEB_MODIFYDN_ABORTED_BY_SUBORDINATE_SCHEMA_ERROR.get(
-                    oldDN.toString(),
-                    newDN.toString(),
-                    invalidReason.toString());
-            throw new DirectoryException(
-                DirectoryServer.getServerErrorResultCode(), message);
-          }
+          id2children.insertID(buffer, parentIDKeyBytes, newID);
+          isParent = false;
         }
+        id2subtree.insertID(buffer, parentIDKeyBytes, newID);
       }
 
-      // Remove the old DN from dn2id.
-      dn2id.remove(txn, oldDN);
-
-      // Put the new DN in dn2id.
-      if (!dn2id.insert(txn, newDN, newID))
+      // Reindex the entry with the new ID.
+      indexRemoveEntry(buffer, oldEntry, oldID);
+      indexInsertEntry(buffer, newEntry, newID);
+    }
+    else
+    {
+      // Update the indexes if needed.
+      if(! modifications.isEmpty())
       {
-        Message message = ERR_JEB_MODIFYDN_ALREADY_EXISTS.get(newDN.toString());
-        throw new DirectoryException(ResultCode.ENTRY_ALREADY_EXISTS,
-            message);
+        indexModifications(buffer, oldEntry, newEntry, oldID, modifications);
       }
 
-      // Remove old ID from id2entry and put the new entry
-      // (old entry with new DN) in id2entry.
-      if (!newID.equals(oldID))
-      {
-        id2entry.remove(txn, oldID);
-      }
-      id2entry.put(txn, newID, newEntry);
-
-      // Update any referral records.
-      dn2uri.replaceEntry(txn, oldEntry, newEntry);
-
       if(isApexEntryMoved)
       {
-        // Remove the old ID from id2subtree of old apex superior entries.
-        for (DN dn = oldSuperiorDN; dn != null; dn = getParentWithinBase(dn))
+        // Add the new ID to the id2s of new apex superior entries.
+        for(DN dn = newSuperiorDN; dn != null; dn = getParentWithinBase(dn))
         {
           EntryID parentID = dn2id.get(txn, dn, LockMode.DEFAULT);
           byte[] parentIDKeyBytes =
               JebFormat.entryIDToDatabase(parentID.longValue());
-          id2subtree.removeID(buffer, parentIDKeyBytes, oldID);
-        }
-      }
-
-      if (!newID.equals(oldID))
-      {
-        // All the subordinates will be renumbered so we have to rebuild
-        // id2c and id2s with the new ID.
-        byte[] oldIDKeyBytes = JebFormat.entryIDToDatabase(oldID.longValue());
-        id2children.delete(buffer, oldIDKeyBytes);
-        id2subtree.delete(buffer, oldIDKeyBytes);
-
-        // Add new ID to the id2c and id2s of our new parent and
-        // new ID to id2s up the tree.
-        EntryID newParentID;
-        byte[] parentIDKeyBytes;
-        boolean isParent = true;
-        for (DN superiorDN = newDN; superiorDN != null;
-             superiorDN = getParentWithinBase(superiorDN))
-        {
-          newParentID = dn2id.get(txn, superiorDN, LockMode.DEFAULT);
-          parentIDKeyBytes =
-              JebFormat.entryIDToDatabase(newParentID.longValue());
-          if(isParent)
-          {
-            id2children.insertID(buffer, parentIDKeyBytes, newID);
-            isParent = false;
-          }
           id2subtree.insertID(buffer, parentIDKeyBytes, newID);
         }
-
-        // Reindex the entry with the new ID.
-        indexRemoveEntry(buffer, oldEntry, oldID);
-        indexInsertEntry(buffer, newEntry, newID);
-      }
-      else
-      {
-        // Update the indexes if needed.
-        if(! modifications.isEmpty())
-        {
-          indexModifications(buffer, oldEntry, newEntry, oldID, modifications);
-        }
-
-        if(isApexEntryMoved)
-        {
-          // Add the new ID to the id2s of new apex superior entries.
-          for(DN dn = newSuperiorDN; dn != null; dn = getParentWithinBase(dn))
-          {
-            EntryID parentID = dn2id.get(txn, dn, LockMode.DEFAULT);
-            byte[] parentIDKeyBytes =
-                JebFormat.entryIDToDatabase(parentID.longValue());
-            id2subtree.insertID(buffer, parentIDKeyBytes, newID);
-          }
-        }
-      }
-
-      // Remove the entry from the entry cache.
-      EntryCache entryCache = DirectoryServer.getEntryCache();
-      if (entryCache != null)
-      {
-        entryCache.removeEntry(oldDN);
       }
     }
 
-    /**
-     * This method is called after the transaction has successfully
-     * committed.
-     */
-    public void postCommitAction()
+    // Remove the entry from the entry cache.
+    EntryCache entryCache = DirectoryServer.getEntryCache();
+    if (entryCache != null)
     {
-      // No implementation needed.
+      entryCache.removeEntry(oldDN);
     }
   }
 
@@ -3644,11 +3046,10 @@
    * @param entryID The ID of the entry to be inserted into the indexes.
    * @throws DatabaseException If an error occurs in the JE database.
    * @throws DirectoryException If a Directory Server error occurs.
-   * @throws JebException If an error occurs in the JE backend.
    */
   private void indexInsertEntry(IndexBuffer buffer, Entry entry,
                                 EntryID entryID)
-      throws DatabaseException, DirectoryException, JebException
+      throws DatabaseException, DirectoryException
   {
     for (AttributeIndex index : attrIndexMap.values())
     {
@@ -3693,11 +3094,10 @@
    * @param entryID The ID of the entry to be removed from the indexes.
    * @throws DatabaseException If an error occurs in the JE database.
    * @throws DirectoryException If a Directory Server error occurs.
-   * @throws JebException If an error occurs in the JE backend.
    */
   private void indexRemoveEntry(IndexBuffer buffer, Entry entry,
                                 EntryID entryID)
-      throws DatabaseException, DirectoryException, JebException
+      throws DatabaseException, DirectoryException
   {
     for (AttributeIndex index : attrIndexMap.values())
     {
@@ -3778,12 +3178,11 @@
    * @param mods The sequence of modifications made to the entry.
    * @throws DatabaseException If an error occurs in the JE database.
    * @throws DirectoryException If a Directory Server error occurs.
-   * @throws JebException If an error occurs in the JE backend.
    */
   private void indexModifications(IndexBuffer buffer, Entry oldEntry,
                                   Entry newEntry,
                                   EntryID entryID, List<Modification> mods)
-      throws DatabaseException, DirectoryException, JebException
+      throws DatabaseException, DirectoryException
   {
     // Process in index configuration order.
     for (AttributeIndex index : attrIndexMap.values())
@@ -4357,8 +3756,6 @@
     id2entry.setDataConfig(entryDataConfig);
 
     this.config = cfg;
-    this.subtreeDeleteSizeLimit = config.getSubtreeDeleteSizeLimit();
-    this.subtreeDeleteBatchSize = config.getSubtreeDeleteBatchSize();
     return new ConfigChangeResult(ResultCode.SUCCESS,
                                   adminActionRequired, messages);
   }
@@ -4654,15 +4051,4 @@
   public void unlock() {
     exclusiveLock.unlock();
   }
-
-  /**
-   * Get the subtree delete batch size.
-   *
-   * @return The subtree delete batch size.
-   */
-  public int getSubtreeDeleteBatchSize()
-  {
-    return subtreeDeleteBatchSize;
-  }
-
 }

--
Gitblit v1.10.0