You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@directory.apache.org by tr...@apache.org on 2004/12/24 01:34:25 UTC

svn commit: r123258 - in incubator/directory/network/trunk/mina/src: java/org/apache/mina/common java/org/apache/mina/io java/org/apache/mina/io/datagram java/org/apache/mina/io/filter java/org/apache/mina/io/socket java/org/apache/mina/protocol java/org/apache/mina/protocol/filter java/org/apache/mina/util test/org test/org/apache test/org/apache/mina test/org/apache/mina/examples test/org/apache/mina/examples/echoserver

Author: trustin
Date: Thu Dec 23 16:34:23 2004
New Revision: 123258

URL: http://svn.apache.org/viewcvs?view=rev&rev=123258
Log:
 * Added ExceptionMonitor to monitor uncaught exceptions from Acceptor and Connector
 * Added missing license statements
Added:
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/common/ExceptionMonitor.java   (contents, props changed)
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramAcceptor.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramAcceptorSession.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramReadBuffer.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramSessionConfig.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramWriteBuffer.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/DefaultExceptionMonitor.java   (contents, props changed)
   incubator/directory/network/trunk/mina/src/test/org/
   incubator/directory/network/trunk/mina/src/test/org/apache/
   incubator/directory/network/trunk/mina/src/test/org/apache/mina/
   incubator/directory/network/trunk/mina/src/test/org/apache/mina/examples/
   incubator/directory/network/trunk/mina/src/test/org/apache/mina/examples/echoserver/
   incubator/directory/network/trunk/mina/src/test/org/apache/mina/examples/echoserver/Test.java
Modified:
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/common/IdleStatus.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/Acceptor.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/Connector.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/IoHandlerAdapter.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/IoHandlerFilterAdapter.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/filter/BlacklistFilter.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/filter/IoThreadPoolFilter.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/socket/TcpAcceptor.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/socket/TcpConnector.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/ProtocolHandlerAdapter.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/ProtocolHandlerFilterAdapter.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/filter/ProtocolThreadPoolFilter.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/AbstractWriteBuffer.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/BlockingSet.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/IoHandlerFilterManager.java
   incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/ProtocolHandlerFilterManager.java

Added: incubator/directory/network/trunk/mina/src/java/org/apache/mina/common/ExceptionMonitor.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/common/ExceptionMonitor.java?view=auto&rev=123258
==============================================================================
--- (empty file)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/common/ExceptionMonitor.java	Thu Dec 23 16:34:23 2004
@@ -0,0 +1,30 @@
+/*
+ *   @(#) $Id$
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.common;
+
+/**
+ * TODO Document me.
+ * 
+ * @author Trustin Lee (trustin@apache.org)
+ * @version $Rev$, $Date$
+ */
+public interface ExceptionMonitor
+{
+    void exceptionCaught(Object source, Throwable cause);
+}

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/common/IdleStatus.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/common/IdleStatus.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/common/IdleStatus.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/common/IdleStatus.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/common/IdleStatus.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/common/IdleStatus.java	Thu Dec 23 16:34:23 2004
@@ -1,15 +1,20 @@
 /*
- * Copyright 2004 The Apache Software Foundation Licensed 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.
- */
-/*
- * @(#) $Id$
+ *   @(#) $Id$
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.common;
 

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/Acceptor.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/Acceptor.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/Acceptor.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/Acceptor.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/Acceptor.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/Acceptor.java	Thu Dec 23 16:34:23 2004
@@ -21,6 +21,8 @@
 import java.io.IOException;
 import java.net.SocketAddress;
 
+import org.apache.mina.common.ExceptionMonitor;
+
 /**
  * TODO Insert type comment.
  * 
@@ -37,4 +39,8 @@
     void addFilter( int priority, IoHandlerFilter filter );
 
     void removeFilter( IoHandlerFilter filter );
+    
+    ExceptionMonitor getExceptionMonitor();
+    
+    void setExceptionMonitor(ExceptionMonitor monitor);
 }

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/Connector.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/Connector.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/Connector.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/Connector.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/Connector.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/Connector.java	Thu Dec 23 16:34:23 2004
@@ -21,6 +21,8 @@
 import java.io.IOException;
 import java.net.SocketAddress;
 
+import org.apache.mina.common.ExceptionMonitor;
+
 /**
  * TODO Insert type comment.
  * 
@@ -38,4 +40,8 @@
     void addFilter( int priority, IoHandlerFilter filter );
 
     void removeFilter( IoHandlerFilter filter );
+
+    ExceptionMonitor getExceptionMonitor();
+    
+    void setExceptionMonitor(ExceptionMonitor monitor);
 }

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/IoHandlerAdapter.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/IoHandlerAdapter.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/IoHandlerAdapter.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/IoHandlerAdapter.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/IoHandlerAdapter.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/IoHandlerAdapter.java	Thu Dec 23 16:34:23 2004
@@ -1,5 +1,20 @@
 /*
- * @(#) $Id$
+ *   @(#) $Id$
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.io;
 

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/IoHandlerFilterAdapter.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/IoHandlerFilterAdapter.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/IoHandlerFilterAdapter.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/IoHandlerFilterAdapter.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/IoHandlerFilterAdapter.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/IoHandlerFilterAdapter.java	Thu Dec 23 16:34:23 2004
@@ -1,5 +1,20 @@
 /*
- * @(#) $Id$
+ *   @(#) $Id$
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.io;
 

Added: incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramAcceptor.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramAcceptor.java?view=auto&rev=123258
==============================================================================
--- (empty file)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramAcceptor.java	Thu Dec 23 16:34:23 2004
@@ -0,0 +1,435 @@
+/*
+ *   @(#) $Id: TcpAcceptor.java 122627 2004-12-17 07:49:42Z trustin $
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.io.datagram;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.DatagramChannel;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.mina.io.Acceptor;
+import org.apache.mina.io.IoHandler;
+import org.apache.mina.io.IoHandlerFilter;
+import org.apache.mina.util.IoHandlerFilterManager;
+import org.apache.mina.util.Queue;
+
+/**
+ * TODO Insert type comment.
+ * 
+ * @author Trustin Lee (trustin@apache.org)
+ * @version $Rev: 122627 $, $Date: 2004-12-17 16:49:42 +0900 (�?, 17 12 2004) $
+ */
+public class DatagramAcceptor implements Acceptor
+{
+    private static volatile int nextId = 0;
+
+    private final int id = nextId++;
+
+    private final Selector selector;
+
+    private final Map channels = new HashMap();
+
+    private final Queue flushingSessions = new Queue();
+
+    private final IoHandlerFilterManager filterManager = new IoHandlerFilterManager();
+
+    private Worker worker;
+
+    /**
+     * Creates a new instance.
+     * 
+     * @throws IOException
+     */
+    public DatagramAcceptor() throws IOException
+    {
+        selector = Selector.open();
+    }
+
+    public void addFilter( int priority, IoHandlerFilter filter )
+    {
+        filterManager.addFilter( priority, filter );
+    }
+
+    public void removeFilter( IoHandlerFilter filter )
+    {
+        filterManager.removeFilter( filter );
+    }
+
+    public synchronized void bind( SocketAddress address,
+                                  IoHandler defaultHandler )
+            throws IOException
+    {
+        if( address == null )
+            throw new NullPointerException( "address" );
+        if( defaultHandler == null )
+            throw new NullPointerException( "defaultHandler" );
+
+        if( ! ( address instanceof InetSocketAddress ) )
+            throw new IllegalArgumentException( "Unexpected address type: "
+                                                + address.getClass() );
+
+        DatagramChannel dgc = DatagramChannel.open();
+        dgc.socket().bind( address );
+        dgc.configureBlocking( false );
+        dgc.register( selector, SelectionKey.OP_READ, defaultHandler );
+
+        channels.put( address, dgc );
+
+        if( worker == null )
+        {
+            worker = new Worker();
+            worker.start();
+        }
+    }
+
+    public synchronized void unbind( SocketAddress address )
+    {
+        if( address == null )
+            throw new NullPointerException( "address" );
+
+        DatagramChannel dgc = ( DatagramChannel ) channels.get( address );
+
+        if( dgc == null )
+            throw new IllegalArgumentException( "Unknown address: " + address );
+
+        SelectionKey key = dgc.keyFor( selector );
+        key.cancel();
+        channels.remove( address );
+
+        try
+        {
+            dgc.close();
+        }
+        catch( IOException e )
+        {
+            e.printStackTrace();
+        }
+    }
+
+    void flushSession( DatagramAcceptorSession session )
+    {
+        scheduleFlush( session );
+        selector.wakeup();
+    }
+
+    private void scheduleFlush( DatagramAcceptorSession session )
+    {
+        synchronized( flushingSessions )
+        {
+            flushingSessions.push( session );
+        }
+    }
+
+    private class Worker extends Thread
+    {
+        public Worker()
+        {
+            super( "DatagramAcceptor-" + id );
+        }
+
+        public void run()
+        {
+            for( ;; )
+            {
+                try
+                {
+                    int nKeys = selector.select();
+
+                    if( nKeys > 0 )
+                    {
+                        processSessions( selector.selectedKeys() );
+                    }
+
+                    flushSessions();
+                }
+                catch( IOException e )
+                {
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
+
+    private void processSessions( Set keys ) throws IOException
+    {
+        Iterator it = keys.iterator();
+
+        while( it.hasNext() )
+        {
+            SelectionKey key = ( SelectionKey ) it.next();
+            // TODO pool sessions
+            DatagramAcceptorSession session = new DatagramAcceptorSession(
+                                                                           this,
+                                                                           ( DatagramChannel ) key
+                                                                                   .channel(),
+                                                                           ( IoHandler ) key
+                                                                                   .attachment() );
+            it.remove();
+
+            if( key.isReadable() )
+            {
+                read( session );
+            }
+
+            if( key.isWritable() )
+            {
+                scheduleFlush( session );
+            }
+        }
+    }
+
+    private void read( DatagramAcceptorSession session ) throws IOException
+    {
+        DatagramChannel dgc = session.getChannel();
+        SocketAddress remoteAddress = null;
+        DatagramReadBuffer lock = ( DatagramReadBuffer ) session
+                .getReadBuffer();
+        ByteBuffer buf = lock.asByteBuffer();
+        boolean received = false;
+        try
+        {
+            synchronized( lock )
+            {
+                buf.compact();
+                remoteAddress = dgc.receive( buf );
+                received = remoteAddress != null;
+
+                if( received )
+                {
+                    int readBytes = buf.position();
+                    buf.flip();
+
+                    if( readBytes > 0 )
+                    {
+                        buf.mark();
+                        lock.markBaseIndex();
+                        session.setRemoteAddress( remoteAddress );
+                        session.increaseReadBytes( readBytes );
+                        fireDataRead( session );
+                    }
+                }
+            }
+        }
+        finally
+        {
+            if( !received )
+            {
+                session.dispose();
+            }
+        }
+    }
+
+    private void flushSessions()
+    {
+        if( flushingSessions.size() == 0 )
+            return;
+
+        for( ;; )
+        {
+            DatagramAcceptorSession session;
+
+            synchronized( flushingSessions )
+            {
+                session = ( DatagramAcceptorSession ) flushingSessions.pop();
+            }
+
+            if( session == null )
+                break;
+
+            if( session.isClosed() )
+                continue;
+
+            flush( session );
+        }
+    }
+
+    private void flush( DatagramAcceptorSession session )
+    {
+        DatagramWriteBuffer lock = ( DatagramWriteBuffer ) session
+                .getWriteBuffer();
+        ByteBuffer writeBuf = lock.buf();
+        DatagramChannel ch = session.getChannel();
+
+        try
+        {
+            synchronized( lock )
+            {
+                writeBuf.flip();
+
+                // ignore empty write buffer
+                if( writeBuf.remaining() == 0 )
+                {
+                    writeBuf.clear();
+
+                    // clear user markers
+                    Queue markers = lock.getMarkers();
+                    for( ;; )
+                    {
+                        DatagramWriteBuffer.Marker marker = ( DatagramWriteBuffer.Marker ) markers
+                                .pop();
+                        if( marker == null )
+                            break;
+
+                        fireMarkerRemoved( session, marker.getValue() );
+                    }
+
+                    // clear flush markers
+                    Queue flushMarkers = lock.getMarkers();
+                    for( ;; )
+                    {
+                        DatagramWriteBuffer.Marker marker = ( DatagramWriteBuffer.Marker ) flushMarkers
+                                .pop();
+                        if( marker == null )
+                            break;
+                    }
+                }
+                else
+                {
+                    int writtenBytes = 0;
+
+                    try
+                    {
+                        writtenBytes = writeBuf.remaining();
+                        int actualWrittenBytes = ch.send( writeBuf, session
+                                .getRemoteAddress() );
+                        if( actualWrittenBytes == 0 )
+                        {
+                            writtenBytes = 0;
+                        }
+                    }
+                    finally
+                    {
+                        SelectionKey key = session.getSelectionKey();
+
+                        if( writtenBytes == 0 )
+                        {
+                            // Kernel buffer is full
+                            key.interestOps( key.interestOps()
+                                             | SelectionKey.OP_WRITE );
+                            writeBuf.compact();
+                        }
+                        else
+                        {
+                            key.interestOps( key.interestOps()
+                                             & ( ~SelectionKey.OP_WRITE ) );
+                            writeBuf.clear();
+                        }
+
+                        writeBuf.mark();
+                    }
+
+                    if( writtenBytes > 0 )
+                    {
+                        session.increaseWrittenBytes( writtenBytes );
+                        lock.markBaseIndex();
+                        fireDataWritten( session );
+                        Queue markers = lock.getMarkers();
+                        for( ;; )
+                        {
+                            DatagramWriteBuffer.Marker marker = ( DatagramWriteBuffer.Marker ) markers
+                                    .first();
+                            if( marker == null )
+                                break;
+
+                            int bytesLeft = marker.getBytesLeft();
+                            if( bytesLeft > writtenBytes )
+                            {
+                                marker.setBytesLeft( bytesLeft - writtenBytes );
+                                break;
+                            }
+                            else if( bytesLeft == writtenBytes )
+                            {
+                                markers.pop();
+                                fireMarkerRemoved( session, marker.getValue() );
+                                break;
+                            }
+                            else
+                            {
+                                markers.pop();
+                                fireMarkerRemoved( session, marker.getValue() );
+                                writtenBytes -= bytesLeft;
+                            }
+                        }
+                    }
+                }
+            }
+        }
+        catch( IOException e )
+        {
+            fireExceptionCaught( session, e );
+        }
+    }
+
+    private void fireDataRead( DatagramAcceptorSession session )
+    {
+        try
+        {
+            session.getHandler().dataRead( session );
+        }
+        catch( Throwable e )
+        {
+            fireExceptionCaught( session, e );
+        }
+    }
+
+    private void fireDataWritten( DatagramAcceptorSession session )
+    {
+        try
+        {
+            session.getHandler().dataWritten( session );
+        }
+        catch( Throwable e )
+        {
+            fireExceptionCaught( session, e );
+        }
+    }
+
+    private void fireMarkerRemoved( DatagramAcceptorSession session,
+                                   Object marker )
+    {
+        try
+        {
+            session.getHandler().markerReleased( session, marker );
+        }
+        catch( Throwable e )
+        {
+            fireExceptionCaught( session, e );
+        }
+    }
+
+    private void fireExceptionCaught( DatagramAcceptorSession session,
+                                     Throwable cause )
+    {
+        try
+        {
+            session.getHandler().exceptionCaught( session, cause );
+        }
+        catch( Throwable t )
+        {
+            t.printStackTrace();
+        }
+    }
+}
\ No newline at end of file

Added: incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramAcceptorSession.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramAcceptorSession.java?view=auto&rev=123258
==============================================================================
--- (empty file)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramAcceptorSession.java	Thu Dec 23 16:34:23 2004
@@ -0,0 +1,237 @@
+/*
+ *   @(#) $Id: TcpSession.java 122627 2004-12-17 07:49:42Z trustin $
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.io.datagram;
+
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.DatagramChannel;
+import java.nio.channels.SelectionKey;
+
+import org.apache.mina.common.IdleStatus;
+import org.apache.mina.common.SessionConfig;
+import org.apache.mina.io.IoHandler;
+import org.apache.mina.io.IoSession;
+import org.apache.mina.io.ReadBuffer;
+import org.apache.mina.io.WriteBuffer;
+import org.apache.mina.util.ByteBufferPool;
+
+/**
+ * TODO Insert type comment.
+ * 
+ * @author Trustin Lee (trustin@apache.org)
+ * @version $Rev: 122627 $, $Date: 2004-12-17 16:49:42 +0900 (�?, 17 12 2004) $
+ */
+class DatagramAcceptorSession implements IoSession
+{
+    private final DatagramAcceptor parent;
+
+    private final DatagramChannel ch;
+
+    private final DatagramSessionConfig config;
+
+    private final DatagramReadBuffer readBuf;
+
+    private final DatagramWriteBuffer writeBuf;
+
+    private final IoHandler handler;
+
+    private SocketAddress remoteAddress;
+
+    private SelectionKey key;
+
+    private Object attachment;
+
+    private long readBytes;
+
+    private long writtenBytes;
+
+    private long lastReadTime;
+
+    private long lastWriteTime;
+
+    private boolean idleForBoth;
+
+    private boolean idleForRead;
+
+    private boolean idleForWrite;
+
+    /**
+     * Creates a new instance.
+     */
+    DatagramAcceptorSession( DatagramAcceptor parent, DatagramChannel ch,
+                            IoHandler defaultHandler )
+    {
+        this.parent = parent;
+        this.ch = ch;
+        this.config = new DatagramSessionConfig( ch );
+        this.readBuf = new DatagramReadBuffer( ( ByteBuffer ) ByteBufferPool
+                .open().limit( 0 ) );
+        this.writeBuf = new DatagramWriteBuffer( this, ByteBufferPool.open() );
+        this.handler = defaultHandler;
+    }
+
+    DatagramChannel getChannel()
+    {
+        return ch;
+    }
+
+    public IoHandler getHandler()
+    {
+        return handler;
+    }
+
+    void setRemoteAddress( SocketAddress remoteAddress )
+    {
+        this.remoteAddress = remoteAddress;
+    }
+
+    SelectionKey getSelectionKey()
+    {
+        return key;
+    }
+
+    void setSelectionKey( SelectionKey key )
+    {
+        this.key = key;
+    }
+
+    void dispose()
+    {
+        ByteBufferPool.close( readBuf.buf() );
+        ByteBufferPool.close( writeBuf.buf() );
+    }
+
+    public void close()
+    {
+    }
+
+    public Object getAttachment()
+    {
+        return attachment;
+    }
+
+    public void setAttachment( Object attachment )
+    {
+        this.attachment = attachment;
+    }
+
+    public ReadBuffer getReadBuffer()
+    {
+        return readBuf;
+    }
+
+    public WriteBuffer getWriteBuffer()
+    {
+        return writeBuf;
+    }
+
+    void flush()
+    {
+        parent.flushSession( this );
+    }
+
+    public boolean isConnected()
+    {
+        return ch.isConnected();
+    }
+
+    public boolean isClosed()
+    {
+        return !isConnected();
+    }
+
+    public SessionConfig getConfig()
+    {
+        return config;
+    }
+
+    public SocketAddress getRemoteAddress()
+    {
+        return remoteAddress;
+    }
+
+    public SocketAddress getLocalAddress()
+    {
+        return ch.socket().getLocalSocketAddress();
+    }
+
+    public long getReadBytes()
+    {
+        return readBytes;
+    }
+
+    public long getWrittenBytes()
+    {
+        return writtenBytes;
+    }
+
+    void increaseReadBytes( int increment )
+    {
+        readBytes += increment;
+        lastReadTime = System.currentTimeMillis();
+    }
+
+    void increaseWrittenBytes( int increment )
+    {
+        writtenBytes += increment;
+        lastWriteTime = System.currentTimeMillis();
+    }
+
+    public long getLastIoTime()
+    {
+        return Math.max( lastReadTime, lastWriteTime );
+    }
+
+    public long getLastReadTime()
+    {
+        return lastReadTime;
+    }
+
+    public long getLastWriteTime()
+    {
+        return lastWriteTime;
+    }
+
+    public boolean isIdle( IdleStatus status )
+    {
+        if( status == IdleStatus.BOTH_IDLE )
+            return idleForBoth;
+
+        if( status == IdleStatus.READER_IDLE )
+            return idleForRead;
+
+        if( status == IdleStatus.WRITER_IDLE )
+            return idleForWrite;
+
+        throw new IllegalArgumentException( "Unknown idle status: " + status );
+    }
+
+    void setIdle( IdleStatus status )
+    {
+        if( status == IdleStatus.BOTH_IDLE )
+            idleForBoth = true;
+        else if( status == IdleStatus.READER_IDLE )
+            idleForRead = true;
+        else if( status == IdleStatus.WRITER_IDLE )
+            idleForWrite = true;
+        else
+            throw new IllegalArgumentException( "Unknown idle status: "
+                                                + status );
+    }
+}
\ No newline at end of file

Added: incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramReadBuffer.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramReadBuffer.java?view=auto&rev=123258
==============================================================================
--- (empty file)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramReadBuffer.java	Thu Dec 23 16:34:23 2004
@@ -0,0 +1,53 @@
+/*
+ *   @(#) $Id: TcpReadBuffer.java 122635 2004-12-17 11:05:52Z trustin $
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.io.datagram;
+
+import java.nio.ByteBuffer;
+
+import org.apache.mina.io.ReadBuffer;
+import org.apache.mina.util.AbstractReadBuffer;
+
+/**
+ * TODO Document me.
+ * 
+ * @author Trustin Lee (trustin@apache.org)
+ * @version $Rev: 122635 $, $Date: 2004-12-17 20:05:52 +0900 (�?, 17 12 2004) $,
+ */
+class DatagramReadBuffer extends AbstractReadBuffer implements ReadBuffer
+{
+    DatagramReadBuffer( ByteBuffer buf )
+    {
+        super( buf );
+    }
+
+    public ReadBuffer signal()
+    {
+        return this;
+    }
+
+    protected ByteBuffer buf()
+    {
+        return super.buf();
+    }
+
+    protected void markBaseIndex()
+    {
+        super.markBaseIndex();
+    }
+}
\ No newline at end of file

Added: incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramSessionConfig.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramSessionConfig.java?view=auto&rev=123258
==============================================================================
--- (empty file)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramSessionConfig.java	Thu Dec 23 16:34:23 2004
@@ -0,0 +1,60 @@
+/*
+ *   @(#) $Id: TcpSessionConfig.java 122618 2004-12-17 06:38:15Z trustin $
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.io.datagram;
+
+import java.net.SocketException;
+import java.nio.channels.DatagramChannel;
+
+import org.apache.mina.util.BasicSessionConfig;
+
+/**
+ * TODO Document me.
+ * 
+ * @author Trustin Lee (trustin@apache.org)
+ * @version $Rev: 122618 $, $Date: 2004-12-17 15:38:15 +0900 (�?, 17 12 2004) $,
+ */
+public class DatagramSessionConfig extends BasicSessionConfig
+{
+    private final DatagramChannel ch;
+
+    DatagramSessionConfig( DatagramChannel ch )
+    {
+        this.ch = ch;
+    }
+
+    public boolean getReuseAddress() throws SocketException
+    {
+        return ch.socket().getReuseAddress();
+    }
+
+    public void setReuseAddress( boolean on ) throws SocketException
+    {
+        ch.socket().setReuseAddress( on );
+    }
+
+    public int getTrafficClass() throws SocketException
+    {
+        return ch.socket().getTrafficClass();
+    }
+
+    public void setTrafficClass( int tc ) throws SocketException
+    {
+        ch.socket().setTrafficClass( tc );
+    }
+}
\ No newline at end of file

Added: incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramWriteBuffer.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramWriteBuffer.java?view=auto&rev=123258
==============================================================================
--- (empty file)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/datagram/DatagramWriteBuffer.java	Thu Dec 23 16:34:23 2004
@@ -0,0 +1,86 @@
+/*
+ *   @(#) $Id: TcpWriteBuffer.java 122635 2004-12-17 11:05:52Z trustin $
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.io.datagram;
+
+import java.nio.ByteBuffer;
+
+import org.apache.mina.io.WriteBuffer;
+import org.apache.mina.util.AbstractWriteBuffer;
+import org.apache.mina.util.Queue;
+
+/**
+ * TODO Document me.
+ * 
+ * @author Trustin Lee (trustin@apache.org)
+ * @version $Rev: 122635 $, $Date: 2004-12-17 20:05:52 +0900 (�?, 17 12 2004) $,
+ */
+class DatagramWriteBuffer extends AbstractWriteBuffer implements WriteBuffer
+{
+    private final DatagramAcceptorSession session;
+    
+    private final Queue flushMarkers = new Queue();
+
+    DatagramWriteBuffer( DatagramAcceptorSession session, ByteBuffer buf )
+    {
+        super( buf );
+        this.session = session;
+    }
+
+    public WriteBuffer flush()
+    {
+        putFlushMarker();
+        session.flush();
+        return this;
+    }
+
+    private void putFlushMarker()
+    {
+        int bytesLeft;
+        if( flushMarkers.isEmpty() )
+        {
+            bytesLeft = buf.position();
+        }
+        else
+        {
+            bytesLeft = buf.position()
+                        - ( ( Marker ) flushMarkers.last() ).getBytesLeft();
+        }
+
+        markers.push( new Marker( null, bytesLeft ) );
+    }
+
+    protected ByteBuffer buf()
+    {
+        return super.buf();
+    }
+
+    protected Queue getMarkers()
+    {
+        return super.getMarkers();
+    }
+    
+    protected Queue getFlushMarkers() {
+        return flushMarkers;
+    }
+
+    protected void markBaseIndex()
+    {
+        super.markBaseIndex();
+    }
+}
\ No newline at end of file

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/filter/BlacklistFilter.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/filter/BlacklistFilter.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/filter/BlacklistFilter.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/filter/BlacklistFilter.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/filter/BlacklistFilter.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/filter/BlacklistFilter.java	Thu Dec 23 16:34:23 2004
@@ -1,5 +1,20 @@
 /*
- * @(#) $Id$
+ *   @(#) $Id$
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.io.filter;
 

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/filter/IoThreadPoolFilter.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/filter/IoThreadPoolFilter.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/filter/IoThreadPoolFilter.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/filter/IoThreadPoolFilter.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/filter/IoThreadPoolFilter.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/filter/IoThreadPoolFilter.java	Thu Dec 23 16:34:23 2004
@@ -1,5 +1,20 @@
 /*
- * @(#) $Id$
+ *   @(#) $Id$
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.io.filter;
 

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/socket/TcpAcceptor.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/socket/TcpAcceptor.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/socket/TcpAcceptor.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/socket/TcpAcceptor.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/socket/TcpAcceptor.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/socket/TcpAcceptor.java	Thu Dec 23 16:34:23 2004
@@ -30,9 +30,11 @@
 import java.util.Iterator;
 import java.util.Map;
 
+import org.apache.mina.common.ExceptionMonitor;
 import org.apache.mina.io.Acceptor;
 import org.apache.mina.io.IoHandler;
 import org.apache.mina.io.IoHandlerFilter;
+import org.apache.mina.util.DefaultExceptionMonitor;
 import org.apache.mina.util.IoHandlerFilterManager;
 import org.apache.mina.util.Queue;
 
@@ -58,6 +60,8 @@
 
     private final Queue cancelQueue = new Queue();
 
+    private ExceptionMonitor exceptionMonitor = new DefaultExceptionMonitor();
+
     private Worker worker;
 
     /**
@@ -143,7 +147,7 @@
         }
         catch( IOException e )
         {
-            e.printStackTrace();
+            exceptionMonitor.exceptionCaught( this, e );
         }
     }
 
@@ -209,7 +213,7 @@
                 }
                 catch( IOException e )
                 {
-                    e.printStackTrace();
+                    exceptionMonitor.exceptionCaught( TcpAcceptor.this, e );
 
                     try
                     {
@@ -289,5 +293,20 @@
             this.channel = channel;
             this.handler = handler;
         }
+    }
+
+    public ExceptionMonitor getExceptionMonitor()
+    {
+        return exceptionMonitor;
+    }
+
+    public void setExceptionMonitor( ExceptionMonitor monitor )
+    {
+        if( monitor == null )
+        {
+            monitor = new DefaultExceptionMonitor();
+        }
+
+        this.exceptionMonitor = monitor;
     }
 }

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/socket/TcpConnector.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/socket/TcpConnector.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/socket/TcpConnector.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/socket/TcpConnector.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/socket/TcpConnector.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/io/socket/TcpConnector.java	Thu Dec 23 16:34:23 2004
@@ -28,10 +28,12 @@
 import java.util.Iterator;
 import java.util.Set;
 
+import org.apache.mina.common.ExceptionMonitor;
 import org.apache.mina.io.Connector;
 import org.apache.mina.io.IoHandler;
 import org.apache.mina.io.IoHandlerFilter;
 import org.apache.mina.io.IoSession;
+import org.apache.mina.util.DefaultExceptionMonitor;
 import org.apache.mina.util.IoHandlerFilterManager;
 
 /**
@@ -50,6 +52,8 @@
 
     private final Selector selector;
 
+    private ExceptionMonitor exceptionMonitor = new DefaultExceptionMonitor();
+
     private Worker worker;
 
     /**
@@ -243,7 +247,7 @@
                 }
                 catch( IOException e )
                 {
-                    e.printStackTrace();
+                    exceptionMonitor.exceptionCaught( TcpConnector.this, e );
 
                     try
                     {
@@ -285,5 +289,20 @@
     public void removeFilter( IoHandlerFilter filter )
     {
         filterManager.removeFilter( filter );
+    }
+
+    public ExceptionMonitor getExceptionMonitor()
+    {
+        return exceptionMonitor;
+    }
+
+    public void setExceptionMonitor( ExceptionMonitor monitor )
+    {
+        if( monitor == null )
+        {
+            monitor = new DefaultExceptionMonitor();
+        }
+
+        this.exceptionMonitor = monitor;
     }
 }

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/ProtocolHandlerAdapter.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/ProtocolHandlerAdapter.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/ProtocolHandlerAdapter.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/ProtocolHandlerAdapter.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/ProtocolHandlerAdapter.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/ProtocolHandlerAdapter.java	Thu Dec 23 16:34:23 2004
@@ -1,5 +1,20 @@
 /*
- * @(#) $Id$
+ *   @(#) $Id$
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.protocol;
 

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/ProtocolHandlerFilterAdapter.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/ProtocolHandlerFilterAdapter.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/ProtocolHandlerFilterAdapter.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/ProtocolHandlerFilterAdapter.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/ProtocolHandlerFilterAdapter.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/ProtocolHandlerFilterAdapter.java	Thu Dec 23 16:34:23 2004
@@ -1,5 +1,20 @@
 /*
- * @(#) $Id$
+ *   @(#) $Id$
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.protocol;
 

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/filter/ProtocolThreadPoolFilter.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/filter/ProtocolThreadPoolFilter.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/filter/ProtocolThreadPoolFilter.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/filter/ProtocolThreadPoolFilter.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/filter/ProtocolThreadPoolFilter.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/protocol/filter/ProtocolThreadPoolFilter.java	Thu Dec 23 16:34:23 2004
@@ -1,5 +1,20 @@
 /*
- * @(#) $Id$
+ *   @(#) $Id$
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.protocol.filter;
 

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/AbstractWriteBuffer.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/AbstractWriteBuffer.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/AbstractWriteBuffer.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/AbstractWriteBuffer.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/AbstractWriteBuffer.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/AbstractWriteBuffer.java	Thu Dec 23 16:34:23 2004
@@ -287,7 +287,7 @@
 
         private int bytesLeft;
 
-        private Marker( Object value, int bytesLeft )
+        public Marker( Object value, int bytesLeft )
         {
             this.value = value;
             this.bytesLeft = bytesLeft;

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/BlockingSet.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/BlockingSet.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/BlockingSet.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/BlockingSet.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/BlockingSet.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/BlockingSet.java	Thu Dec 23 16:34:23 2004
@@ -1,5 +1,20 @@
 /*
- * @(#) $Id$
+ *   @(#) $Id$
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.util;
 

Added: incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/DefaultExceptionMonitor.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/DefaultExceptionMonitor.java?view=auto&rev=123258
==============================================================================
--- (empty file)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/DefaultExceptionMonitor.java	Thu Dec 23 16:34:23 2004
@@ -0,0 +1,114 @@
+/*
+ *   @(#) $Id$
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.util;
+
+import java.lang.reflect.Method;
+import java.text.DateFormat;
+import java.util.Date;
+
+import org.apache.mina.common.ExceptionMonitor;
+
+/**
+ * TODO Document me.
+ * 
+ * @author Trustin Lee (trustin@apache.org)
+ * @version $Rev$, $Date$
+ */
+public class DefaultExceptionMonitor implements ExceptionMonitor
+{
+    private static final Object log;
+
+    private static final Method errorMethod;
+
+    static
+    {
+        Object tempLog = null;
+        Method tempErrorMethod = null;
+
+        try
+        {
+            Class logCls = Class.forName( "org.apache.commons.logging.Log" );
+            Class logFactoryCls = Class
+                    .forName( "org.apache.commons.logging.LogFactory" );
+            Method getLogMethod = logFactoryCls
+                    .getMethod( "getLog", new Class[] { String.class } );
+            tempLog = getLogMethod
+                    .invoke( null,
+                             new Object[] { DefaultExceptionMonitor.class
+                                     .getPackage().getName() } );
+            tempErrorMethod = logCls
+                    .getMethod( "error", new Class[] { Object.class,
+                                                      Throwable.class } );
+        }
+        catch( Exception e )
+        {
+            tempLog = null;
+            tempErrorMethod = null;
+        }
+
+        log = tempLog;
+        errorMethod = tempErrorMethod;
+    }
+
+    private final DateFormat df = DateFormat
+            .getDateTimeInstance( DateFormat.MEDIUM, DateFormat.MEDIUM );
+
+    private final Date date = new Date();
+
+    public void exceptionCaught( Object source, Throwable cause )
+    {
+        if( log == null )
+        {
+            logToStdErr( cause );
+        }
+        else
+        {
+            logToCommonsLogging( cause );
+        }
+    }
+
+    private void logToCommonsLogging( Throwable cause )
+    {
+        try
+        {
+            errorMethod.invoke( log, new Object[] { "Uncaught exception: ",
+                                                   cause } );
+        }
+        catch( Exception e )
+        {
+            logToStdErr( cause );
+        }
+    }
+
+    private void logToStdErr( Throwable cause )
+    {
+        synchronized( System.err )
+        {
+            date.setTime( System.currentTimeMillis() );
+
+            System.err.print( '[' );
+            System.err.print( df.format( date ) );
+            System.err.print( "] [" );
+            System.err.print( Thread.currentThread().getName() );
+            System.err.print( "] Uncaught exception: " );
+            cause.printStackTrace();
+        }
+    }
+
+}
\ No newline at end of file

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/IoHandlerFilterManager.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/IoHandlerFilterManager.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/IoHandlerFilterManager.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/IoHandlerFilterManager.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/IoHandlerFilterManager.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/IoHandlerFilterManager.java	Thu Dec 23 16:34:23 2004
@@ -1,5 +1,20 @@
 /*
- * @(#) $Id$
+ *   @(#) $Id$
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.util;
 

Modified: incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/ProtocolHandlerFilterManager.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/ProtocolHandlerFilterManager.java?view=diff&rev=123258&p1=incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/ProtocolHandlerFilterManager.java&r1=123257&p2=incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/ProtocolHandlerFilterManager.java&r2=123258
==============================================================================
--- incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/ProtocolHandlerFilterManager.java	(original)
+++ incubator/directory/network/trunk/mina/src/java/org/apache/mina/util/ProtocolHandlerFilterManager.java	Thu Dec 23 16:34:23 2004
@@ -1,5 +1,20 @@
 /*
- * @(#) $Id$
+ *   @(#) $Id$
+ *
+ *   Copyright 2004 The Apache Software Foundation
+ *
+ *   Licensed 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.mina.util;
 

Added: incubator/directory/network/trunk/mina/src/test/org/apache/mina/examples/echoserver/Test.java
Url: http://svn.apache.org/viewcvs/incubator/directory/network/trunk/mina/src/test/org/apache/mina/examples/echoserver/Test.java?view=auto&rev=123258
==============================================================================
--- (empty file)
+++ incubator/directory/network/trunk/mina/src/test/org/apache/mina/examples/echoserver/Test.java	Thu Dec 23 16:34:23 2004
@@ -0,0 +1,69 @@
+/*
+ * @(#) $Id$
+ */
+package org.apache.mina.examples.echoserver;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+
+import junit.framework.TestCase;
+
+import org.apache.mina.io.Acceptor;
+import org.apache.mina.io.filter.IoThreadPoolFilter;
+import org.apache.mina.io.socket.TcpAcceptor;
+
+/**
+ * TODO Document me.
+ * 
+ * @author Trustin Lee (trustin@apache.org)
+ * @version $Rev$, $Date$
+ */
+public class Test extends TestCase
+{
+    private static final int PORT = 8080;
+
+    private Acceptor acceptor;
+
+    protected void setUp() throws Exception
+    {
+        acceptor = new TcpAcceptor();
+        acceptor.bind( new InetSocketAddress( PORT ),
+                       new EchoProtocolHandler() );
+        acceptor.addFilter( Integer.MAX_VALUE, new IoThreadPoolFilter() );
+    }
+
+    protected void tearDown() throws Exception
+    {
+        acceptor.unbind( new InetSocketAddress( PORT ) );
+    }
+
+    public void testPool() throws Exception
+    {
+        Socket s = new Socket( "localhost", PORT );
+        InputStream in = s.getInputStream();
+        OutputStream out = s.getOutputStream();
+        try
+        {
+            for( int i = 0; i < 1024; i++ )
+            {
+                int b = ( ( byte ) i ) & 0xff;
+                System.out.println( "Test: " + b );
+                out.write( b );
+                assertEquals( b, in.read() );
+            }
+        }
+        finally
+        {
+            in.close();
+            out.close();
+            s.close();
+        }
+    }
+
+    public static void main( String[] args )
+    {
+        junit.textui.TestRunner.run( Test.class );
+    }
+}
\ No newline at end of file