mirror of https://github.com/OpenIdentityPlatform/OpenDJ.git

Jean-Noel Rouvignac
04.08.2014 17e216ed32e21eddece22d782f8a256711ab3ad8
OPENDJ-1602 (CR-5566) New pluggable storage based backend

Preparatory work: Converted code using JE Transaction to use IndexBuffer instead.

EntryContainer.java:
Converted code using JE Transaction to use IndexBuffer instead.
Some code cleanup.

VLVIndex.java:
Fixed a bug in updateIndex().
Removed several transaction related methods.

AttributeIndex.java, Index.java, NullIndex.java:
Removed several transaction related methods.
5 files modified
833 ■■■■■ changed files
opendj3-server-dev/src/server/org/opends/server/backends/jeb/AttributeIndex.java 124 ●●●●● patch | view | raw | blame | history
opendj3-server-dev/src/server/org/opends/server/backends/jeb/EntryContainer.java 175 ●●●● patch | view | raw | blame | history
opendj3-server-dev/src/server/org/opends/server/backends/jeb/Index.java 312 ●●●●● patch | view | raw | blame | history
opendj3-server-dev/src/server/org/opends/server/backends/jeb/NullIndex.java 140 ●●●●● patch | view | raw | blame | history
opendj3-server-dev/src/server/org/opends/server/backends/jeb/VLVIndex.java 82 ●●●●● patch | view | raw | blame | history
opendj3-server-dev/src/server/org/opends/server/backends/jeb/AttributeIndex.java
@@ -55,7 +55,6 @@
import com.sleepycat.je.DatabaseEntry;
import com.sleepycat.je.DatabaseException;
import com.sleepycat.je.Environment;
import com.sleepycat.je.Transaction;
import static org.opends.messages.JebMessages.*;
import static org.opends.server.util.ServerConstants.*;
@@ -115,29 +114,22 @@
   * class in the SDK could implement the logic, I hope).
   */
  /**
   * A database key for the presence index.
   */
  public static final DatabaseEntry presenceKey =
       new DatabaseEntry("+".getBytes());
  /** A database key for the presence index. */
  static final DatabaseEntry presenceKey = new DatabaseEntry("+".getBytes());
  /**
   * The entryContainer in which this attribute index resides.
   */
  private EntryContainer entryContainer;
  private Environment env;
  private State state;
  /** The entryContainer in which this attribute index resides. */
  private final EntryContainer entryContainer;
  private final Environment env;
  private final State state;
  /**
   * The attribute index configuration.
   */
  /** The attribute index configuration. */
  private LocalDBIndexCfg indexConfig;
  /** The mapping from names to indexes. */
  private final Map<String, Index> nameToIndexes;
  private final IndexQueryFactory<IndexQuery> indexQueryFactory;
  private int cursorEntryLimit = 100000;
  private final int cursorEntryLimit = 100000;
  /**
   * The mapping from extensible index types (e.g. "substring" or "shared") to list of indexes.
@@ -362,33 +354,6 @@
  }
  /**
   * Update the attribute index for a new entry.
   *
   * @param txn         The database transaction to be used for the insertions.
   * @param entryID     The entry ID.
   * @param entry       The contents of the new entry.
   * @return True if all the index keys for the entry are added. False if the
   *         entry ID already exists for some keys.
   * @throws DatabaseException If an error occurs in the JE database.
   * @throws DirectoryException If a Directory Server error occurs.
   */
  public boolean addEntry(Transaction txn, EntryID entryID, Entry entry)
       throws DatabaseException, DirectoryException
  {
    boolean success = true;
    final IndexingOptions options = indexQueryFactory.getIndexingOptions();
    for (Index index : nameToIndexes.values())
    {
      if (!index.addEntry(txn, entryID, entry, options))
      {
        success = false;
      }
    }
    return success;
  }
  /**
   * Update the attribute index for a deleted entry.
   *
   * @param buffer The index buffer to use to store the deleted keys
@@ -408,51 +373,6 @@
  }
  /**
   * Update the attribute index for a deleted entry.
   *
   * @param txn         The database transaction to be used for the deletions
   * @param entryID     The entry ID
   * @param entry       The contents of the deleted entry.
   * @throws DatabaseException If an error occurs in the JE database.
   * @throws DirectoryException If a Directory Server error occurs.
   */
  public void removeEntry(Transaction txn, EntryID entryID, Entry entry)
       throws DatabaseException, DirectoryException
  {
    final IndexingOptions options = indexQueryFactory.getIndexingOptions();
    for (Index index : nameToIndexes.values())
    {
      index.removeEntry(txn, entryID, entry, options);
    }
  }
  /**
   * Update the index to reflect a sequence of modifications in a Modify
   * operation.
   *
   * @param txn The JE transaction to use for database updates.
   * @param entryID The ID of the entry that was modified.
   * @param oldEntry The entry before the modifications were applied.
   * @param newEntry The entry after the modifications were applied.
   * @param mods The sequence of modifications in the Modify operation.
   * @throws DatabaseException If an error occurs during an operation on a
   * JE database.
   */
  public void modifyEntry(Transaction txn,
                          EntryID entryID,
                          Entry oldEntry,
                          Entry newEntry,
                          List<Modification> mods)
       throws DatabaseException
  {
    final IndexingOptions options = indexQueryFactory.getIndexingOptions();
    for (Index index : nameToIndexes.values())
    {
      index.modifyEntry(txn, entryID, oldEntry, newEntry, mods, options);
    }
  }
  /**
   * Update the index to reflect a sequence of modifications in a Modify
   * operation.
   *
@@ -704,18 +624,6 @@
  }
  /**
   * Close cursors related to the attribute indexes.
   *
   * @throws DatabaseException If a database error occurs.
   */
  public void closeCursors() throws DatabaseException {
    for (Index index : nameToIndexes.values())
    {
      index.closeCursor();
    }
  }
  /**
   * Return the number of values that have exceeded the entry limit since this
   * object was created.
   *
@@ -1025,22 +933,6 @@
  }
  /**
   * Set the index truststate.
   * @param txn A database transaction, or null if none is required.
   * @param trusted True if this index should be trusted or false
   *                otherwise.
   * @throws DatabaseException If an error occurs in the JE database.
   */
  public synchronized void setTrusted(Transaction txn, boolean trusted)
      throws DatabaseException
  {
    for (Index index : nameToIndexes.values())
    {
      index.setTrusted(txn, trusted);
    }
  }
  /**
   * Return true iff this index is trusted.
   * @return the trusted state of this index
   */
opendj3-server-dev/src/server/org/opends/server/backends/jeb/EntryContainer.java
@@ -1512,7 +1512,8 @@
      }
      // Insert into the indexes, in index configuration order.
      indexInsertEntry(txn, entry, entryID);
      final IndexBuffer indexBuffer = new IndexBuffer(this);
      indexInsertEntry(indexBuffer, entry, entryID);
      // Insert into id2children and id2subtree.
      // The database transaction locks on these records will be hotly
@@ -1520,11 +1521,9 @@
      // shortest duration.
      if (parentDN != null)
      {
        // Insert into id2children for parent ID.
        id2children.insertID(txn, parentID.getDatabaseEntry(), entryID);
        // Insert into id2subtree for parent ID.
        id2subtree.insertID(txn, parentID.getDatabaseEntry(), entryID);
        final ByteString parentIDKeyBytes = toByteString(parentID);
        id2children.insertID(indexBuffer, parentIDKeyBytes, entryID);
        id2subtree.insertID(indexBuffer, parentIDKeyBytes, entryID);
        // Iterate up through the superior entries, starting above the parent.
        for (DN dn = getParentWithinBase(parentDN); dn != null;
@@ -1538,9 +1537,10 @@
          }
          // Insert into id2subtree for this node.
          id2subtree.insertID(txn, nodeID.getDatabaseEntry(), entryID);
          id2subtree.insertID(indexBuffer, toByteString(nodeID), entryID);
        }
      }
      indexBuffer.flush(txn);
      if(addOperation != null)
      {
@@ -1588,6 +1588,11 @@
    }
  }
  private ByteString toByteString(EntryID entryID)
  {
    return ByteString.wrap(entryID.getDatabaseEntry().getData());
  }
  /**
   * Removes the specified entry from this database.  This method must ensure
   * that the entry exists and that it does not have any subordinate entries
@@ -1608,7 +1613,7 @@
  throws DirectoryException, DatabaseException, CanceledOperationException
  {
    Transaction txn = beginTransaction();
    IndexBuffer indexBuffer = null;
    final IndexBuffer indexBuffer = new IndexBuffer(this);
    try
    {
@@ -1674,13 +1679,6 @@
                ERR_JEB_DELETE_NOT_ALLOWED_ON_NONLEAF.get(entryDN));
          }
          // This is a subtree delete so create a index buffer
          // if it there isn't one.
          if(indexBuffer == null)
          {
            indexBuffer = new IndexBuffer(EntryContainer.this);
          }
          /*
           * Delete this entry which by now must be a leaf because
           * we have been deleting from the bottom of the tree upwards.
@@ -1732,11 +1730,7 @@
          isSubtreeDelete || isManageDsaITOperation(deleteOperation),
          entryDN, null, null);
      if(indexBuffer != null)
      {
        indexBuffer.flush(txn);
      }
      indexBuffer.flush(txn);
      if(deleteOperation != null)
@@ -1844,28 +1838,12 @@
    }
    // Remove from the indexes, in index config order.
    if(indexBuffer != null)
    {
      indexRemoveEntry(indexBuffer, entry, leafID);
    }
    else
    {
      indexRemoveEntry(txn, entry, leafID);
    }
    indexRemoveEntry(indexBuffer, entry, leafID);
    // Remove the id2c and id2s records for this entry.
    if(indexBuffer != null)
    {
      ByteString leafIDKeyBytes = ByteString.valueOf(leafID.longValue());
      id2children.delete(indexBuffer, leafIDKeyBytes);
      id2subtree.delete(indexBuffer, leafIDKeyBytes);
    }
    else
    {
      DatabaseEntry leafIDKey = leafID.getDatabaseEntry();
      id2children.delete(txn, leafIDKey);
      id2subtree.delete(txn, leafIDKey);
    }
    final ByteString leafIDKeyBytes = ByteString.valueOf(leafID.longValue());
    id2children.delete(indexBuffer, leafIDKeyBytes);
    id2subtree.delete(indexBuffer, leafIDKeyBytes);
    // Iterate up through the superior entries from the target entry.
    boolean isParent = true;
@@ -1879,28 +1857,14 @@
        throw new JebException(ERR_JEB_MISSING_DN2ID_RECORD.get(parentDN));
      }
      if(indexBuffer != null)
      ByteString parentIDBytes = ByteString.valueOf(parentID.longValue());
      // Remove from id2children.
      if (isParent)
      {
        ByteString parentIDBytes = ByteString.valueOf(parentID.longValue());
        // Remove from id2children.
        if (isParent)
        {
          id2children.removeID(indexBuffer, parentIDBytes, leafID);
          isParent = false;
        }
        id2subtree.removeID(indexBuffer, parentIDBytes, leafID);
        id2children.removeID(indexBuffer, parentIDBytes, leafID);
        isParent = false;
      }
      else
      {
        DatabaseEntry nodeIDData = parentID.getDatabaseEntry();
        // Remove from id2children.
        if(isParent)
        {
          id2children.removeID(txn, nodeIDData, leafID);
          isParent = false;
        }
        id2subtree.removeID(txn, nodeIDData, leafID);
      }
      id2subtree.removeID(indexBuffer, parentIDBytes, leafID);
    }
    // Remove the entry from the entry cache.
@@ -2060,19 +2024,22 @@
      id2entry.put(txn, entryID, newEntry);
      // Update the indexes.
      final IndexBuffer indexBuffer = new IndexBuffer(this);
      if (modifyOperation != null)
      {
        // In this case we know from the operation what the modifications were.
        List<Modification> mods = modifyOperation.getModifications();
        indexModifications(txn, oldEntry, newEntry, entryID, mods);
        indexModifications(indexBuffer, oldEntry, newEntry, entryID, mods);
      }
      else
      {
        // The most optimal would be to figure out what the modifications were.
        indexRemoveEntry(txn, oldEntry, entryID);
        indexInsertEntry(txn, newEntry, entryID);
        indexRemoveEntry(indexBuffer, oldEntry, entryID);
        indexInsertEntry(indexBuffer, newEntry, entryID);
      }
      indexBuffer.flush(txn);
      if(modifyOperation != null)
      {
        // One last check before committing
@@ -2655,30 +2622,6 @@
  /**
   * Insert a new entry into the attribute indexes.
   *
   * @param txn The database transaction to be used for the updates.
   * @param entry The entry to be inserted into the indexes.
   * @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(Transaction txn, Entry entry, EntryID entryID)
  throws DatabaseException, DirectoryException, JebException
  {
    for (AttributeIndex index : attrIndexMap.values())
    {
      index.addEntry(txn, entryID, entry);
    }
    for (VLVIndex vlvIndex : vlvIndexMap.values())
    {
      vlvIndex.addEntry(txn, entryID, entry);
    }
  }
  /**
   * Insert a new entry into the attribute indexes.
   *
   * @param buffer The index buffer used to buffer up the index changes.
   * @param entry The entry to be inserted into the indexes.
   * @param entryID The ID of the entry to be inserted into the indexes.
@@ -2702,30 +2645,6 @@
  /**
   * Remove an entry from the attribute indexes.
   *
   * @param txn The database transaction to be used for the updates.
   * @param entry The entry to be removed from the indexes.
   * @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(Transaction txn, Entry entry, EntryID entryID)
  throws DatabaseException, DirectoryException, JebException
  {
    for (AttributeIndex index : attrIndexMap.values())
    {
      index.removeEntry(txn, entryID, entry);
    }
    for (VLVIndex vlvIndex : vlvIndexMap.values())
    {
      vlvIndex.removeEntry(txn, entryID, entry);
    }
  }
  /**
   * Remove an entry from the attribute indexes.
   *
   * @param buffer The index buffer used to buffer up the index changes.
   * @param entry The entry to be removed from the indexes.
   * @param entryID The ID of the entry to be removed from the indexes.
@@ -2750,40 +2669,6 @@
   * Update the attribute indexes to reflect the changes to the
   * attributes of an entry resulting from a sequence of modifications.
   *
   * @param txn The database transaction to be used for the updates.
   * @param oldEntry The contents of the entry before the change.
   * @param newEntry The contents of the entry after the change.
   * @param entryID The ID of the entry that was changed.
   * @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(Transaction txn, Entry oldEntry,
      Entry newEntry,
      EntryID entryID, List<Modification> mods)
  throws DatabaseException, DirectoryException, JebException
  {
    // Process in index configuration order.
    for (AttributeIndex index : attrIndexMap.values())
    {
      // Check whether any modifications apply to this indexed attribute.
      if (isAttributeModified(index, mods))
      {
        index.modifyEntry(txn, entryID, oldEntry, newEntry, mods);
      }
    }
    for(VLVIndex vlvIndex : vlvIndexMap.values())
    {
      vlvIndex.modifyEntry(txn, entryID, oldEntry, newEntry, mods);
    }
  }
  /**
   * Update the attribute indexes to reflect the changes to the
   * attributes of an entry resulting from a sequence of modifications.
   *
   * @param buffer The index buffer used to buffer up the index changes.
   * @param oldEntry The contents of the entry before the change.
   * @param newEntry The contents of the entry after the change.
opendj3-server-dev/src/server/org/opends/server/backends/jeb/Index.java
@@ -198,73 +198,6 @@
    return true;
  }
  /**
   * Insert an entry ID into the set of IDs indexed by a given key.
   *
   * @param txn A database transaction, or null if none is required.
   * @param key         The index key.
   * @param entryID     The entry ID.
   * @return True if the entry ID is inserted or ignored because the entry limit
   *         count is exceeded. False if it already exists in the entry ID set
   *         for the given key.
   * @throws DatabaseException If an error occurs in the JE database.
   */
  public boolean insertID(Transaction txn, DatabaseEntry key, EntryID entryID)
       throws DatabaseException
  {
    DatabaseEntry entryIDData = entryID.getDatabaseEntry();
    DatabaseEntry data = new DatabaseEntry();
    if(maintainCount)
    {
      for (int i = 0; i < phantomWriteRetries; i++)
      {
        if (insertIDWithRMW(txn, key, data, entryIDData, entryID) == SUCCESS)
        {
          return true;
        }
      }
    }
    else
    {
      final OperationStatus status = read(txn, key, data, LockMode.READ_COMMITTED);
      if(status == OperationStatus.SUCCESS)
      {
        EntryIDSet entryIDList = new EntryIDSet(key.getData(), data.getData());
        if (entryIDList.isDefined())
        {
          for (int i = 0; i < phantomWriteRetries; i++)
          {
            if (insertIDWithRMW(txn, key, data, entryIDData, entryID) == SUCCESS)
            {
              return true;
            }
          }
        }
      }
      else if(rebuildRunning || trusted)
      {
        if (insert(txn, key, entryIDData) == OperationStatus.KEYEXIST)
        {
          for (int i = 1; i < phantomWriteRetries; i++)
          {
            if (insertIDWithRMW(txn, key, data, entryIDData, entryID) == SUCCESS)
            {
              return true;
            }
          }
        }
      }
      else
      {
        return true;
      }
    }
    return false;
  }
  private void deleteKey(DatabaseEntry key, ImportIDSet importIdSet, DatabaseEntry data) throws DatabaseException {
    final OperationStatus status = read(null, key, data, LockMode.DEFAULT);
    if(status == SUCCESS) {
@@ -370,56 +303,6 @@
    return true;
  }
  private OperationStatus insertIDWithRMW(Transaction txn, DatabaseEntry key,
                                          DatabaseEntry data,
                                          DatabaseEntry entryIDData,
                                          EntryID entryID)
      throws DatabaseException
  {
    final OperationStatus status = read(txn, key, data, LockMode.RMW);
    if(status == OperationStatus.SUCCESS)
    {
      EntryIDSet entryIDList =
          new EntryIDSet(key.getData(), data.getData());
      if (entryIDList.isDefined() && indexEntryLimit > 0 &&
          entryIDList.size() >= indexEntryLimit)
      {
        if(maintainCount)
        {
          entryIDList = new EntryIDSet(entryIDList.size());
        }
        else
        {
          entryIDList = new EntryIDSet();
        }
        entryLimitExceededCount++;
        if(logger.isTraceEnabled())
        {
          StringBuilder builder = new StringBuilder();
          StaticUtils.byteArrayToHexPlusAscii(builder, key.getData(), 4);
          logger.trace("Index entry exceeded in index %s. " +
              "Limit: %d. ID list size: %d.\nKey:%s",
              name, indexEntryLimit, entryIDList.size(), builder);
        }
      }
      entryIDList.add(entryID);
      byte[] after = entryIDList.toDatabase();
      data.setData(after);
      return put(txn, key, data);
    }
    else if(rebuildRunning || trusted)
    {
      return insert(txn, key, entryIDData);
    }
    else
    {
      return OperationStatus.SUCCESS;
    }
  }
  /**
   * Update the set of entry IDs for a given key.
   *
@@ -633,111 +516,13 @@
    return true;
  }
  /**
   * Remove an entry ID from the set of IDs indexed by a given key.
   *
   * @param txn A database transaction, or null if none is required.
   * @param key         The index key.
   * @param entryID     The entry ID.
   * @throws DatabaseException If an error occurs in the JE database.
   */
  public void removeID(Transaction txn, DatabaseEntry key, EntryID entryID)
      throws DatabaseException
  {
    DatabaseEntry data = new DatabaseEntry();
    if(maintainCount)
    {
      removeIDWithRMW(txn, key, data, entryID);
    }
    else
    {
      final OperationStatus status = read(txn, key, data, LockMode.READ_COMMITTED);
      if(status == SUCCESS)
      {
        EntryIDSet entryIDList = new EntryIDSet(key.getData(), data.getData());
        if(entryIDList.isDefined())
        {
          removeIDWithRMW(txn, key, data, entryID);
        }
      }
      else if (trusted && !rebuildRunning)
      {
        logIndexCorruptError(txn, key);
      }
      // Ignore failures if rebuild is running since a empty entryIDset
      // will probably not be rebuilt.
    }
  }
  /**
   * Delete specified entry ID from all keys in the provided key set.
   *
   * @param txn  A Transaction.
   * @param keySet A set of keys.
   * @param entryID The entry ID to delete.
   * @throws DatabaseException If a database error occurs.
   */
  public void delete(Transaction txn, Set<byte[]> keySet, EntryID entryID) throws DatabaseException {
    setTrusted(txn, false);
    for(byte[] key : keySet) {
       removeIDWithRMW(txn, new DatabaseEntry(key), new DatabaseEntry(), entryID);
    }
    setTrusted(txn, true);
  }
  private void removeIDWithRMW(Transaction txn, DatabaseEntry key,
                               DatabaseEntry data, EntryID entryID)
      throws DatabaseException
  {
    final OperationStatus status = read(txn, key, data, LockMode.RMW);
    if (status == SUCCESS)
    {
      EntryIDSet entryIDList = new EntryIDSet(key.getData(), data.getData());
      // Ignore failures if rebuild is running since the entry ID is
      // probably already removed.
      if (!entryIDList.remove(entryID) && trusted && !rebuildRunning)
      {
        logIndexCorruptError(txn, key,
            "The expected entry ID does not exist in the entry ID list for index %s.\nKey:%s");
      }
      else
      {
        byte[] after = entryIDList.toDatabase();
        if (after != null)
        {
          data.setData(after);
          put(txn, key, data);
        }
        else
        {
          // No more IDs, so remove the key. If index is not
          // trusted then this will cause all subsequent reads
          // for this key to return undefined set.
          delete(txn, key);
        }
      }
    }
    else if (trusted && !rebuildRunning)
    {
      logIndexCorruptError(txn, key);
    }
    // Ignore failures if rebuild is running since a empty entryIDset
    // will probably not be rebuilt.
  }
  private void logIndexCorruptError(Transaction txn, DatabaseEntry key)
  {
    logIndexCorruptError(txn, key, "The expected key does not exist in the index %s.\nKey:%s");
  }
  private void logIndexCorruptError(Transaction txn, DatabaseEntry key, String traceMsg)
  {
    if (logger.isTraceEnabled())
    {
      StringBuilder builder = new StringBuilder();
      StaticUtils.byteArrayToHexPlusAscii(builder, key.getData(), 4);
      logger.trace(traceMsg, name, builder);
      logger.trace("The expected key does not exist in the index %s.\nKey:%s", name, builder);
    }
    setTrusted(txn, false);
@@ -1060,37 +845,6 @@
  }
  /**
   * Update the index for a new entry.
   *
   * @param txn A database transaction, or null if none is required.
   * @param entryID     The entry ID.
   * @param entry       The entry to be indexed.
   * @param options     The indexing options to use
   * @return True if all the indexType keys for the entry are added. False if
   *         the entry ID already exists for some keys.
   * @throws DatabaseException If an error occurs in the JE database.
   * @throws DirectoryException If a Directory Server error occurs.
   */
  public boolean addEntry(Transaction txn, EntryID entryID, Entry entry,
      IndexingOptions options) throws DatabaseException, DirectoryException
  {
    TreeSet<ByteString> addKeys = new TreeSet<ByteString>(indexer.getBSComparator());
    indexer.indexEntry(entry, addKeys, options);
    DatabaseEntry key = new DatabaseEntry();
    boolean success = true;
    for (ByteString keyBytes : addKeys)
    {
      key.setData(keyBytes.toByteArray());
      if(!insertID(txn, key, entryID))
      {
        success = false;
      }
    }
    return success;
  }
  /**
   * Update the index buffer for a deleted entry.
   *
   * @param buffer The index buffer to use to store the deleted keys
@@ -1113,70 +867,6 @@
  }
  /**
   * Update the index for a deleted entry.
   *
   * @param txn A database transaction, or null if none is required.
   * @param entryID     The entry ID
   * @param entry       The contents of the deleted entry.
   * @param options     The indexing options to use
   * @throws DatabaseException If an error occurs in the JE database.
   * @throws DirectoryException If a Directory Server error occurs.
   */
  public void removeEntry(Transaction txn, EntryID entryID, Entry entry,
      IndexingOptions options) throws DatabaseException, DirectoryException
  {
    TreeSet<ByteString> delKeys =
        new TreeSet<ByteString>(indexer.getBSComparator());
    indexer.indexEntry(entry, delKeys, options);
    DatabaseEntry key = new DatabaseEntry();
    for (ByteString keyBytes : delKeys)
    {
      key.setData(keyBytes.toByteArray());
      removeID(txn, key, entryID);
    }
  }
  /**
   * Update the index to reflect a sequence of modifications in a Modify
   * operation.
   *
   * @param txn A database transaction, or null if none is required.
   * @param entryID The ID of the entry that was modified.
   * @param oldEntry The entry before the modifications were applied.
   * @param newEntry The entry after the modifications were applied.
   * @param mods The sequence of modifications in the Modify operation.
   * @param options The indexing options to use
   * @throws DatabaseException If an error occurs in the JE database.
   */
  public void modifyEntry(Transaction txn,
                          EntryID entryID,
                          Entry oldEntry,
                          Entry newEntry,
                          List<Modification> mods, IndexingOptions options)
       throws DatabaseException
  {
    TreeMap<ByteString, Boolean> modifiedKeys =
        new TreeMap<ByteString, Boolean>(indexer.getBSComparator());
    indexer.modifyEntry(oldEntry, newEntry, mods, modifiedKeys, options);
    DatabaseEntry key = new DatabaseEntry();
    for (Map.Entry<ByteString, Boolean> modifiedKey : modifiedKeys.entrySet())
    {
      key.setData(modifiedKey.getKey().toByteArray());
      if(modifiedKey.getValue())
      {
        insertID(txn, key, entryID);
      }
      else
      {
        removeID(txn, key, entryID);
      }
    }
  }
  /**
   * Update the index to reflect a sequence of modifications in a Modify
   * operation.
   *
opendj3-server-dev/src/server/org/opends/server/backends/jeb/NullIndex.java
@@ -67,28 +67,6 @@
    super(name, indexer, state, 0, 0, false, env, entryContainer);
  }
  /** {@inheritDoc} */
  @Override
  public boolean insertID(IndexBuffer buffer, ByteString keyBytes,
      EntryID entryID)
  {
    return true;
  }
  /** {@inheritDoc} */
  @Override
  public boolean insertID(Transaction txn, DatabaseEntry key, EntryID entryID)
      throws DatabaseException
  {
    return true;
  }
  /** {@inheritDoc} */
  @Override
  public void insert(DatabaseEntry key, ImportIDSet importIdSet,
@@ -97,8 +75,6 @@
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  public void delete(DatabaseEntry key, ImportIDSet importIdSet,
@@ -107,8 +83,6 @@
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  public boolean insert(ImportIDSet importIDSet,
@@ -118,8 +92,6 @@
    return true;
  }
  /** {@inheritDoc} */
  @Override
  void updateKey(Transaction txn, DatabaseEntry key, EntryIDSet deletedIDs,
@@ -128,38 +100,6 @@
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  public boolean removeID(IndexBuffer buffer, ByteString keyBytes,
      EntryID entryID)
  {
    return true;
  }
  /** {@inheritDoc} */
  @Override
  public void removeID(Transaction txn, DatabaseEntry key, EntryID entryID)
      throws DatabaseException
  {
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  public void delete(Transaction txn, Set<byte[]> keySet, EntryID entryID)
      throws DatabaseException
  {
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  public void delete(IndexBuffer buffer, ByteString keyBytes)
@@ -167,8 +107,6 @@
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  public ConditionResult containsID(Transaction txn, DatabaseEntry key,
@@ -177,8 +115,6 @@
    return ConditionResult.UNDEFINED;
  }
  /** {@inheritDoc} */
  @Override
  public EntryIDSet readKey(DatabaseEntry key, Transaction txn,
@@ -187,8 +123,6 @@
    return new EntryIDSet();
  }
  /** {@inheritDoc} */
  @Override
  public void writeKey(Transaction txn, DatabaseEntry key,
@@ -197,8 +131,6 @@
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  public EntryIDSet readRange(byte[] lower, byte[] upper,
@@ -207,8 +139,6 @@
    return new EntryIDSet();
  }
  /** {@inheritDoc} */
  @Override
  public int getEntryLimitExceededCount()
@@ -216,8 +146,6 @@
    return 0;
  }
  /** {@inheritDoc} */
  @Override
  public void closeCursor() throws DatabaseException
@@ -225,8 +153,6 @@
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  public boolean addEntry(IndexBuffer buffer, EntryID entryID, Entry entry, IndexingOptions options)
@@ -235,18 +161,6 @@
    return true;
  }
  /** {@inheritDoc} */
  @Override
  public boolean addEntry(Transaction txn, EntryID entryID, Entry entry, IndexingOptions options)
      throws DatabaseException, DirectoryException
  {
    return true;
  }
  /** {@inheritDoc} */
  @Override
  public void removeEntry(IndexBuffer buffer, EntryID entryID, Entry entry, IndexingOptions options)
@@ -255,28 +169,6 @@
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  public void removeEntry(Transaction txn, EntryID entryID, Entry entry, IndexingOptions options)
      throws DatabaseException, DirectoryException
  {
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  public void modifyEntry(Transaction txn, EntryID entryID, Entry oldEntry,
      Entry newEntry, List<Modification> mods, IndexingOptions options) throws DatabaseException
  {
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  public void modifyEntry(IndexBuffer buffer, EntryID entryID, Entry oldEntry,
@@ -285,8 +177,6 @@
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  public boolean setIndexEntryLimit(int indexEntryLimit)
@@ -294,8 +184,6 @@
    return false;
  }
  /** {@inheritDoc} */
  @Override
  public int getIndexEntryLimit()
@@ -303,8 +191,6 @@
    return 0;
  }
  /** {@inheritDoc} */
  @Override
  public void setTrusted(Transaction txn, boolean trusted)
@@ -313,8 +199,6 @@
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  public boolean isTrusted()
@@ -322,8 +206,6 @@
    return true;
  }
  /** {@inheritDoc} */
  @Override
  public boolean isRebuildRunning()
@@ -331,8 +213,6 @@
    return false;
  }
  /** {@inheritDoc} */
  @Override
  public void setRebuildStatus(boolean rebuildRunning)
@@ -340,8 +220,6 @@
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  public boolean getMaintainCount()
@@ -349,8 +227,6 @@
    return false;
  }
  /** {@inheritDoc} */
  @Override
  public void open() throws DatabaseException
@@ -358,8 +234,6 @@
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  public void close() throws DatabaseException
@@ -367,8 +241,6 @@
    // Do nothing.
  }
  /** {@inheritDoc} */
  @Override
  protected OperationStatus put(Transaction txn, DatabaseEntry key,
@@ -377,8 +249,6 @@
    return OperationStatus.SUCCESS;
  }
  /** {@inheritDoc} */
  @Override
  protected OperationStatus read(Transaction txn, DatabaseEntry key,
@@ -387,8 +257,6 @@
    return OperationStatus.SUCCESS;
  }
  /** {@inheritDoc} */
  @Override
  protected OperationStatus insert(Transaction txn, DatabaseEntry key,
@@ -397,8 +265,6 @@
    return OperationStatus.SUCCESS;
  }
  /** {@inheritDoc} */
  @Override
  protected OperationStatus delete(Transaction txn, DatabaseEntry key)
@@ -407,8 +273,6 @@
    return OperationStatus.SUCCESS;
  }
  /** {@inheritDoc} */
  @Override
  public Cursor openCursor(Transaction txn, CursorConfig cursorConfig)
@@ -417,8 +281,6 @@
    throw new IllegalStateException();
  }
  /** {@inheritDoc} */
  @Override
  public long getRecordCount() throws DatabaseException
@@ -426,8 +288,6 @@
    return 0;
  }
  /** {@inheritDoc} */
  @Override
  public PreloadStats preload(PreloadConfig config) throws DatabaseException
opendj3-server-dev/src/server/org/opends/server/backends/jeb/VLVIndex.java
@@ -320,26 +320,6 @@
    return false;
  }
  /**
   * Update the vlvIndex for a deleted entry.
   *
   * @param txn         The database transaction to be used for the deletions
   * @param entryID     The entry ID
   * @param entry       The contents of the deleted entry.
   * @return True if the entry was successfully removed from this VLV index
   * or False otherwise.
   * @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 boolean removeEntry(Transaction txn, EntryID entryID, Entry entry)
      throws DatabaseException, DirectoryException, JebException
  {
    return shouldInclude(entry)
        && removeValues(txn, entryID.longValue(), entry);
  }
  /**
   * Update the vlvIndex for a deleted entry.
   *
@@ -377,66 +357,6 @@
   * Update the vlvIndex to reflect a sequence of modifications in a Modify
   * operation.
   *
   * @param txn The JE transaction to use for database updates.
   * @param entryID The ID of the entry that was modified.
   * @param oldEntry The entry before the modifications were applied.
   * @param newEntry The entry after the modifications were applied.
   * @param mods The sequence of modifications in the Modify operation.
   * @return True if the modification was successfully processed or False
   * otherwise.
   * @throws JebException If an error occurs during an operation on a
   * JE database.
   * @throws DatabaseException If an error occurs during an operation on a
   * JE database.
   * @throws DirectoryException If a Directory Server error occurs.
   */
  public boolean modifyEntry(Transaction txn,
                          EntryID entryID,
                          Entry oldEntry,
                          Entry newEntry,
                          List<Modification> mods)
       throws DatabaseException, DirectoryException, JebException
  {
    if (shouldInclude(oldEntry))
    {
      if (shouldInclude(newEntry))
      {
        // The entry should still be indexed. See if any sorted attributes are
        // changed.
        if (isSortAttributeModified(mods))
        {
          boolean success;
          // Sorted attributes have changed. Reindex the entry;
          success = removeValues(txn, entryID.longValue(), oldEntry);
          success &= insertValues(txn, entryID.longValue(), newEntry);
          return success;
        }
      }
      else
      {
        // The modifications caused the new entry to be unindexed. Remove from
        // vlvIndex.
        return removeValues(txn, entryID.longValue(), oldEntry);
      }
    }
    else
    {
      if (shouldInclude(newEntry))
      {
        // The modifications caused the new entry to be indexed. Add to
        // vlvIndex.
        return insertValues(txn, entryID.longValue(), newEntry);
      }
    }
    // The modifications does not affect this vlvIndex
    return true;
  }
  /**
   * Update the vlvIndex to reflect a sequence of modifications in a Modify
   * operation.
   *
   * @param buffer The database transaction to be used for the deletions
   * @param entryID The ID of the entry that was modified.
   * @param oldEntry The entry before the modifications were applied.
@@ -812,7 +732,7 @@
        break;
      }
      final SortValuesSet sortValuesSet = getSortValuesSet(txn, data, data, LockMode.RMW);
      final SortValuesSet sortValuesSet = getSortValuesSet(txn, key, data, LockMode.RMW);
      int oldSize = sortValuesSet.size();
      if(key.getData().length == 0)
      {