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/08/24 21:07:01 UTC

svn commit: r688548 - in /directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap: ./ gui/ handlers/

Author: akarasulu
Date: Sun Aug 24 12:07:01 2008
New Revision: 688548

URL: http://svn.apache.org/viewvc?rev=688548&view=rev
Log:
restructured LdapServer with respect to LdapProtocolHandler and codec

  o cleaned up a bunch of inlined code to explicitly mark off handling of 
    session cleanup and codec handling
  o moved inner classes up for ProtocolCodec and ProtocolCodec factories out of
    the LdapServer
  o created new LdapSessionManager object wrapping the map of IoSessions to 
    LdapSessions with synchronization to be accessed by various parts of the 
    server - LdapSessionManager is created and a member of LdapServer now
  o LdapServer is clean and simple now with configuration interfaces and methods
    to start, stop and query state instead of containing all the codec and 
    factory code.


Added:
    directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapProtocolCodecFactory.java
    directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapProtocolHandler.java
    directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapSessionManager.java
Modified:
    directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapServer.java
    directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/gui/SessionsFrame.java
    directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/handlers/LdapRequestHandler.java
    directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/handlers/UnbindHandler.java

Added: directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapProtocolCodecFactory.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapProtocolCodecFactory.java?rev=688548&view=auto
==============================================================================
--- directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapProtocolCodecFactory.java (added)
+++ directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapProtocolCodecFactory.java Sun Aug 24 12:07:01 2008
@@ -0,0 +1,93 @@
+/*
+ *  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.ldap;
+
+
+import org.apache.directory.server.core.DirectoryService;
+import org.apache.directory.server.schema.registries.AttributeTypeRegistry;
+import org.apache.directory.shared.asn1.codec.Asn1CodecDecoder;
+import org.apache.directory.shared.asn1.codec.Asn1CodecEncoder;
+import org.apache.directory.shared.ldap.message.MessageDecoder;
+import org.apache.directory.shared.ldap.message.MessageEncoder;
+import org.apache.directory.shared.ldap.message.spi.BinaryAttributeDetector;
+import org.apache.directory.shared.ldap.schema.AttributeType;
+import org.apache.mina.filter.codec.ProtocolCodecFactory;
+import org.apache.mina.filter.codec.ProtocolDecoder;
+import org.apache.mina.filter.codec.ProtocolEncoder;
+
+
+/**
+ * An LDAP BER Decoder/Encoder factory implementing {@link ProtocolCodecFactory}.
+ *
+ * @author <a href="mailto:dev@directory.apache.org">Apache Directory Project</a>
+ * @version $Rev$
+ */
+final class LdapProtocolCodecFactory implements ProtocolCodecFactory
+{
+    /** the directory service for which this factor generates codecs */
+    final private DirectoryService directoryService;
+
+
+    /**
+     * Creates a new instance of LdapProtocolCodecFactory.
+     *
+     * @param directoryService the {@link DirectoryService} for which this 
+     * factory generates codecs.
+     */
+    public LdapProtocolCodecFactory( DirectoryService directoryService )
+    {
+        this.directoryService = directoryService;
+    }
+
+
+    /*
+     * (non-Javadoc)
+     * @see org.apache.mina.filter.codec.ProtocolCodecFactory#getEncoder()
+     */
+    public ProtocolEncoder getEncoder()
+    {
+        return new Asn1CodecEncoder( new MessageEncoder() );
+    }
+
+
+    /*
+     * (non-Javadoc)
+     * @see org.apache.mina.filter.codec.ProtocolCodecFactory#getDecoder()
+     */
+    public ProtocolDecoder getDecoder()
+    {
+        return new Asn1CodecDecoder( new MessageDecoder( new BinaryAttributeDetector()
+        {
+            public boolean isBinary( String id )
+            {
+                AttributeTypeRegistry attrRegistry = directoryService.getRegistries().getAttributeTypeRegistry();
+                try
+                {
+                    AttributeType type = attrRegistry.lookup( id );
+                    return ! type.getSyntax().isHumanReadable();
+                }
+                catch ( Exception e )
+                {
+                    return false;
+                }
+            }
+        }) );
+    }
+}
\ No newline at end of file

Added: directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapProtocolHandler.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapProtocolHandler.java?rev=688548&view=auto
==============================================================================
--- directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapProtocolHandler.java (added)
+++ directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapProtocolHandler.java Sun Aug 24 12:07:01 2008
@@ -0,0 +1,204 @@
+/*
+ *   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.ldap;
+
+
+import javax.naming.ldap.Control;
+
+import org.apache.directory.shared.ldap.message.ExtendedRequest;
+import org.apache.directory.shared.ldap.message.ExtendedRequestImpl;
+import org.apache.directory.shared.ldap.message.MutableControl;
+import org.apache.directory.shared.ldap.message.Request;
+import org.apache.directory.shared.ldap.message.ResponseCarryingMessageException;
+import org.apache.directory.shared.ldap.message.ResultCodeEnum;
+import org.apache.directory.shared.ldap.message.ResultResponse;
+import org.apache.directory.shared.ldap.message.ResultResponseRequest;
+import org.apache.directory.shared.ldap.message.extended.NoticeOfDisconnect;
+import org.apache.mina.common.IoFilterChain;
+import org.apache.mina.common.IoHandler;
+import org.apache.mina.common.IoSession;
+import org.apache.mina.filter.SSLFilter;
+import org.apache.mina.filter.codec.ProtocolCodecFilter;
+import org.apache.mina.handler.demux.DemuxingIoHandler;
+import org.apache.mina.util.SessionLog;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The MINA IoHandler implementation extending {@link DemuxingIoHandler} for 
+ * the LDAP protocol.  THe {@link LdapServer} creates this multiplexing 
+ * {@link IoHandler} handler and populates it with subordinate handlers for
+ * the various kinds of LDAP {@link Request} messages.  This is done in the
+ * setXxxHandler() methods of the LdapServer where Xxxx is Add, Modify, etc.
+ *
+ * @author <a href="mailto:dev@directory.apache.org">Apache Directory Project</a>
+ * @version $Rev$, $Date$
+ */
+class LdapProtocolHandler extends DemuxingIoHandler
+{
+    /** The logger */
+    private static final Logger LOG = LoggerFactory.getLogger( LdapProtocolHandler.class );
+    
+    /** the {@link LdapServer} this handler is associated with */
+    private final LdapServer ldapServer;
+
+
+    /**
+     * Creates a new instance of LdapProtocolHandler.
+     *
+     * @param ldapServer
+     */
+    LdapProtocolHandler( LdapServer ldapServer )
+    {
+        this.ldapServer = ldapServer;
+    }
+
+
+    /*
+     * (non-Javadoc)
+     * @see org.apache.mina.common.IoHandlerAdapter#sessionCreated(org.apache.mina.common.IoSession)
+     */
+    public void sessionCreated( IoSession session ) throws Exception
+    {
+        LdapSession ldapSession = new LdapSession( session );
+        IoFilterChain filters = session.getFilterChain();
+        filters.addLast( "codec", new ProtocolCodecFilter( ldapServer.getProtocolCodecFactory() ) );
+        ldapServer.getLdapSessionManager().addLdapSession( ldapSession );
+    }
+
+
+    /*
+     * (non-Javadoc)
+     * @see org.apache.mina.common.IoHandlerAdapter#sessionClosed(org.apache.mina.common.IoSession)
+     */
+    public void sessionClosed( IoSession session )
+    {
+        LdapSession ldapSession = ldapServer.getLdapSessionManager().removeLdapSession( session );
+        cleanUpSession( ldapSession );
+    }
+
+    
+    /**
+     * Explicitly handles {@link LdapSession} and {@link IoSession} cleanup tasks.
+     *
+     * @param ldapSession the LdapSession to cleanup after being removed from 
+     * the {@link LdapSessionManager}
+     */
+    private void cleanUpSession( LdapSession ldapSession )
+    {
+        if ( ldapSession != null )
+        {
+            ldapSession.abandonAllOutstandingRequests();
+        }
+        
+        if ( ! ldapSession.getIoSession().isClosing() || ldapSession.getIoSession().isConnected() )
+        {
+            try
+            {
+                ldapSession.getIoSession().close();
+            }
+            catch ( Throwable t )
+            {
+                LOG.warn( "Failed to close IoSession for LdapSession." );
+            }
+        }
+    }
+    
+
+    /*
+     * (non-Javadoc)
+     * @see org.apache.mina.handler.demux.DemuxingIoHandler#messageReceived(org.apache.mina.common.IoSession, java.lang.Object)
+     */
+    public void messageReceived( IoSession session, Object message ) throws Exception
+    {
+        // Translate SSLFilter messages into LDAP extended request
+        // defined in RFC #2830, 'Lightweight Directory Access Protocol (v3):
+        // Extension for Transport Layer Security'.
+        // 
+        // The RFC specifies the payload should be empty, but we use
+        // it to notify the TLS state changes.  This hack should be
+        // OK from the viewpointd of security because StartTLS
+        // handler should react to only SESSION_UNSECURED message
+        // and degrade authentication level to 'anonymous' as specified
+        // in the RFC, and this is no threat.
+
+        if ( message == SSLFilter.SESSION_SECURED )
+        {
+            ExtendedRequest req = new ExtendedRequestImpl( 0 );
+            req.setOid( "1.3.6.1.4.1.1466.20037" );
+            req.setPayload( "SECURED".getBytes( "ISO-8859-1" ) );
+            message = req;
+        }
+        else if ( message == SSLFilter.SESSION_UNSECURED )
+        {
+            ExtendedRequest req = new ExtendedRequestImpl( 0 );
+            req.setOid( "1.3.6.1.4.1.1466.20037" );
+            req.setPayload( "UNSECURED".getBytes( "ISO-8859-1" ) );
+            message = req;
+        }
+
+        if ( ( ( Request ) message ).getControls().size() > 0 && message instanceof ResultResponseRequest )
+        {
+            ResultResponseRequest req = ( ResultResponseRequest ) message;
+            for ( Control control1 : req.getControls().values() )
+            {
+                MutableControl control = ( MutableControl ) control1;
+                if ( control.isCritical() && ! ldapServer.getSupportedControls().contains( control.getID() ) )
+                {
+                    ResultResponse resp = req.getResultResponse();
+                    resp.getLdapResult().setErrorMessage( "Unsupport critical control: " + control.getID() );
+                    resp.getLdapResult().setResultCode( ResultCodeEnum.UNAVAILABLE_CRITICAL_EXTENSION );
+                    session.write( resp );
+                    return;
+                }
+            }
+        }
+
+        super.messageReceived( session, message );
+    }
+
+
+    /*
+     * (non-Javadoc)
+     * @see org.apache.mina.common.IoHandlerAdapter#exceptionCaught(org.apache.mina.common.IoSession, java.lang.Throwable)
+     */
+    public void exceptionCaught( IoSession session, Throwable cause )
+    {
+        if ( cause.getCause() instanceof ResponseCarryingMessageException )
+        {
+            ResponseCarryingMessageException rcme = ( ResponseCarryingMessageException ) cause.getCause();
+
+            if ( rcme.getResponse() != null )
+            {
+                session.write( rcme.getResponse() );
+                return;
+            }                
+        }
+        
+        SessionLog.warn( session,
+            "Unexpected exception forcing session to close: sending disconnect notice to client.", cause );
+
+        session.write( NoticeOfDisconnect.PROTOCOLERROR );
+        LdapSession ldapSession = this.ldapServer.getLdapSessionManager().removeLdapSession( session );
+        cleanUpSession( ldapSession );
+        session.close();
+    }
+}
\ No newline at end of file

Modified: directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapServer.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapServer.java?rev=688548&r1=688547&r2=688548&view=diff
==============================================================================
--- directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapServer.java (original)
+++ directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapServer.java Sun Aug 24 12:07:01 2008
@@ -26,9 +26,7 @@
 import java.security.Provider;
 import java.security.Security;
 import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
 
-import javax.naming.ldap.Control;
 
 import org.apache.directory.server.core.DirectoryService;
 import org.apache.directory.server.core.partition.PartitionNexus;
@@ -47,9 +45,6 @@
 import org.apache.directory.server.ldap.handlers.bind.*;
 import org.apache.directory.server.ldap.handlers.ssl.LdapsInitializer;
 import org.apache.directory.server.protocol.shared.DirectoryBackedService;
-import org.apache.directory.server.schema.registries.AttributeTypeRegistry;
-import org.apache.directory.shared.asn1.codec.Asn1CodecDecoder;
-import org.apache.directory.shared.asn1.codec.Asn1CodecEncoder;
 import org.apache.directory.shared.ldap.constants.SaslQoP;
 import org.apache.directory.shared.ldap.exception.LdapConfigurationException;
 import org.apache.directory.shared.ldap.message.AbandonRequest;
@@ -60,41 +55,23 @@
 import org.apache.directory.shared.ldap.message.DeleteRequest;
 import org.apache.directory.shared.ldap.message.EntryChangeControl;
 import org.apache.directory.shared.ldap.message.ExtendedRequest;
-import org.apache.directory.shared.ldap.message.ExtendedRequestImpl;
 import org.apache.directory.shared.ldap.message.ManageDsaITControl;
-import org.apache.directory.shared.ldap.message.MessageDecoder;
-import org.apache.directory.shared.ldap.message.MessageEncoder;
 import org.apache.directory.shared.ldap.message.ModifyDnRequest;
 import org.apache.directory.shared.ldap.message.ModifyRequest;
-import org.apache.directory.shared.ldap.message.MutableControl;
 import org.apache.directory.shared.ldap.message.PersistentSearchControl;
-import org.apache.directory.shared.ldap.message.Request;
-import org.apache.directory.shared.ldap.message.ResponseCarryingMessageException;
-import org.apache.directory.shared.ldap.message.ResultCodeEnum;
-import org.apache.directory.shared.ldap.message.ResultResponse;
-import org.apache.directory.shared.ldap.message.ResultResponseRequest;
 import org.apache.directory.shared.ldap.message.SearchRequest;
 import org.apache.directory.shared.ldap.message.SubentriesControl;
 import org.apache.directory.shared.ldap.message.UnbindRequest;
 import org.apache.directory.shared.ldap.message.extended.NoticeOfDisconnect;
-import org.apache.directory.shared.ldap.message.spi.BinaryAttributeDetector;
-import org.apache.directory.shared.ldap.schema.AttributeType;
 import org.apache.mina.common.DefaultIoFilterChainBuilder;
-import org.apache.mina.common.IoFilterChain;
 import org.apache.mina.common.IoFilterChainBuilder;
 import org.apache.mina.common.IoHandler;
 import org.apache.mina.common.IoSession;
 import org.apache.mina.common.ThreadModel;
 import org.apache.mina.common.WriteFuture;
-import org.apache.mina.filter.SSLFilter;
 import org.apache.mina.filter.codec.ProtocolCodecFactory;
-import org.apache.mina.filter.codec.ProtocolCodecFilter;
-import org.apache.mina.filter.codec.ProtocolDecoder;
-import org.apache.mina.filter.codec.ProtocolEncoder;
-import org.apache.mina.handler.demux.DemuxingIoHandler;
 import org.apache.mina.handler.demux.MessageHandler;
 import org.apache.mina.transport.socket.nio.SocketAcceptorConfig;
-import org.apache.mina.util.SessionLog;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -109,6 +86,17 @@
  */
 public class LdapServer extends DirectoryBackedService
 {
+    /** Value (0) for configuration where size limit is unlimited. */
+    public static final int NO_SIZE_LIMIT = 0;
+
+    /** Value (0) for configuration where time limit is unlimited. */
+    public static final int NO_TIME_LIMIT = 0;
+
+    /** the constant service name of this ldap protocol provider **/
+    public static final String SERVICE_NAME = "ldap";
+
+    
+    
     private static final long serialVersionUID = 3757127143811666817L;
 
     /** logger for this class */
@@ -120,12 +108,6 @@
     /** The default maximum time limit. */
     private static final int MAX_TIME_LIMIT_DEFAULT = 10000;
 
-    /** Value (0) for configuration where size limit is unlimited. */
-    public static final int NO_SIZE_LIMIT = 0;
-
-    /** Value (0) for configuration where time limit is unlimited. */
-    public static final int NO_TIME_LIMIT = 0;
-
     /** The default service pid. */
     private static final String SERVICE_PID_DEFAULT = "org.apache.directory.server.ldap";
 
@@ -135,10 +117,8 @@
     /** The default IP port. */
     private static final int IP_PORT_DEFAULT = 389;
 
-    /** the constant service name of this ldap protocol provider **/
-    public static final String SERVICE_NAME = "ldap";
-
-    
+    /** the session manager for this LdapServer */
+    private LdapSessionManager ldapSessionManager = new LdapSessionManager();
     
     /** a set of supported controls */
     private Set<String> supportedControls;
@@ -198,7 +178,7 @@
     private ProtocolCodecFactory codecFactory;
 
     /** the MINA protocol handler */
-    private final LdapProtocolHandler handler = new LdapProtocolHandler();
+    private final LdapProtocolHandler handler = new LdapProtocolHandler(this);
 
     /** tracks start state of the server */
     private boolean started;
@@ -453,18 +433,24 @@
     }
 
 
-    public ProtocolCodecFactory getCodecFactory()
-    {
-        return codecFactory;
-    }
-
-
     public IoHandler getHandler()
     {
         return handler;
     }
+    
+    
+    public LdapSessionManager getLdapSessionManager()
+    {
+        return ldapSessionManager;
+    }
+    
+    
+    public ProtocolCodecFactory getProtocolCodecFactory()
+    {
+        return codecFactory;
+    }
 
-
+    
     // ------------------------------------------------------------------------
     // Configuration Methods
     // ------------------------------------------------------------------------
@@ -842,7 +828,7 @@
     public void setDirectoryService( DirectoryService directoryService )
     {
         super.setDirectoryService( directoryService );
-        this.codecFactory = new ProtocolCodecFactoryImpl( directoryService );
+        this.codecFactory = new LdapProtocolCodecFactory( directoryService );
     }
 
 
@@ -1018,184 +1004,4 @@
     {
         this.started = started;
     }
-
-
-    /**
-     * A snickers based BER Decoder factory.
-     */
-    private static final class ProtocolCodecFactoryImpl implements ProtocolCodecFactory
-    {
-        final DirectoryService directoryService;
-
-
-        public ProtocolCodecFactoryImpl( DirectoryService directoryService )
-        {
-            this.directoryService = directoryService;
-        }
-
-
-        public ProtocolEncoder getEncoder()
-        {
-            return new Asn1CodecEncoder( new MessageEncoder() );
-        }
-
-
-        public ProtocolDecoder getDecoder()
-        {
-            return new Asn1CodecDecoder( new MessageDecoder( new BinaryAttributeDetector()
-            {
-                public boolean isBinary( String id )
-                {
-                    AttributeTypeRegistry attrRegistry = directoryService.getRegistries().getAttributeTypeRegistry();
-                    try
-                    {
-                        AttributeType type = attrRegistry.lookup( id );
-                        return ! type.getSyntax().isHumanReadable();
-                    }
-                    catch ( Exception e )
-                    {
-                        return false;
-                    }
-                }
-            }) );
-        }
-    }
-    
-    
-    Map<IoSession, LdapSession> ldapSessions = new ConcurrentHashMap<IoSession, LdapSession>( 100 );
-
-    
-    public LdapSession[] getSessions()
-    {
-        return ldapSessions.values().toArray( new LdapSession[0] );
-    }
-    
-    
-    public LdapSession removeLdapSession( IoSession session )
-    {
-        LdapSession ldapSession = null; 
-        
-        synchronized ( ldapSessions )
-        {
-            ldapSession = ldapSessions.remove( session );
-        }
-        
-        if ( ldapSession != null )
-        {
-            ldapSession.abandonAllOutstandingRequests();
-        }
-        
-        if ( ! session.isClosing() || session.isConnected() )
-        {
-            try
-            {
-                session.close();
-            }
-            catch ( Throwable t )
-            {
-                LOG.warn( "Failed to close IoSession for LdapSession." );
-            }
-        }
-        
-        return ldapSession;
-    }
-    
-    
-    public LdapSession getLdapSession( IoSession session )
-    {
-        return ldapSessions.get( session );
-    }
-    
-    
-    private class LdapProtocolHandler extends DemuxingIoHandler
-    {
-        public void sessionCreated( IoSession session ) throws Exception
-        {
-            LdapSession ldapSession = new LdapSession( session );
-            IoFilterChain filters = session.getFilterChain();
-            filters.addLast( "codec", new ProtocolCodecFilter( codecFactory ) );
-            
-            synchronized( ldapSessions )
-            {
-                ldapSessions.put( session, ldapSession );
-            }
-        }
-
-
-        public void sessionClosed( IoSession session )
-        {
-            removeLdapSession( session );
-        }
-
-
-        public void messageReceived( IoSession session, Object message ) throws Exception
-        {
-            // Translate SSLFilter messages into LDAP extended request
-            // defined in RFC #2830, 'Lightweight Directory Access Protocol (v3):
-            // Extension for Transport Layer Security'.
-            // 
-            // The RFC specifies the payload should be empty, but we use
-            // it to notify the TLS state changes.  This hack should be
-            // OK from the viewpointd of security because StartTLS
-            // handler should react to only SESSION_UNSECURED message
-            // and degrade authentication level to 'anonymous' as specified
-            // in the RFC, and this is no threat.
-
-            if ( message == SSLFilter.SESSION_SECURED )
-            {
-                ExtendedRequest req = new ExtendedRequestImpl( 0 );
-                req.setOid( "1.3.6.1.4.1.1466.20037" );
-                req.setPayload( "SECURED".getBytes( "ISO-8859-1" ) );
-                message = req;
-            }
-            else if ( message == SSLFilter.SESSION_UNSECURED )
-            {
-                ExtendedRequest req = new ExtendedRequestImpl( 0 );
-                req.setOid( "1.3.6.1.4.1.1466.20037" );
-                req.setPayload( "UNSECURED".getBytes( "ISO-8859-1" ) );
-                message = req;
-            }
-
-            if ( ( ( Request ) message ).getControls().size() > 0 && message instanceof ResultResponseRequest )
-            {
-                ResultResponseRequest req = ( ResultResponseRequest ) message;
-                for ( Control control1 : req.getControls().values() )
-                {
-                    MutableControl control = ( MutableControl ) control1;
-                    if ( control.isCritical() && !supportedControls.contains( control.getID() ) )
-                    {
-                        ResultResponse resp = req.getResultResponse();
-                        resp.getLdapResult().setErrorMessage( "Unsupport critical control: " + control.getID() );
-                        resp.getLdapResult().setResultCode( ResultCodeEnum.UNAVAILABLE_CRITICAL_EXTENSION );
-                        session.write( resp );
-                        return;
-                    }
-                }
-            }
-
-            super.messageReceived( session, message );
-        }
-
-
-        public void exceptionCaught( IoSession session, Throwable cause )
-        {
-            if ( cause.getCause() instanceof ResponseCarryingMessageException )
-            {
-                ResponseCarryingMessageException rcme = ( ResponseCarryingMessageException ) cause.getCause();
-
-                if ( rcme.getResponse() != null )
-                {
-                    session.write( rcme.getResponse() );
-                    return;
-                }                
-            }
-            
-            SessionLog.warn( session,
-                "Unexpected exception forcing session to close: sending disconnect notice to client.", cause );
-
-            session.write( NoticeOfDisconnect.PROTOCOLERROR );
-            removeLdapSession( session );
-            session.close();
-        }
-    }
 }

Added: directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapSessionManager.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapSessionManager.java?rev=688548&view=auto
==============================================================================
--- directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapSessionManager.java (added)
+++ directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/LdapSessionManager.java Sun Aug 24 12:07:01 2008
@@ -0,0 +1,98 @@
+/*
+ *   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.ldap;
+
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.mina.common.IoSession;
+
+
+/**
+ * Manages sessions in a thread safe manner for the LdapServer.  This class is 
+ * used primarily by the {@link LdapProtocolHandler} to manage sessions and is
+ * created by the LdapServer which makes it available to the handler.  It's job
+ * is simple and this class was mainly created to be able to expose the session
+ * manager safely to things like the LdapProtocolHandler.
+ *
+ * @author <a href="mailto:dev@directory.apache.org">Apache Directory Project</a>
+ * @version $Rev$, $Date$
+ */
+public class LdapSessionManager
+{
+    /** Concurrent hashMap backing for IoSession to LdapSession mapping */
+    private Map<IoSession, LdapSession> ldapSessions = new ConcurrentHashMap<IoSession, LdapSession>( 100 );
+
+    
+    /**
+     * Gets the active sessions managed by the LdapServer.
+     */
+    public LdapSession[] getSessions()
+    {
+        return ldapSessions.values().toArray( new LdapSession[0] );
+    }
+    
+
+    /**
+     * Adds a new LdapSession to the LdapServer.
+     *
+     * @param ldapSession the newly created {@link LdapSession}
+     */
+    public void addLdapSession( LdapSession ldapSession )
+    {
+        synchronized ( ldapSessions )
+        {
+            ldapSessions.put( ldapSession.getIoSession(), ldapSession );
+        }
+    }
+    
+    
+    /**
+     * Removes an LdapSession managed by the {@link LdapServer}.  This method
+     * has no side effects: meaning it does not perform cleanup tasks after
+     * removing the session.  This task is handled by the callers.
+     *
+     * @param session the MINA session of the LdapSession to be removed 
+     * @return the LdapSession to remove
+     */
+    public LdapSession removeLdapSession( IoSession session )
+    {
+        synchronized ( ldapSessions )
+        {
+            return ldapSessions.remove( session );
+        }
+    }
+    
+    
+    /**
+     * Gets the LdapSession associated with the MINA session.
+     *
+     * @param session the MINA session of the LdapSession to retrieve
+     * @return the LdapSession associated with the MINA {@link IoSession}
+     */
+    public LdapSession getLdapSession( IoSession session )
+    {
+        synchronized ( ldapSessions )
+        {
+            return ldapSessions.get( session );
+        }
+    }
+}

Modified: directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/gui/SessionsFrame.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/gui/SessionsFrame.java?rev=688548&r1=688547&r2=688548&view=diff
==============================================================================
--- directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/gui/SessionsFrame.java (original)
+++ directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/gui/SessionsFrame.java Sun Aug 24 12:07:01 2008
@@ -191,7 +191,7 @@
         {
             sessionsTable = new JTable();
             sessionsTable.setSelectionMode( javax.swing.ListSelectionModel.SINGLE_SELECTION );
-            sessionsTable.setModel( new SessionsModel( ldapServer.getSessions() ) );
+            sessionsTable.setModel( new SessionsModel( ldapServer.getLdapSessionManager().getSessions() ) );
             sessionsTable.getSelectionModel().addListSelectionListener( new ListSelectionListener()
             {
                 public void valueChanged( ListSelectionEvent e )
@@ -363,7 +363,7 @@
             {
                 public void actionPerformed( java.awt.event.ActionEvent e )
                 {
-                    ldapServer.removeLdapSession( selected.getIoSession() );
+                    ldapServer.getLdapSessionManager().removeLdapSession( selected.getIoSession() );
                     try
                     {
                         Thread.sleep( 250 );
@@ -691,7 +691,7 @@
     private void refresh()
     {
         LOG.info( "Refreshing Sessions UI" );
-        sessionsTable.setModel( new SessionsModel( ldapServer.getSessions() ) );
+        sessionsTable.setModel( new SessionsModel( ldapServer.getLdapSessionManager().getSessions() ) );
         closeItem.setEnabled( false );
         menuSendNoD.setEnabled( false );
         showRequests.setEnabled( false );

Modified: directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/handlers/LdapRequestHandler.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/handlers/LdapRequestHandler.java?rev=688548&r1=688547&r2=688548&view=diff
==============================================================================
--- directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/handlers/LdapRequestHandler.java (original)
+++ directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/handlers/LdapRequestHandler.java Sun Aug 24 12:07:01 2008
@@ -116,7 +116,7 @@
      */
     public final void messageReceived( IoSession session, T message ) throws Exception
     {
-        LdapSession ldapSession = ldapServer.getLdapSession( session );
+        LdapSession ldapSession = ldapServer.getLdapSessionManager().getLdapSession( session );
         
         // TODO - session you get from LdapServer should have the ldapServer 
         // member already set no?  Should remove these lines where ever they

Modified: directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/handlers/UnbindHandler.java
URL: http://svn.apache.org/viewvc/directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/handlers/UnbindHandler.java?rev=688548&r1=688547&r2=688548&view=diff
==============================================================================
--- directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/handlers/UnbindHandler.java (original)
+++ directory/apacheds/trunk/protocol-ldap/src/main/java/org/apache/directory/server/ldap/handlers/UnbindHandler.java Sun Aug 24 12:07:01 2008
@@ -45,7 +45,7 @@
         {
             session.getCoreSession().unbind( request );
             session.getIoSession().close();
-            ldapServer.removeLdapSession( session.getIoSession() );
+            ldapServer.getLdapSessionManager().removeLdapSession( session.getIoSession() );
         }
         catch ( Throwable t )
         {