You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@directory.apache.org by ak...@apache.org on 2008/03/30 09:19:33 UTC

svn commit: r642692 - /directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/

Author: akarasulu
Date: Sun Mar 30 00:19:32 2008
New Revision: 642692

URL: http://svn.apache.org/viewvc?rev=642692&view=rev
Log:
Working on ScopeNode based Evaluators and Cursors ...

 o Added new OneLevelScopeCursor, and SubtreeScopeCursor with respective 
   Evaluators: thus we split up handling of scope across these classes
 o minor edits for Rev tag in AndCursor
 o Added code to CursorBuilder and EvaluatorBuilder for creating the respective
   enities
 

Added:
    directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/OneLevelScopeCursor.java
    directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/OneLevelScopeEvaluator.java
      - copied, changed from r642683, directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/ScopeEvaluator.java
    directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/SubtreeScopeCursor.java
    directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/SubtreeScopeEvaluator.java   (contents, props changed)
      - copied, changed from r642683, directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/ScopeEvaluator.java
Removed:
    directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/ScopeEvaluator.java
Modified:
    directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/AndCursor.java
    directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/CursorBuilder.java
    directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/EvaluatorBuilder.java

Modified: directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/AndCursor.java
URL: http://svn.apache.org/viewvc/directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/AndCursor.java?rev=642692&r1=642691&r2=642692&view=diff
==============================================================================
--- directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/AndCursor.java (original)
+++ directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/AndCursor.java Sun Mar 30 00:19:32 2008
@@ -34,7 +34,7 @@
  * A Cursor returning candidates satisfying a logical conjunction expression.
  *
  * @author <a href="mailto:dev@directory.apache.org">Apache Directory Project</a>
- * @version $$Rev$$
+ * @version $Rev$
  */
 public class AndCursor extends AbstractCursor<IndexEntry<?,Attributes>>
 {

Modified: directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/CursorBuilder.java
URL: http://svn.apache.org/viewvc/directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/CursorBuilder.java?rev=642692&r1=642691&r2=642692&view=diff
==============================================================================
--- directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/CursorBuilder.java (original)
+++ directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/CursorBuilder.java Sun Mar 30 00:19:32 2008
@@ -24,15 +24,13 @@
 import java.util.ArrayList;
 
 import javax.naming.directory.Attributes;
+import javax.naming.directory.SearchControls;
 
 import org.apache.directory.server.xdbm.IndexEntry;
 import org.apache.directory.server.xdbm.Store;
 import org.apache.directory.server.core.cursor.Cursor;
 import org.apache.directory.shared.ldap.NotImplementedException;
-import org.apache.directory.shared.ldap.filter.AndNode;
-import org.apache.directory.shared.ldap.filter.ExprNode;
-import org.apache.directory.shared.ldap.filter.NotNode;
-import org.apache.directory.shared.ldap.filter.OrNode;
+import org.apache.directory.shared.ldap.filter.*;
 
 
 /**
@@ -80,7 +78,14 @@
             case PRESENCE:
                 return new PresenceCursor( db, ( PresenceEvaluator ) evaluatorBuilder.build( node ) );
             case SCOPE:
-                throw new NotImplementedException();
+                if ( ( ( ScopeNode ) node ).getScope() == SearchControls.ONELEVEL_SCOPE )
+                {
+                    return new OneLevelScopeCursor( db, ( OneLevelScopeEvaluator ) evaluatorBuilder.build( node ) );
+                }
+                else
+                {
+                    return new SubtreeScopeCursor( db, ( SubtreeScopeEvaluator ) evaluatorBuilder.build( node ) );
+                }
             case SUBSTRING:
                 return new SubstringCursor( db, ( SubstringEvaluator ) evaluatorBuilder.build( node ) );
 

Modified: directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/EvaluatorBuilder.java
URL: http://svn.apache.org/viewvc/directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/EvaluatorBuilder.java?rev=642692&r1=642691&r2=642692&view=diff
==============================================================================
--- directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/EvaluatorBuilder.java (original)
+++ directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/EvaluatorBuilder.java Sun Mar 30 00:19:32 2008
@@ -21,6 +21,7 @@
 
 
 import javax.naming.directory.Attributes;
+import javax.naming.directory.SearchControls;
 
 import org.apache.directory.server.schema.registries.Registries;
 import org.apache.directory.server.xdbm.Store;
@@ -43,11 +44,6 @@
     private final Registries registries;
 
 
-    // ------------------------------------------------------------------------
-    // C O N S T R U C T O R S
-    // ------------------------------------------------------------------------
-
-
     /**
      * Creates a top level Evaluator where leaves are delegated to a leaf node
      * evaluator which will be created.
@@ -67,7 +63,7 @@
     {
         switch ( node.getAssertionType() )
         {
-                /* ---------- LEAF NODE HANDLING ---------- */
+            /* ---------- LEAF NODE HANDLING ---------- */
 
             case APPROXIMATE:
                 return new ApproximateEvaluator( ( ApproximateNode ) node, db, registries );
@@ -80,12 +76,18 @@
             case PRESENCE:
                 return new PresenceEvaluator( ( PresenceNode ) node, db, registries );
             case SCOPE:
-//                return new ScopeEvaluator( ( ScopeNode ) node, db, registries );
-                throw new NotImplementedException( "SOON!!!!!" );
+                if ( ( ( ScopeNode ) node ).getScope() == SearchControls.ONELEVEL_SCOPE )
+                {
+                    return new OneLevelScopeEvaluator<Attributes>( db, ( ScopeNode ) node );
+                }
+                else
+                {
+                    return new SubtreeScopeEvaluator<Attributes>( db, ( ScopeNode ) node );
+                }
             case SUBSTRING:
                 return new SubstringEvaluator( ( SubstringNode ) node, db, registries );
 
-                /* ---------- LOGICAL OPERATORS ---------- */
+            /* ---------- LOGICAL OPERATORS ---------- */
 
             case AND:
                 return buildAndEvaluator( ( AndNode ) node );
@@ -94,7 +96,7 @@
             case OR:
                 return buildOrEvaluator( ( OrNode ) node );
 
-                /* ----------  NOT IMPLEMENTED  ---------- */
+            /* ----------  NOT IMPLEMENTED  ---------- */
 
             case ASSERTION:
             case EXTENSIBLE:

Added: directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/OneLevelScopeCursor.java
URL: http://svn.apache.org/viewvc/directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/OneLevelScopeCursor.java?rev=642692&view=auto
==============================================================================
--- directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/OneLevelScopeCursor.java (added)
+++ directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/OneLevelScopeCursor.java Sun Mar 30 00:19:32 2008
@@ -0,0 +1,227 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *  
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *  
+ *  Unless required by applicable law or agreed to in writing,
+ *  software distributed under the License is distributed on an
+ *  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ *  specific language governing permissions and limitations
+ *  under the License. 
+ *  
+ */
+package org.apache.directory.server.xdbm.search.impl;
+
+
+import org.apache.directory.server.core.cursor.AbstractCursor;
+import org.apache.directory.server.core.cursor.Cursor;
+import org.apache.directory.server.core.cursor.InvalidCursorPositionException;
+import org.apache.directory.server.xdbm.IndexEntry;
+import org.apache.directory.server.xdbm.Store;
+
+import javax.naming.directory.Attributes;
+
+
+/**
+ * A Cursor over entries satisfying one level scope constraints with alias
+ * dereferencing considerations when enabled during search.
+ *
+ * @author <a href="mailto:dev@directory.apache.org">Apache Directory Project</a>
+ * @version $Rev$
+ */
+public class OneLevelScopeCursor extends AbstractCursor<IndexEntry<?, Attributes>>
+{
+    private static final String UNSUPPORTED_MSG =
+        "Scope Cursors are not ordered and do not support positioning by element.";
+
+    /** A onelevel ScopeNode Evaluator */
+    private final OneLevelScopeEvaluator evaluator;
+
+    /** A Cursor over the entries in the scope of the search base */
+    private final Cursor<IndexEntry<Long,Attributes>> scopeCursor;
+
+    /** A Cursor over entries brought into scope by alias dereferencing */
+    private final Cursor<IndexEntry<Long,Attributes>> dereferencedCursor;
+
+    /** Currently active Cursor: we switch between two cursors */
+    private Cursor<IndexEntry<Long,Attributes>> cursor;
+
+    /** Whether or not this Cursor is positioned so an entry is available */
+    private boolean available = false;
+
+
+    /**
+     * Creates a Cursor over entries satisfying one level scope criteria.
+     *
+     * @param db the entry store
+     * @param evaluator an IndexEntry (candidate) evaluator
+     * @throws Exception on db access failures
+     */
+    public OneLevelScopeCursor( Store<Attributes> db, OneLevelScopeEvaluator evaluator ) throws Exception
+    {
+        this.evaluator = evaluator;
+        scopeCursor = db.getOneLevelIndex().forwardCursor( evaluator.getBaseId() );
+
+        if ( evaluator.isDereferencing() )
+        {
+            dereferencedCursor = db.getOneAliasIndex().forwardCursor( evaluator.getBaseId() );
+        }
+        else
+        {
+            dereferencedCursor = null;
+        }
+    }
+
+
+    public boolean available()
+    {
+        return available;
+    }
+
+
+    public void before( IndexEntry<?, Attributes> element ) throws Exception
+    {
+        throw new UnsupportedOperationException( UNSUPPORTED_MSG );
+    }
+
+
+    public void after( IndexEntry<?, Attributes> element ) throws Exception
+    {
+        throw new UnsupportedOperationException( UNSUPPORTED_MSG );
+    }
+
+
+    public void beforeFirst() throws Exception
+    {
+        cursor = scopeCursor;
+        cursor.beforeFirst();
+        available = false;
+    }
+
+
+    public void afterLast() throws Exception
+    {
+        if ( evaluator.isDereferencing() )
+        {
+            cursor = dereferencedCursor;
+        }
+        else
+        {
+            cursor = scopeCursor;
+        }
+
+        cursor.afterLast();
+        available = false;
+    }
+
+
+    public boolean first() throws Exception
+    {
+        beforeFirst();
+        return next();
+    }
+
+
+    public boolean last() throws Exception
+    {
+        afterLast();
+        return previous();
+    }
+
+
+    public boolean previous() throws Exception
+    {
+        // if the cursor has not been set - position it after last element
+        if ( cursor == null )
+        {
+            afterLast();
+        }
+
+        available = cursor.previous();
+
+        // if we're using the scopeCursor (1st Cursor) then return result as is
+        if ( cursor == scopeCursor )
+        {
+            return available;
+        }
+
+        /*
+         * Below here we are using the dereferencedCursor so if nothing is
+         * available after an advance backwards we need to switch to the
+         * scopeCursor and try a previous call after positioning past it's 
+         * last element.
+         */
+        if ( ! available )
+        {
+            cursor = scopeCursor;
+            cursor.afterLast();
+            return available = cursor.previous();
+        }
+
+        return true;
+    }
+
+
+    public boolean next() throws Exception
+    {
+        // if the cursor hasn't been set position it before the first element
+        if ( cursor == null )
+        {
+            beforeFirst();
+            return scopeCursor.next();
+        }
+
+        available = cursor.next();
+
+        // if we're using dereferencedCursor (2nd) then we return the result
+        if ( cursor == dereferencedCursor )
+        {
+            return available;
+        }
+
+        /*
+         * Below here we are using the scopeCursor so if nothing is
+         * available after an advance forward we need to switch to the
+         * dereferencedCursor and try a previous call after positioning past
+         * it's last element.
+         */
+        if ( ! available )
+        {
+            if ( dereferencedCursor != null )
+            {
+                cursor = dereferencedCursor;
+                cursor.beforeFirst();
+                return available = cursor.next();
+            }
+
+            return false;
+        }
+
+        return true;
+    }
+
+
+    public IndexEntry<Long, Attributes> get() throws Exception
+    {
+        if ( available )
+        {
+            return cursor.get();
+        }
+
+        throw new InvalidCursorPositionException( "Cursor has not been positioned yet." );
+    }
+
+
+    public boolean isElementReused()
+    {
+        return scopeCursor.isElementReused() ||
+            ( dereferencedCursor != null && dereferencedCursor.isElementReused() );
+    }
+}
\ No newline at end of file

Copied: directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/OneLevelScopeEvaluator.java (from r642683, directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/ScopeEvaluator.java)
URL: http://svn.apache.org/viewvc/directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/OneLevelScopeEvaluator.java?p2=directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/OneLevelScopeEvaluator.java&p1=directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/ScopeEvaluator.java&r1=642683&r2=642692&rev=642692&view=diff
==============================================================================
--- directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/ScopeEvaluator.java (original)
+++ directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/OneLevelScopeEvaluator.java Sun Mar 30 00:19:32 2008
@@ -25,88 +25,78 @@
 import org.apache.directory.shared.ldap.filter.ScopeNode;
 import org.apache.directory.server.xdbm.IndexEntry;
 import org.apache.directory.server.xdbm.Store;
+import org.apache.directory.server.xdbm.Index;
 
 
 /**
- * Evaluates ScopeNode assertions on candidates using an entry database.
+ * Evaluates one level scope assertions on candidates using an entry database.
  * 
  * @author <a href="mailto:dev@directory.apache.org">Apache Directory Project</a>
  * @version $Rev$
  */
-public class ScopeEvaluator<E> implements Evaluator<ScopeNode,E>
+public class OneLevelScopeEvaluator<E> implements Evaluator<ScopeNode,E>
 {
-    /** Database used to evaluate scope with */
-    private final Store<E> db;
-
     /** The ScopeNode containing initial search scope constraints */
     private final ScopeNode node;
 
     /** The entry identifier of the scope base */
     private final Long baseId;
 
+    /** True if the scope requires alias dereferencing while searching */
+    private final boolean dereferencing;
+
+    /** The one level alias index used for scope expansion */
+    private final Index<Long,E> aliasIndex;
+
+    /** The one level scope index for parent-child mappings */
+    private final Index<Long,E> scopeIndex;
+
 
     /**
-     * Creates a scope node evaluator for search expressions.
+     * Creates a one level scope node Evaluator for search expressions.
      *
      * @param node the scope node
      * @param db the database used to evaluate scope node
      * @throws Exception on db access failure
      */
-    public ScopeEvaluator( Store<E> db, ScopeNode node ) throws Exception
+    public OneLevelScopeEvaluator( Store<E> db, ScopeNode node ) throws Exception
     {
-        this.db = db;
         this.node = node;
 
-        baseId = db.getEntryId( node.getBaseDn() );
-    }
-
-
-    /**
-     * @see Evaluator#evaluate(org.apache.directory.server.xdbm.IndexEntry)
-     */
-    public boolean evaluate( IndexEntry<?,E> entry ) throws Exception
-    {
-        switch ( node.getScope() )
+        if ( node.getScope() != SearchControls.ONELEVEL_SCOPE )
         {
-            case ( SearchControls.OBJECT_SCOPE  ):
-                return entry.getId().longValue() == baseId.longValue();
-            case ( SearchControls.ONELEVEL_SCOPE  ):
-                return isInOneLevelScope( entry.getId() );
-            case ( SearchControls.SUBTREE_SCOPE  ):
-                return isInSubtreeScope( entry.getId() );
-            default:
-                throw new IllegalStateException( "Unrecognized search scope!" );
+            throw new IllegalStateException( "ScopeNode is not of onelevel scope." );
         }
-    }
 
-
-    public ScopeNode getExpression()
-    {
-        return node;
+        baseId = db.getEntryId( node.getBaseDn() );
+        scopeIndex = db.getOneLevelIndex();
+        dereferencing = node.getDerefAliases().isDerefInSearching() ||
+            node.getDerefAliases().isDerefAlways();
+        aliasIndex = db.getOneAliasIndex();
     }
 
 
     /**
      * Asserts whether or not a candidate has one level scope while taking
      * alias dereferencing into account.
-     * 
-     * @param id the candidate to assert which can be any db entry's id
-     * @return true if the candidate is within one level scope whether or not
-     * alias dereferencing is enabled.
-     * @throws Exception if the index lookups fail.
+     *
+     * @param candidate the candidate to assert
+     * @return true if the candidate is within one level scope
+     * @throws Exception if db lookups fail
+     * @see Evaluator#evaluate(IndexEntry)
      */
-    public boolean isInSubtreeScope( final Long id ) throws Exception
+    public boolean evaluate( IndexEntry<?,E> candidate ) throws Exception
     {
-        boolean isDescendant = db.getSubLevelIndex().has( baseId, id );
+        boolean isChild = scopeIndex.has( baseId, candidate.getId() );
 
         /*
-         * The candidate id could be any entry in the db.  If search 
-         * dereferencing is not enabled then we return the results of the 
-         * descendant test.
+         * The candidate id could be any entry in the db.  If search
+         * dereferencing is not enabled then we return the results of the child
+         * test.
          */
-        if ( !node.getDerefAliases().isDerefInSearching() )
+        if ( ! dereferencing )
         {
-            return isDescendant;
+            return isChild;
         }
 
         /*
@@ -114,89 +104,48 @@
          * candidate id is an alias, if so we reject it since aliases should
          * not be returned.
          */
-        if ( null != db.getAliasIndex().reverseLookup( id ) )
+        if ( null != aliasIndex.reverseLookup( candidate.getId() ) )
         {
             return false;
         }
 
         /*
-         * The candidate is NOT an alias at this point.  So if it is a 
-         * descendant we just return true since it is in normal subtree scope.
+         * The candidate is NOT an alias at this point.  So if it is a child we
+         * just return true since it is in normal one level scope.
          */
-        if ( isDescendant )
+        if ( isChild )
         {
             return true;
         }
 
         /*
-         * At this point the candidate is not a descendant and it is not an 
-         * alias.  We need to check if the candidate is in extended subtree 
-         * scope by performing a lookup on the subtree alias index.  This index 
-         * stores a tuple mapping the baseId to the ids of objects brought 
-         * into subtree scope of the base by an alias: 
-         * 
-         * ( baseId, aliasedObjId )
-         * 
-         * If the candidate id is an object brought into subtree scope then 
-         * the lookup returns true accepting the candidate.  Otherwise the 
+         * At this point the candidate is not a child and it is not an alias.
+         * We need to check if the candidate is in extended one level scope by
+         * performing a lookup on the one level alias index.  This index stores
+         * a tuple mapping the baseId to the id of objects brought into the
+         * one level scope of the base by an alias: ( baseId, aliasedObjId )
+         * If the candidate id is an object brought into one level scope then
+         * the lookup returns true accepting the candidate.  Otherwise the
          * candidate is rejected with a false return because it is not in scope.
          */
-        return db.getSubAliasIndex().has( baseId, id );
+        return aliasIndex.has( baseId, candidate.getId() );
     }
 
 
-    /**
-     * Asserts whether or not a candidate has one level scope while taking
-     * alias dereferencing into account.
-     * 
-     * @param id the candidate to assert which can be any db entry's id 
-     * @return true if the candidate is within one level scope whether or not
-     * alias dereferencing is enabled.
-     * @throws Exception if the index lookups fail.
-     */
-    public boolean isInOneLevelScope( final Long id ) throws Exception
+    public ScopeNode getExpression()
     {
-        boolean isChild = db.getOneLevelIndex().has( baseId, id );
+        return node;
+    }
 
-        /*
-         * The candidate id could be any entry in the db.  If search 
-         * dereferencing is not enabled then we return the results of the child 
-         * test. 
-         */
-        if ( !node.getDerefAliases().isDerefInSearching() )
-        {
-            return isChild;
-        }
 
-        /*
-         * From here down alias dereferencing is enabled.  We determine if the
-         * candidate id is an alias, if so we reject it since aliases should
-         * not be returned.
-         */
-        if ( null != db.getAliasIndex().reverseLookup( id ) )
-        {
-            return false;
-        }
+    public Long getBaseId()
+    {
+        return baseId;
+    }
 
-        /*
-         * The candidate is NOT an alias at this point.  So if it is a child we
-         * just return true since it is in normal one level scope.
-         */
-        if ( isChild )
-        {
-            return true;
-        }
 
-        /*
-         * At this point the candidate is not a child and it is not an alias.
-         * We need to check if the candidate is in extended one level scope by 
-         * performing a lookup on the one level alias index.  This index stores
-         * a tuple mapping the baseId to the id of objects brought into the 
-         * one level scope of the base by an alias: ( baseId, aliasedObjId )
-         * If the candidate id is an object brought into one level scope then 
-         * the lookup returns true accepting the candidate.  Otherwise the 
-         * candidate is rejected with a false return because it is not in scope.
-         */
-        return db.getOneAliasIndex().has( baseId, id );
+    public boolean isDereferencing()
+    {
+        return dereferencing;
     }
-}
+}
\ No newline at end of file

Added: directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/SubtreeScopeCursor.java
URL: http://svn.apache.org/viewvc/directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/SubtreeScopeCursor.java?rev=642692&view=auto
==============================================================================
--- directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/SubtreeScopeCursor.java (added)
+++ directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/SubtreeScopeCursor.java Sun Mar 30 00:19:32 2008
@@ -0,0 +1,227 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *  
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *  
+ *  Unless required by applicable law or agreed to in writing,
+ *  software distributed under the License is distributed on an
+ *  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  KIND, either express or implied.  See the License for the
+ *  specific language governing permissions and limitations
+ *  under the License. 
+ *  
+ */
+package org.apache.directory.server.xdbm.search.impl;
+
+
+import org.apache.directory.server.core.cursor.AbstractCursor;
+import org.apache.directory.server.core.cursor.Cursor;
+import org.apache.directory.server.core.cursor.InvalidCursorPositionException;
+import org.apache.directory.server.xdbm.IndexEntry;
+import org.apache.directory.server.xdbm.Store;
+
+import javax.naming.directory.Attributes;
+
+
+/**
+ * A Cursor over entries satisfying scope constraints with alias dereferencing
+ * considerations.
+ *
+ * @author <a href="mailto:dev@directory.apache.org">Apache Directory Project</a>
+ * @version $Rev$
+ */
+public class SubtreeScopeCursor extends AbstractCursor<IndexEntry<?, Attributes>>
+{
+    private static final String UNSUPPORTED_MSG =
+        "Scope Cursors are not ordered and do not support positioning by element.";
+
+    /** A ScopeNode Evaluator */
+    private final SubtreeScopeEvaluator evaluator;
+
+    /** A Cursor over the entries in the scope of the search base */
+    private final Cursor<IndexEntry<Long,Attributes>> scopeCursor;
+
+    /** A Cursor over entries brought into scope by alias dereferencing */
+    private final Cursor<IndexEntry<Long,Attributes>> dereferencedCursor;
+
+    /** Currently active Cursor: we switch between two cursors */
+    private Cursor<IndexEntry<Long,Attributes>> cursor;
+
+    /** Whether or not this Cursor is positioned so an entry is available */
+    private boolean available = false;
+
+
+    /**
+     * Creates a Cursor over entries satisfying subtree level scope criteria.
+     *
+     * @param db the entry store
+     * @param evaluator an IndexEntry (candidate) evaluator
+     * @throws Exception on db access failures
+     */
+    public SubtreeScopeCursor( Store<Attributes> db, SubtreeScopeEvaluator evaluator ) throws Exception
+    {
+        this.evaluator = evaluator;
+        scopeCursor = db.getSubLevelIndex().forwardCursor( evaluator.getBaseId() );
+
+        if ( evaluator.isDereferencing() )
+        {
+            dereferencedCursor = db.getSubAliasIndex().forwardCursor( evaluator.getBaseId() );
+        }
+        else
+        {
+            dereferencedCursor = null;
+        }
+    }
+
+
+    public boolean available()
+    {
+        return available;
+    }
+
+
+    public void before( IndexEntry<?, Attributes> element ) throws Exception
+    {
+        throw new UnsupportedOperationException( UNSUPPORTED_MSG );
+    }
+
+
+    public void after( IndexEntry<?, Attributes> element ) throws Exception
+    {
+        throw new UnsupportedOperationException( UNSUPPORTED_MSG );
+    }
+
+
+    public void beforeFirst() throws Exception
+    {
+        cursor = scopeCursor;
+        cursor.beforeFirst();
+        available = false;
+    }
+
+
+    public void afterLast() throws Exception
+    {
+        if ( evaluator.isDereferencing() )
+        {
+            cursor = dereferencedCursor;
+        }
+        else
+        {
+            cursor = scopeCursor;
+        }
+
+        cursor.afterLast();
+        available = false;
+    }
+
+
+    public boolean first() throws Exception
+    {
+        beforeFirst();
+        return next();
+    }
+
+
+    public boolean last() throws Exception
+    {
+        afterLast();
+        return previous();
+    }
+
+
+    public boolean previous() throws Exception
+    {
+        // if the cursor has not been set - position it after last element
+        if ( cursor == null )
+        {
+            afterLast();
+        }
+
+        available = cursor.previous();
+
+        // if we're using the scopeCursor (1st Cursor) then return result as is
+        if ( cursor == scopeCursor )
+        {
+            return available;
+        }
+
+        /*
+         * Below here we are using the dereferencedCursor so if nothing is
+         * available after an advance backwards we need to switch to the
+         * scopeCursor and try a previous call after positioning past it's
+         * last element.
+         */
+        if ( ! available )
+        {
+            cursor = scopeCursor;
+            cursor.afterLast();
+            return available = cursor.previous();
+        }
+
+        return true;
+    }
+
+
+    public boolean next() throws Exception
+    {
+        // if the cursor hasn't been set position it before the first element
+        if ( cursor == null )
+        {
+            beforeFirst();
+            return scopeCursor.next();
+        }
+
+        available = cursor.next();
+
+        // if we're using dereferencedCursor (2nd) then we return the result
+        if ( cursor == dereferencedCursor )
+        {
+            return available;
+        }
+
+        /*
+         * Below here we are using the scopeCursor so if nothing is
+         * available after an advance forward we need to switch to the
+         * dereferencedCursor and try a previous call after positioning past
+         * it's last element.
+         */
+        if ( ! available )
+        {
+            if ( dereferencedCursor != null )
+            {
+                cursor = dereferencedCursor;
+                cursor.beforeFirst();
+                return available = cursor.next();
+            }
+
+            return false;
+        }
+
+        return true;
+    }
+
+
+    public IndexEntry<Long, Attributes> get() throws Exception
+    {
+        if ( available )
+        {
+            return cursor.get();
+        }
+
+        throw new InvalidCursorPositionException( "Cursor has not been positioned yet." );
+    }
+
+
+    public boolean isElementReused()
+    {
+        return scopeCursor.isElementReused() ||
+            ( dereferencedCursor != null && dereferencedCursor.isElementReused() );
+    }
+}

Copied: directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/SubtreeScopeEvaluator.java (from r642683, directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/ScopeEvaluator.java)
URL: http://svn.apache.org/viewvc/directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/SubtreeScopeEvaluator.java?p2=directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/SubtreeScopeEvaluator.java&p1=directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/ScopeEvaluator.java&r1=642683&r2=642692&rev=642692&view=diff
==============================================================================
--- directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/ScopeEvaluator.java (original)
+++ directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/SubtreeScopeEvaluator.java Sun Mar 30 00:19:32 2008
@@ -20,91 +20,83 @@
 package org.apache.directory.server.xdbm.search.impl;
 
 
-import javax.naming.directory.SearchControls;
-
 import org.apache.directory.shared.ldap.filter.ScopeNode;
 import org.apache.directory.server.xdbm.IndexEntry;
 import org.apache.directory.server.xdbm.Store;
+import org.apache.directory.server.xdbm.Index;
+
+import javax.naming.directory.SearchControls;
 
 
 /**
- * Evaluates ScopeNode assertions on candidates using an entry database.
+ * Evaluates ScopeNode assertions with subtree scope on candidates using an
+ * entry database.
  * 
  * @author <a href="mailto:dev@directory.apache.org">Apache Directory Project</a>
  * @version $Rev$
  */
-public class ScopeEvaluator<E> implements Evaluator<ScopeNode,E>
+public class SubtreeScopeEvaluator<E> implements Evaluator<ScopeNode,E>
 {
-    /** Database used to evaluate scope with */
-    private final Store<E> db;
-
     /** The ScopeNode containing initial search scope constraints */
     private final ScopeNode node;
 
     /** The entry identifier of the scope base */
     private final Long baseId;
 
+    /** True if the scope requires alias dereferencing while searching */
+    private final boolean dereferencing;
+
+    /** The alias index used for subtree scope expansion */
+    private final Index<Long,E> aliasIndex;
+
+    /** The subtree scope index for parent-descendant mappings */
+    private final Index<Long,E> scopeIndex;
+
 
     /**
-     * Creates a scope node evaluator for search expressions.
+     * Creates a subtree scope node evaluator for search expressions.
      *
      * @param node the scope node
      * @param db the database used to evaluate scope node
      * @throws Exception on db access failure
      */
-    public ScopeEvaluator( Store<E> db, ScopeNode node ) throws Exception
+    public SubtreeScopeEvaluator( Store<E> db, ScopeNode node ) throws Exception
     {
-        this.db = db;
         this.node = node;
 
-        baseId = db.getEntryId( node.getBaseDn() );
-    }
-
-
-    /**
-     * @see Evaluator#evaluate(org.apache.directory.server.xdbm.IndexEntry)
-     */
-    public boolean evaluate( IndexEntry<?,E> entry ) throws Exception
-    {
-        switch ( node.getScope() )
+        if ( node.getScope() != SearchControls.SUBTREE_SCOPE )
         {
-            case ( SearchControls.OBJECT_SCOPE  ):
-                return entry.getId().longValue() == baseId.longValue();
-            case ( SearchControls.ONELEVEL_SCOPE  ):
-                return isInOneLevelScope( entry.getId() );
-            case ( SearchControls.SUBTREE_SCOPE  ):
-                return isInSubtreeScope( entry.getId() );
-            default:
-                throw new IllegalStateException( "Unrecognized search scope!" );
+            throw new IllegalStateException( "ScopeNode is not of subtree scope." );
         }
-    }
 
-
-    public ScopeNode getExpression()
-    {
-        return node;
+        baseId = db.getEntryId( node.getBaseDn() );
+        scopeIndex = db.getSubLevelIndex();
+        dereferencing = node.getDerefAliases().isDerefInSearching() ||
+            node.getDerefAliases().isDerefAlways();
+        aliasIndex = db.getSubAliasIndex();
     }
 
 
     /**
      * Asserts whether or not a candidate has one level scope while taking
      * alias dereferencing into account.
-     * 
-     * @param id the candidate to assert which can be any db entry's id
+     *
+     * @param candidate the entry tested to see if it is in subtree scope
      * @return true if the candidate is within one level scope whether or not
      * alias dereferencing is enabled.
      * @throws Exception if the index lookups fail.
+     * @see Evaluator#evaluate(org.apache.directory.server.xdbm.IndexEntry)
      */
-    public boolean isInSubtreeScope( final Long id ) throws Exception
+    public boolean evaluate( IndexEntry<?,E> candidate ) throws Exception
     {
-        boolean isDescendant = db.getSubLevelIndex().has( baseId, id );
+        boolean isDescendant = scopeIndex.has( baseId, candidate.getId() );
 
         /*
-         * The candidate id could be any entry in the db.  If search 
-         * dereferencing is not enabled then we return the results of the 
+         * The candidate id could be any entry in the db.  If search
+         * dereferencing is not enabled then we return the results of the
          * descendant test.
          */
-        if ( !node.getDerefAliases().isDerefInSearching() )
+        if ( ! isDereferencing() )
         {
             return isDescendant;
         }
@@ -114,13 +106,13 @@
          * candidate id is an alias, if so we reject it since aliases should
          * not be returned.
          */
-        if ( null != db.getAliasIndex().reverseLookup( id ) )
+        if ( null != aliasIndex.reverseLookup( candidate.getId() ) )
         {
             return false;
         }
 
         /*
-         * The candidate is NOT an alias at this point.  So if it is a 
+         * The candidate is NOT an alias at this point.  So if it is a
          * descendant we just return true since it is in normal subtree scope.
          */
         if ( isDescendant )
@@ -129,74 +121,36 @@
         }
 
         /*
-         * At this point the candidate is not a descendant and it is not an 
-         * alias.  We need to check if the candidate is in extended subtree 
-         * scope by performing a lookup on the subtree alias index.  This index 
-         * stores a tuple mapping the baseId to the ids of objects brought 
-         * into subtree scope of the base by an alias: 
-         * 
+         * At this point the candidate is not a descendant and it is not an
+         * alias.  We need to check if the candidate is in extended subtree
+         * scope by performing a lookup on the subtree alias index.  This index
+         * stores a tuple mapping the baseId to the ids of objects brought
+         * into subtree scope of the base by an alias:
+         *
          * ( baseId, aliasedObjId )
-         * 
-         * If the candidate id is an object brought into subtree scope then 
-         * the lookup returns true accepting the candidate.  Otherwise the 
+         *
+         * If the candidate id is an object brought into subtree scope then
+         * the lookup returns true accepting the candidate.  Otherwise the
          * candidate is rejected with a false return because it is not in scope.
          */
-        return db.getSubAliasIndex().has( baseId, id );
+        return aliasIndex.has( baseId, candidate.getId() );
     }
 
 
-    /**
-     * Asserts whether or not a candidate has one level scope while taking
-     * alias dereferencing into account.
-     * 
-     * @param id the candidate to assert which can be any db entry's id 
-     * @return true if the candidate is within one level scope whether or not
-     * alias dereferencing is enabled.
-     * @throws Exception if the index lookups fail.
-     */
-    public boolean isInOneLevelScope( final Long id ) throws Exception
+    public ScopeNode getExpression()
     {
-        boolean isChild = db.getOneLevelIndex().has( baseId, id );
+        return node;
+    }
 
-        /*
-         * The candidate id could be any entry in the db.  If search 
-         * dereferencing is not enabled then we return the results of the child 
-         * test. 
-         */
-        if ( !node.getDerefAliases().isDerefInSearching() )
-        {
-            return isChild;
-        }
 
-        /*
-         * From here down alias dereferencing is enabled.  We determine if the
-         * candidate id is an alias, if so we reject it since aliases should
-         * not be returned.
-         */
-        if ( null != db.getAliasIndex().reverseLookup( id ) )
-        {
-            return false;
-        }
+    public Long getBaseId()
+    {
+        return baseId;
+    }
 
-        /*
-         * The candidate is NOT an alias at this point.  So if it is a child we
-         * just return true since it is in normal one level scope.
-         */
-        if ( isChild )
-        {
-            return true;
-        }
 
-        /*
-         * At this point the candidate is not a child and it is not an alias.
-         * We need to check if the candidate is in extended one level scope by 
-         * performing a lookup on the one level alias index.  This index stores
-         * a tuple mapping the baseId to the id of objects brought into the 
-         * one level scope of the base by an alias: ( baseId, aliasedObjId )
-         * If the candidate id is an object brought into one level scope then 
-         * the lookup returns true accepting the candidate.  Otherwise the 
-         * candidate is rejected with a false return because it is not in scope.
-         */
-        return db.getOneAliasIndex().has( baseId, id );
+    public boolean isDereferencing()
+    {
+        return dereferencing;
     }
 }

Propchange: directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/SubtreeScopeEvaluator.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: directory/sandbox/akarasulu/bigbang/apacheds/xdbm-search/src/main/java/org/apache/directory/server/xdbm/search/impl/SubtreeScopeEvaluator.java
------------------------------------------------------------------------------
--- svn:keywords (added)
+++ svn:keywords Sun Mar 30 00:19:32 2008
@@ -0,0 +1,4 @@
+Rev
+Revision
+Date
+Id