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

svn commit: r1199702 - in /directory/apacheds/trunk: core-api/src/main/java/org/apache/directory/server/core/api/ core-api/src/main/java/org/apache/directory/server/core/api/interceptor/ core-api/src/test/java/org/apache/directory/server/core/api/ core...

Author: elecharny
Date: Wed Nov  9 11:45:10 2011
New Revision: 1199702

URL: http://svn.apache.org/viewvc?rev=1199702&view=rev
Log:
Moved the Bind operation out of the InterceptorChain

Modified:
    directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/DirectoryService.java
    directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/LdapCoreSessionConnection.java
    directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/OperationEnum.java
    directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/BaseInterceptor.java
    directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/Interceptor.java
    directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/InterceptorChain.java
    directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/NextInterceptor.java
    directory/apacheds/trunk/core-api/src/test/java/org/apache/directory/server/core/api/MockDirectoryService.java
    directory/apacheds/trunk/core-api/src/test/java/org/apache/directory/server/core/api/MockInterceptor.java
    directory/apacheds/trunk/core/src/main/java/org/apache/directory/server/core/DefaultDirectoryService.java
    directory/apacheds/trunk/core/src/main/java/org/apache/directory/server/core/DefaultOperationManager.java
    directory/apacheds/trunk/interceptors/authn/src/main/java/org/apache/directory/server/core/authn/AuthenticationInterceptor.java
    directory/apacheds/trunk/interceptors/logger/src/main/java/org/apache/directory/server/core/logger/TimerInterceptor.java
    directory/apacheds/trunk/interceptors/normalization/src/main/java/org/apache/directory/server/core/normalization/NormalizationInterceptor.java
    directory/apacheds/trunk/ldap-client-test/src/test/java/org/apache/directory/shared/client/api/operations/bind/SimpleBindRequestTest.java
    directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/handlers/BindHandler.java

Modified: directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/DirectoryService.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/DirectoryService.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/DirectoryService.java (original)
+++ directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/DirectoryService.java Wed Nov  9 11:45:10 2011
@@ -341,6 +341,34 @@ public interface DirectoryService extend
      * @param interceptors the interceptors to be used in the server.
      */
     void setInterceptors( List<Interceptor> interceptors );
+    
+    
+    /**
+     * Add an interceptor in the first position in the interceptor list.
+     * @param interceptor The added interceptor
+     */
+    void addFirst( Interceptor interceptor ) throws LdapException;
+    
+    
+    /**
+     * Add an interceptor in the last position in the interceptor list.
+     * @param interceptor The added interceptor
+     */
+    void addLast( Interceptor interceptor ) throws LdapException;
+    
+    /**
+     * Add an interceptor after a given interceptor in the interceptor list.
+     * @param interceptorName The interceptor name to find
+     * @param interceptor The added interceptor
+     */
+    void addAfter( String interceptorName, Interceptor interceptor );
+    
+    
+    /**
+     * Remove an interceptor from the list of interceptors
+     * @param interceptorName The interceptor to remove
+     */
+    void remove( String interceptorName );
 
 
     /**

Modified: directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/LdapCoreSessionConnection.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/LdapCoreSessionConnection.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/LdapCoreSessionConnection.java (original)
+++ directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/LdapCoreSessionConnection.java Wed Nov  9 11:45:10 2011
@@ -1183,6 +1183,7 @@ public class LdapCoreSessionConnection i
         BindOperationContext bindContext = new BindOperationContext( null );
         bindContext.setCredentials( bindRequest.getCredentials() );
         bindContext.setDn( bindRequest.getName() );
+        bindContext.setInterceptors( directoryService.getInterceptors( OperationEnum.BIND ) );
 
         OperationManager operationManager = directoryService.getOperationManager();
 

Modified: directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/OperationEnum.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/OperationEnum.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/OperationEnum.java (original)
+++ directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/OperationEnum.java Wed Nov  9 11:45:10 2011
@@ -43,6 +43,25 @@ public enum OperationEnum {
 	/** The associated method name */
 	private String methodName;
 	
+	/** A list of all the operations */
+	private static OperationEnum[] operations = new OperationEnum[] 
+			{
+				ADD,
+				BIND,
+				COMPARE,
+				DELETE,
+				GET_ROOT_DSE,
+				HAS_ENTRY,
+				LIST,
+				LOOKUP,
+				MODIFY,
+				MOVE,
+				MOVE_AND_RENAME,
+				RENAME,
+				SEARCH,
+				UNBIND
+			};
+	
 	/**
 	 * The private constructor 
 	 * @param methodName The associated method name
@@ -60,4 +79,13 @@ public enum OperationEnum {
 	{
 		return methodName;
 	}
+	
+	
+	/**
+	 * @return The list of all the operations
+	 */
+	public static OperationEnum[] getOperations()
+	{
+		return operations;
+	}
 }

Modified: directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/BaseInterceptor.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/BaseInterceptor.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/BaseInterceptor.java (original)
+++ directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/BaseInterceptor.java Wed Nov  9 11:45:10 2011
@@ -58,6 +58,9 @@ import org.apache.directory.shared.ldap.
  */
 public abstract class BaseInterceptor implements Interceptor
 {
+	/** The interceptor's name. Default to the class name */
+	private String name;
+	
     /** A reference to the DirectoryService instance */
     protected DirectoryService directoryService;
 
@@ -220,7 +223,7 @@ public abstract class BaseInterceptor im
         }
 
 
-        public void bind( NextInterceptor next, BindOperationContext bindContext ) throws LdapException
+        public void bind( BindOperationContext bindContext ) throws LdapException
         {
             nexus.bind( bindContext );
         }
@@ -243,7 +246,7 @@ public abstract class BaseInterceptor im
      */
     public String getName()
     {
-        return getClass().getSimpleName();
+        return name;
     }
 
 
@@ -264,6 +267,16 @@ public abstract class BaseInterceptor im
      */
     protected BaseInterceptor()
     {
+    	name = getClass().getSimpleName();
+    }
+
+
+    /**
+     * Creates a new instance.
+     */
+    protected BaseInterceptor( String name )
+    {
+    	this.name = name;
     }
 
 
@@ -377,7 +390,7 @@ public abstract class BaseInterceptor im
     /**
      * Calls the next interceptor for the getRootDse operation.
      * 
-     * @param deleteContext The context in which we are executing this operation
+     * @param getRootDseContext The context in which we are executing this operation
      * @throws LdapException If something went wrong
      */
     protected final Entry next( GetRootDSEOperationContext getRootDseContext ) throws LdapException
@@ -447,9 +460,26 @@ public abstract class BaseInterceptor im
     }
 
 
-    public void bind( NextInterceptor next, BindOperationContext bindContext ) throws LdapException
+    /**
+     * {@inheritDoc}
+     */
+    public void bind( BindOperationContext bindContext ) throws LdapException
     {
-        next.bind( bindContext );
+        // Do nothing
+    }
+    
+
+    /**
+     * Calls the next interceptor for the bind operation.
+     * 
+     * @param bindContext The context in which we are executing this operation
+     * @throws LdapException If something went wrong
+     */
+    protected final void next( BindOperationContext bindContext ) throws LdapException
+    {
+    	Interceptor interceptor = getNextInterceptor( bindContext );
+    	
+    	interceptor.bind( bindContext );
     }
 
 
@@ -465,7 +495,7 @@ public abstract class BaseInterceptor im
     /**
      * Compute the next interceptor for the unbind operation.
      * 
-     * @param deleteContext The context in which we are executing this operation
+     * @param unbindContext The context in which we are executing this operation
      * @throws LdapException If something went wrong
      */
     protected final void next( UnbindOperationContext unbindContext ) throws LdapException

Modified: directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/Interceptor.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/Interceptor.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/Interceptor.java (original)
+++ directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/Interceptor.java Wed Nov  9 11:45:10 2011
@@ -139,7 +139,7 @@ public interface Interceptor
     /**
      * Filters {@link Partition#bind( BindOperationContext )} call.
      */
-    void bind( NextInterceptor next, BindOperationContext bindContext ) throws LdapException;
+    void bind( BindOperationContext bindContext ) throws LdapException;
 
 
     /**

Modified: directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/InterceptorChain.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/InterceptorChain.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/InterceptorChain.java (original)
+++ directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/InterceptorChain.java Wed Nov  9 11:45:10 2011
@@ -172,9 +172,9 @@ public class InterceptorChain
         }
 
 
-        public void bind( NextInterceptor next, BindOperationContext bindContext ) throws LdapException
+        public void bind( BindOperationContext bindContext ) throws LdapException
         {
-            nexus.bind( bindContext );
+            // Do nothing
         }
 
 
@@ -633,27 +633,6 @@ public class InterceptorChain
     }
 
 
-    public void bind( BindOperationContext bindContext ) throws LdapException
-    {
-        Element node = getStartingEntry();
-        Interceptor head = node.interceptor;
-        NextInterceptor next = node.nextInterceptor;
-
-        try
-        {
-            head.bind( next, bindContext );
-        }
-        catch ( LdapException le )
-        {
-            throw le;
-        }
-        catch ( Throwable e )
-        {
-            throwInterceptorException( head, e );
-        }
-    }
-
-
     public void modify( ModifyOperationContext modifyContext ) throws LdapException
     {
         Element entry = getStartingEntry();
@@ -1136,28 +1115,6 @@ public class InterceptorChain
                         throwInterceptorException( interceptor, e );
                     }
                 }
-
-
-                public void bind( BindOperationContext bindContext ) throws LdapException
-                {
-                    Element next = getNextEntry();
-                    Interceptor interceptor = next.interceptor;
-
-                    try
-                    {
-                        //System.out.println( ">>> Entering into " + interceptor.getClass().getSimpleName() + ", bindRequest" );
-                        interceptor.bind( next.nextInterceptor, bindContext );
-                        //System.out.println( "<<< Exiting from " + interceptor.getClass().getSimpleName() + ", bindRequest" );
-                    }
-                    catch ( LdapException le )
-                    {
-                        throw le;
-                    }
-                    catch ( Throwable e )
-                    {
-                        throwInterceptorException( interceptor, e );
-                    }
-                }
             };
         }
     }

Modified: directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/NextInterceptor.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/NextInterceptor.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/NextInterceptor.java (original)
+++ directory/apacheds/trunk/core-api/src/main/java/org/apache/directory/server/core/api/interceptor/NextInterceptor.java Wed Nov  9 11:45:10 2011
@@ -22,7 +22,6 @@ package org.apache.directory.server.core
 
 import org.apache.directory.server.core.api.filtering.EntryFilteringCursor;
 import org.apache.directory.server.core.api.interceptor.context.AddOperationContext;
-import org.apache.directory.server.core.api.interceptor.context.BindOperationContext;
 import org.apache.directory.server.core.api.interceptor.context.CompareOperationContext;
 import org.apache.directory.server.core.api.interceptor.context.EntryOperationContext;
 import org.apache.directory.server.core.api.interceptor.context.ListOperationContext;
@@ -102,10 +101,4 @@ public interface NextInterceptor
      * Calls the next interceptor's {@link Interceptor#moveAndRename( NextInterceptor, MoveAndRenameOperationContext )}.
      */
     void moveAndRename( MoveAndRenameOperationContext moveAndRenameContext ) throws LdapException;
-
-
-    /**
-     * Calls the next interceptor's {@link Interceptor#bind( NextInterceptor, BindOperationContext )}
-     */
-    void bind( BindOperationContext bindContext ) throws LdapException;
 }

Modified: directory/apacheds/trunk/core-api/src/test/java/org/apache/directory/server/core/api/MockDirectoryService.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/core-api/src/test/java/org/apache/directory/server/core/api/MockDirectoryService.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/core-api/src/test/java/org/apache/directory/server/core/api/MockDirectoryService.java (original)
+++ directory/apacheds/trunk/core-api/src/test/java/org/apache/directory/server/core/api/MockDirectoryService.java Wed Nov  9 11:45:10 2011
@@ -581,8 +581,37 @@ public class MockDirectoryService implem
 
 
 	@Override
-	public List<String> getInterceptors(OperationEnum operation) {
+	public List<String> getInterceptors( OperationEnum operation ) 
+	{
 		// TODO Auto-generated method stub
 		return null;
 	}
+
+
+	@Override
+	public void addFirst( Interceptor interceptor )  throws LdapException 
+	{
+		// TODO Auto-generated method stub
+	}
+
+
+	@Override
+	public void addLast( Interceptor interceptor ) throws LdapException 
+	{
+		// TODO Auto-generated method stub
+	}
+
+
+	@Override
+	public void addAfter( String interceptorName, Interceptor interceptor ) 
+	{
+		// TODO Auto-generated method stub
+	}
+
+
+	@Override
+	public void remove( String interceptorName ) 
+	{
+		// TODO Auto-generated method stub
+	}
 }

Modified: directory/apacheds/trunk/core-api/src/test/java/org/apache/directory/server/core/api/MockInterceptor.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/core-api/src/test/java/org/apache/directory/server/core/api/MockInterceptor.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/core-api/src/test/java/org/apache/directory/server/core/api/MockInterceptor.java (original)
+++ directory/apacheds/trunk/core-api/src/test/java/org/apache/directory/server/core/api/MockInterceptor.java Wed Nov  9 11:45:10 2011
@@ -170,10 +170,10 @@ public class MockInterceptor extends Bas
     }
 
 
-    public void bind( NextInterceptor next, BindOperationContext bindContext ) throws LdapException
+    public void bind( BindOperationContext bindContext ) throws LdapException
     {
         interceptors.add( this );
-        next.bind( bindContext );
+        next( bindContext );
     }
 
 

Modified: directory/apacheds/trunk/core/src/main/java/org/apache/directory/server/core/DefaultDirectoryService.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/core/src/main/java/org/apache/directory/server/core/DefaultDirectoryService.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/core/src/main/java/org/apache/directory/server/core/DefaultDirectoryService.java (original)
+++ directory/apacheds/trunk/core/src/main/java/org/apache/directory/server/core/DefaultDirectoryService.java Wed Nov  9 11:45:10 2011
@@ -31,12 +31,16 @@ import java.nio.channels.FileLock;
 import java.nio.channels.OverlappingFileLockException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import javax.naming.directory.Attributes;
 
@@ -238,6 +242,14 @@ public class DefaultDirectoryService imp
 
     /** The list of declared interceptors */
     private List<Interceptor> interceptors;
+    private Map<String, Interceptor> interceptorNames;
+    
+    /** A lock to protect the interceptors List */
+    private ReadWriteLock interceptorsLock = new ReentrantReadWriteLock();
+    
+    /** The read and write locks */
+    private Lock readLock  = interceptorsLock.readLock();
+    private Lock writeLock  = interceptorsLock.writeLock();
     
     /** A map associating a list of interceptor to each operation */
     private Map<OperationEnum, List<String>> operationInterceptors;
@@ -465,9 +477,19 @@ public class DefaultDirectoryService imp
     public List<Interceptor> getInterceptors()
     {
         List<Interceptor> cloned = new ArrayList<Interceptor>();
-        cloned.addAll( interceptors );
+    	
+        try
+        {
+            readLock.lock();
+            
+        	cloned.addAll( interceptors );
         
-        return cloned;
+        	return cloned;
+        }
+        finally
+        {
+            readLock.unlock();
+        }
     }
 
 
@@ -479,38 +501,149 @@ public class DefaultDirectoryService imp
     public List<String> getInterceptors( OperationEnum operation )
     {
         List<String> cloned = new ArrayList<String>();
-        cloned.addAll( operationInterceptors.get( operation ) );
         
-        return cloned;
+        try
+        {
+	        readLock.lock();
+	        cloned.addAll( operationInterceptors.get( operation ) );
+	        
+	        return cloned;
+        }
+        finally
+        {
+            readLock.unlock();
+        }
+
     }
     
     
     /**
-     * Compute the list of interceptors to call for each operation
+     * Compute the list of  to call for each operation
      */
-    private void initOperationsList( OperationEnum... operations)
+    private void initOperationsList()
     {
-    	operationInterceptors = new ConcurrentHashMap<OperationEnum, List<String>>();
+    	try
+    	{
+	        writeLock.lock();
+	    	operationInterceptors = new ConcurrentHashMap<OperationEnum, List<String>>();
+	    	
+	    	for ( OperationEnum operation : OperationEnum.getOperations() )
+	    	{
+		    	List<String> operationList = new ArrayList<String>();
+		    	
+		        for ( Interceptor interceptor : interceptors )
+		        {
+			    	Method[] methods = interceptor.getClass().getDeclaredMethods();
+			    	
+			    	for ( Method method : methods )
+			    	{
+			    		if ( method.getName().equals( operation.getMethodName() ) )
+			    		{
+			    			operationList.add( interceptor.getName() );
+			    			break;
+			    		}
+			    	}
+		        }
+		        
+		        operationInterceptors.put( operation, operationList );
+	    	}
+    	}
+    	finally
+    	{
+    		writeLock.unlock();
+    	}
+    }
+    
+    
+    /**
+     * Add an interceptor to the list of interceptors to call for each operation
+     * @throws LdapException 
+     */
+    private void addInterceptor( Interceptor interceptor, int position ) throws LdapException
+    {
+    	// First, init the interceptor
+    	interceptor.init( this );
     	
-    	for ( OperationEnum operation : operations )
+    	try
     	{
-	    	List<String> operationList = new ArrayList<String>();
+	        writeLock.lock();
 	    	
-	        for ( Interceptor interceptor : interceptors )
-	        {
+	    	for ( OperationEnum operation : OperationEnum.getOperations() )
+	    	{
+		    	List<String> operationList = operationInterceptors.get( operation );
+		    		    	
 		    	Method[] methods = interceptor.getClass().getDeclaredMethods();
 		    	
 		    	for ( Method method : methods )
 		    	{
 		    		if ( method.getName().equals( operation.getMethodName() ) )
 		    		{
-		    			operationList.add( interceptor.getName() );
+		    	    	if ( position == -1 )
+		    	    	{
+		    	    		operationList.add( interceptor.getName() );
+		    	    	}
+		    	    	else
+		    	    	{
+		    	    		operationList.add( position, interceptor.getName() );
+		    	    	}
+		    	    	
 		    			break;
 		    		}
 		    	}
-	        }
-	        
-	        operationInterceptors.put( operation, operationList );
+	    	}
+	    	
+	    	interceptorNames.put( interceptor.getName(), interceptor );
+	    	
+	    	if ( position == -1 )
+	    	{
+	    		interceptors.add( interceptor );
+	    	}
+	    	else
+	    	{
+	    		interceptors.add( position, interceptor );
+	    	}
+    	}
+    	finally
+    	{
+    		writeLock.unlock();
+    	}
+    }
+
+    
+    /**
+     * Remove an interceptor to the list of interceptors to call for each operation
+     */
+    private void removeOperationsList( String interceptorName )
+    {
+    	Interceptor interceptor = interceptorNames.get( interceptorName );
+    	
+    	try
+    	{
+	        writeLock.lock();
+	    	
+	    	for ( OperationEnum operation : OperationEnum.getOperations() )
+	    	{
+		    	List<String> operationList = operationInterceptors.get( operation );
+		    		    	
+		    	Method[] methods = interceptor.getClass().getDeclaredMethods();
+		    	
+		    	for ( Method method : methods )
+		    	{
+		    		if ( method.getName().equals( operation.getMethodName() ) )
+		    		{
+	    	    		operationList.remove( interceptor.getName() );
+		    	    	
+		    			break;
+		    		}
+		    	}
+	    	}
+	    	
+	    	interceptorNames.remove( interceptorName );
+	    	interceptors.remove( interceptor );
+    	}
+    	finally
+    	{
+    		writeLock.unlock();
     	}
     }
 
@@ -522,39 +655,25 @@ public class DefaultDirectoryService imp
      */
     public void setInterceptors( List<Interceptor> interceptors )
     {
-        Set<String> names = new HashSet<String>();
+        Map<String, Interceptor> interceptorNames = new HashMap<String, Interceptor>();
 
         // Check if we don't have duplicate names in the interceptors list
         for ( Interceptor interceptor : interceptors )
         {
-            if ( names.contains( interceptor.getName() ) )
+            if ( interceptorNames.containsKey( interceptor.getName() ) )
             {
                 LOG.warn( "Encountered duplicate definitions for {} interceptor", interceptor.getName() );
                 continue;
             }
             
-            names.add( interceptor.getName() );
+            interceptorNames.put( interceptor.getName(), interceptor );
         }
-        
+
         this.interceptors = interceptors;
+        this.interceptorNames = interceptorNames;
 
         // Now update the Map that connect each operation with the list of interceptors.
-    	initOperationsList( 
-    			OperationEnum.ADD, 
-    			OperationEnum.BIND,
-    			OperationEnum.COMPARE,
-    			OperationEnum.DELETE,
-    			OperationEnum.GET_ROOT_DSE,
-    			OperationEnum.HAS_ENTRY,
-    			OperationEnum.LIST,
-    			OperationEnum.LOOKUP,
-    			OperationEnum.MODIFY,
-    			OperationEnum.MOVE,
-    			OperationEnum.MOVE_AND_RENAME,
-    			OperationEnum.RENAME,
-    			OperationEnum.SEARCH,
-    			OperationEnum.UNBIND
-    			);
+    	initOperationsList();
     }
 
 
@@ -568,6 +687,7 @@ public class DefaultDirectoryService imp
     {
         List<LdifEntry> cloned = new ArrayList<LdifEntry>();
         cloned.addAll( testEntries );
+        
         return cloned;
     }
 
@@ -840,6 +960,8 @@ public class DefaultDirectoryService imp
         BindOperationContext bindContext = new BindOperationContext( null );
         bindContext.setCredentials( credentials );
         bindContext.setDn( principalDn );
+        bindContext.setInterceptors( getInterceptors( OperationEnum.BIND ) );
+        
         operationManager.bind( bindContext );
 
         return bindContext.getSession();
@@ -858,6 +980,8 @@ public class DefaultDirectoryService imp
         bindContext.setCredentials( credentials );
         bindContext.setDn( principalDn );
         bindContext.setSaslMechanism( saslMechanism );
+        bindContext.setInterceptors( getInterceptors( OperationEnum.BIND ) );
+
         operationManager.bind( bindContext );
 
         return bindContext.getSession();
@@ -1846,19 +1970,65 @@ public class DefaultDirectoryService imp
      */
     public Interceptor getInterceptor( String interceptorName )
     {
-        for ( Interceptor interceptor:interceptors )
+    	try
+    	{
+	        readLock.lock();
+	
+	        for ( Interceptor interceptor:interceptors )
+	        {
+	            if ( interceptor.getName().equalsIgnoreCase( interceptorName ) )
+	            {
+	                return interceptor;
+	            }
+	        }
+	        
+	        return null;
+        }
+        finally
         {
-            if ( interceptor.getName().equalsIgnoreCase( interceptorName ) )
-            {
-                return interceptor;
-            }
+            readLock.unlock();
         }
-
-        return null;
     }
 
 
     /**
+     * {@inheritDoc}
+     * @throws LdapException 
+     */
+	public void addFirst( Interceptor interceptor ) throws LdapException 
+	{
+		addInterceptor( interceptor, 0 );
+	}
+
+
+    /**
+     * {@inheritDoc}
+     * @throws LdapException 
+     */
+	public void addLast( Interceptor interceptor ) throws LdapException 
+	{
+		addInterceptor( interceptor, -1 );
+	}
+
+
+    /**
+     * {@inheritDoc}
+     */
+	public void addAfter( String interceptorName, Interceptor interceptor ) 
+	{
+	}
+
+
+    /**
+     * {@inheritDoc}
+     */
+	public void remove( String interceptorName ) 
+	{
+		removeOperationsList( interceptorName );
+	}
+
+
+    /**
      * Get a new CSN
      * @return The CSN generated for this directory service
      */
@@ -2060,5 +2230,4 @@ public class DefaultDirectoryService imp
     {
         return evaluator;
     }
-
 }
\ No newline at end of file

Modified: directory/apacheds/trunk/core/src/main/java/org/apache/directory/server/core/DefaultOperationManager.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/core/src/main/java/org/apache/directory/server/core/DefaultOperationManager.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/core/src/main/java/org/apache/directory/server/core/DefaultOperationManager.java (original)
+++ directory/apacheds/trunk/core/src/main/java/org/apache/directory/server/core/DefaultOperationManager.java Wed Nov  9 11:45:10 2011
@@ -326,7 +326,10 @@ public class DefaultOperationManager imp
 
         try
         {
-            directoryService.getInterceptorChain().bind( bindContext );
+            // Call the Delete method
+            Interceptor head = directoryService.getInterceptor( bindContext.getNextInterceptor() );
+
+            head.bind( bindContext );
         }
         finally
         {

Modified: directory/apacheds/trunk/interceptors/authn/src/main/java/org/apache/directory/server/core/authn/AuthenticationInterceptor.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/interceptors/authn/src/main/java/org/apache/directory/server/core/authn/AuthenticationInterceptor.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/interceptors/authn/src/main/java/org/apache/directory/server/core/authn/AuthenticationInterceptor.java (original)
+++ directory/apacheds/trunk/interceptors/authn/src/main/java/org/apache/directory/server/core/authn/AuthenticationInterceptor.java Wed Nov  9 11:45:10 2011
@@ -810,7 +810,7 @@ public class AuthenticationInterceptor e
     }
 
 
-    public void bind( NextInterceptor next, BindOperationContext bindContext ) throws LdapException
+    public void bind( BindOperationContext bindContext ) throws LdapException
     {
         if ( IS_DEBUG )
         {
@@ -842,7 +842,7 @@ public class AuthenticationInterceptor e
             LOG.debug( "No authenticators found, delegating bind to the nexus." );
 
             // as a last resort try binding via the nexus
-            next.bind( bindContext );
+            next( bindContext );
 
             LOG.debug( "Nexus succeeded on bind operation." );
 

Modified: directory/apacheds/trunk/interceptors/logger/src/main/java/org/apache/directory/server/core/logger/TimerInterceptor.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/interceptors/logger/src/main/java/org/apache/directory/server/core/logger/TimerInterceptor.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/interceptors/logger/src/main/java/org/apache/directory/server/core/logger/TimerInterceptor.java (original)
+++ directory/apacheds/trunk/interceptors/logger/src/main/java/org/apache/directory/server/core/logger/TimerInterceptor.java Wed Nov  9 11:45:10 2011
@@ -174,7 +174,7 @@ public class TimerInterceptor extends Ba
     public void bind( NextInterceptor next, BindOperationContext bindContext ) throws LdapException
     {
         long t0 = System.nanoTime();
-        next.bind( bindContext );
+        next( bindContext );
         long delta = System.nanoTime() - t0;
         
         if ( IS_DEBUG_STATS )

Modified: directory/apacheds/trunk/interceptors/normalization/src/main/java/org/apache/directory/server/core/normalization/NormalizationInterceptor.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/interceptors/normalization/src/main/java/org/apache/directory/server/core/normalization/NormalizationInterceptor.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/interceptors/normalization/src/main/java/org/apache/directory/server/core/normalization/NormalizationInterceptor.java (original)
+++ directory/apacheds/trunk/interceptors/normalization/src/main/java/org/apache/directory/server/core/normalization/NormalizationInterceptor.java Wed Nov  9 11:45:10 2011
@@ -385,10 +385,10 @@ public class NormalizationInterceptor ex
     /**
      * {@inheritDoc}
      */
-    public void bind( NextInterceptor next, BindOperationContext bindContext ) throws LdapException
+    public void bind( BindOperationContext bindContext ) throws LdapException
     {
         bindContext.getDn().apply( schemaManager );
-        next.bind( bindContext );
+        next( bindContext );
     }
 
 

Modified: directory/apacheds/trunk/ldap-client-test/src/test/java/org/apache/directory/shared/client/api/operations/bind/SimpleBindRequestTest.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/ldap-client-test/src/test/java/org/apache/directory/shared/client/api/operations/bind/SimpleBindRequestTest.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/ldap-client-test/src/test/java/org/apache/directory/shared/client/api/operations/bind/SimpleBindRequestTest.java (original)
+++ directory/apacheds/trunk/ldap-client-test/src/test/java/org/apache/directory/shared/client/api/operations/bind/SimpleBindRequestTest.java Wed Nov  9 11:45:10 2011
@@ -38,7 +38,6 @@ import org.apache.directory.server.annot
 import org.apache.directory.server.core.annotations.ApplyLdifs;
 import org.apache.directory.server.core.api.interceptor.BaseInterceptor;
 import org.apache.directory.server.core.api.interceptor.Interceptor;
-import org.apache.directory.server.core.api.interceptor.NextInterceptor;
 import org.apache.directory.server.core.api.interceptor.context.BindOperationContext;
 import org.apache.directory.server.core.integ.AbstractLdapTestUnit;
 import org.apache.directory.server.core.integ.FrameworkRunner;
@@ -455,12 +454,12 @@ public class SimpleBindRequestTest exten
         {
             // Inject the interceptor that waits 1 second when binding 
             // in order to be able to send a request before we get the response
-            Interceptor interceptor = new BaseInterceptor()
+            Interceptor interceptor = new BaseInterceptor( "test" )
             {
                 /**
                  * Wait 1 second before going any further
                  */
-                public void bind( NextInterceptor next, BindOperationContext bindContext ) throws LdapException
+                public void bind( BindOperationContext bindContext ) throws LdapException
                 {
                     // Wait 1 second
                     try
@@ -472,11 +471,11 @@ public class SimpleBindRequestTest exten
                         // Ok, get out
                     }
 
-                    next.bind( bindContext );
+                    next( bindContext );
                 }
             };
             
-            getService().getInterceptorChain().addFirst( interceptor );
+            getService().addFirst( interceptor );
 
             // Send another BindRequest
             BindRequest bindRequest = new BindRequestImpl();
@@ -509,7 +508,7 @@ public class SimpleBindRequestTest exten
         }
         finally
         {
-            getService().getInterceptorChain().remove( "" );
+            getService().remove( "test" );
         }
     }
 

Modified: directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/handlers/BindHandler.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/handlers/BindHandler.java?rev=1199702&r1=1199701&r2=1199702&view=diff
==============================================================================
--- directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/handlers/BindHandler.java (original)
+++ directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/handlers/BindHandler.java Wed Nov  9 11:45:10 2011
@@ -29,6 +29,7 @@ import org.apache.commons.lang.exception
 import org.apache.directory.server.core.api.CoreSession;
 import org.apache.directory.server.core.api.DirectoryService;
 import org.apache.directory.server.core.api.LdapPrincipal;
+import org.apache.directory.server.core.api.OperationEnum;
 import org.apache.directory.server.core.api.entry.ClonedServerEntry;
 import org.apache.directory.server.core.api.interceptor.context.BindOperationContext;
 import org.apache.directory.server.i18n.I18n;
@@ -168,6 +169,7 @@ public class BindHandler extends LdapReq
             // opContext.setEntry( principalEntry );
 
             // And call the OperationManager bind operation.
+            bindContext.setInterceptors( getLdapServer().getDirectoryService().getInterceptors( OperationEnum.BIND ) );
             getLdapServer().getDirectoryService().getOperationManager().bind( bindContext );
 
             // As a result, store the created session in the Core Session