You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mina.apache.org by tr...@apache.org on 2007/09/27 18:44:00 UTC

svn commit: r580090 - in /mina/trunk/core/src/main/java/org/apache/mina: common/AbstractIoProcessor.java common/IdleStatusChecker.java transport/socket/nio/NioDatagramAcceptor.java

Author: trustin
Date: Thu Sep 27 09:43:59 2007
New Revision: 580090

URL: http://svn.apache.org/viewvc?rev=580090&view=rev
Log:
* Removed duplication related with idleness notification
* Resolved issue: DIRMINA-318 (sessionIdle event is not fired for datagram sessions.)
** NioDatagramConnector works just fine thanks to my last refactoring with AbstractIoProcessor
** NioDatagramAcceptor now fires sessionIdle event properly
** But this refactoring implies a lot of down-casting, so we need to see if it impacts performance.


Modified:
    mina/trunk/core/src/main/java/org/apache/mina/common/AbstractIoProcessor.java
    mina/trunk/core/src/main/java/org/apache/mina/common/IdleStatusChecker.java
    mina/trunk/core/src/main/java/org/apache/mina/transport/socket/nio/NioDatagramAcceptor.java

Modified: mina/trunk/core/src/main/java/org/apache/mina/common/AbstractIoProcessor.java
URL: http://svn.apache.org/viewvc/mina/trunk/core/src/main/java/org/apache/mina/common/AbstractIoProcessor.java?rev=580090&r1=580089&r2=580090&view=diff
==============================================================================
--- mina/trunk/core/src/main/java/org/apache/mina/common/AbstractIoProcessor.java (original)
+++ mina/trunk/core/src/main/java/org/apache/mina/common/AbstractIoProcessor.java Thu Sep 27 09:43:59 2007
@@ -332,54 +332,12 @@
         }
     }
 
-    private void notifyIdleness() throws Exception {
+    private void notifyIdleSessions() throws Exception {
         // process idle sessions
         long currentTime = System.currentTimeMillis();
         if (currentTime - lastIdleCheckTime >= 1000) {
             lastIdleCheckTime = currentTime;
-            for (Iterator<AbstractIoSession> i = allSessions(); i.hasNext();) {
-                AbstractIoSession session = i.next();
-                try {
-                    notifyIdleness(session, currentTime);
-                } catch (Exception e) {
-                    session.getFilterChain().fireExceptionCaught(e);
-                }
-            }
-        }
-    }
-
-    private void notifyIdleness(AbstractIoSession session, long currentTime) throws Exception {
-        notifyIdleness0(session, currentTime, session
-                .getConfig().getIdleTimeInMillis(IdleStatus.BOTH_IDLE),
-                IdleStatus.BOTH_IDLE, Math.max(session.getLastIoTime(), session
-                .getLastIdleTime(IdleStatus.BOTH_IDLE)));
-        notifyIdleness0(session, currentTime, session
-                .getConfig().getIdleTimeInMillis(IdleStatus.READER_IDLE),
-                IdleStatus.READER_IDLE, Math.max(session.getLastReadTime(),
-                session.getLastIdleTime(IdleStatus.READER_IDLE)));
-        notifyIdleness0(session, currentTime, session
-                .getConfig().getIdleTimeInMillis(IdleStatus.WRITER_IDLE),
-                IdleStatus.WRITER_IDLE, Math.max(session.getLastWriteTime(),
-                session.getLastIdleTime(IdleStatus.WRITER_IDLE)));
-
-        notifyWriteTimeout(session, currentTime, session
-                .getConfig().getWriteTimeoutInMillis(), session.getLastWriteTime());
-    }
-
-    private void notifyIdleness0(AbstractIoSession session, long currentTime,
-                                 long idleTime, IdleStatus status, long lastIoTime) {
-        if (idleTime > 0 && lastIoTime != 0
-                && currentTime - lastIoTime >= idleTime) {
-            session.increaseIdleCount(status);
-            session.getFilterChain().fireSessionIdle(status);
-        }
-    }
-
-    private void notifyWriteTimeout(AbstractIoSession session,
-                                    long currentTime, long writeTimeout, long lastIoTime) throws Exception {
-        if (writeTimeout > 0 && currentTime - lastIoTime >= writeTimeout
-                && isOpWrite(session)) {
-            session.getFilterChain().fireExceptionCaught(new WriteTimeoutException());
+            IdleStatusChecker.notifyIdleSessions(allSessions(), currentTime);
         }
     }
 
@@ -585,7 +543,7 @@
 
                     flush();
                     nSessions -= remove();
-                    notifyIdleness();
+                    notifyIdleSessions();
 
                     if (nSessions == 0) {
                         synchronized (lock) {

Modified: mina/trunk/core/src/main/java/org/apache/mina/common/IdleStatusChecker.java
URL: http://svn.apache.org/viewvc/mina/trunk/core/src/main/java/org/apache/mina/common/IdleStatusChecker.java?rev=580090&r1=580089&r2=580090&view=diff
==============================================================================
--- mina/trunk/core/src/main/java/org/apache/mina/common/IdleStatusChecker.java (original)
+++ mina/trunk/core/src/main/java/org/apache/mina/common/IdleStatusChecker.java Thu Sep 27 09:43:59 2007
@@ -81,8 +81,27 @@
             }
         }
     }
+    
+    /**
+     * Fires a {@link IoEventType#SESSION_IDLE} event to any applicable
+     * sessions in the specified collection.
+     *   
+     * @param currentTime the current time (i.e. {@link System#currentTimeMillis()})
+     */
+    public static void notifyIdleSessions(Iterator<? extends IoSession> sessions, long currentTime) {
+        while (sessions.hasNext()) {
+            IoSession s = sessions.next();
+            notifyIdleSession(s, currentTime);
+        }
+    }
 
-    private void notifyIdleSession(AbstractIoSession session, long currentTime) {
+    /**
+     * Fires a {@link IoEventType#SESSION_IDLE} event if applicable for the
+     * specified {@code session}.
+     * 
+     * @param currentTime the current time (i.e. {@link System#currentTimeMillis()})
+     */
+    public static void notifyIdleSession(IoSession session, long currentTime) {
         notifyIdleSession0(session, currentTime, session
                 .getConfig().getIdleTimeInMillis(IdleStatus.BOTH_IDLE),
                 IdleStatus.BOTH_IDLE, Math.max(session.getLastIoTime(), session
@@ -95,15 +114,27 @@
                 .getConfig().getIdleTimeInMillis(IdleStatus.WRITER_IDLE),
                 IdleStatus.WRITER_IDLE, Math.max(session.getLastWriteTime(),
                         session.getLastIdleTime(IdleStatus.WRITER_IDLE)));
+        notifyWriteTimeout(session, currentTime, session
+                .getConfig().getWriteTimeoutInMillis(), session.getLastWriteTime());
     }
 
-    private void notifyIdleSession0(AbstractIoSession session, long currentTime,
+    private static void notifyIdleSession0(IoSession session, long currentTime,
             long idleTime, IdleStatus status, long lastIoTime) {
         if (idleTime > 0 && lastIoTime != 0
                 && currentTime - lastIoTime >= idleTime) {
-            session.increaseIdleCount(status);
+            if (session instanceof AbstractIoSession) {
+                ((AbstractIoSession) session).increaseIdleCount(status);
+            }
             session.getFilterChain().fireSessionIdle(status);
         }
     }
 
+    private static void notifyWriteTimeout(IoSession session,
+            long currentTime, long writeTimeout, long lastIoTime) {
+        if (session instanceof AbstractIoSession &&
+                writeTimeout > 0 && currentTime - lastIoTime >= writeTimeout &&
+                !((AbstractIoSession) session).getWriteRequestQueue().isEmpty()) {
+            session.getFilterChain().fireExceptionCaught(new WriteTimeoutException());
+        }
+    }
 }

Modified: mina/trunk/core/src/main/java/org/apache/mina/transport/socket/nio/NioDatagramAcceptor.java
URL: http://svn.apache.org/viewvc/mina/trunk/core/src/main/java/org/apache/mina/transport/socket/nio/NioDatagramAcceptor.java?rev=580090&r1=580089&r2=580090&view=diff
==============================================================================
--- mina/trunk/core/src/main/java/org/apache/mina/transport/socket/nio/NioDatagramAcceptor.java (original)
+++ mina/trunk/core/src/main/java/org/apache/mina/transport/socket/nio/NioDatagramAcceptor.java Thu Sep 27 09:43:59 2007
@@ -35,6 +35,7 @@
 import org.apache.mina.common.ByteBuffer;
 import org.apache.mina.common.ExceptionMonitor;
 import org.apache.mina.common.ExpiringSessionRecycler;
+import org.apache.mina.common.IdleStatusChecker;
 import org.apache.mina.common.IoAcceptor;
 import org.apache.mina.common.IoProcessor;
 import org.apache.mina.common.IoServiceListenerSupport;
@@ -60,25 +61,19 @@
 
     private static volatile int nextId = 0;
 
-    private IoSessionRecycler sessionRecycler = DEFAULT_RECYCLER;
-
     private final Executor executor;
-
     private final int id = nextId++;
-
     private final Selector selector;
-
     private final IoProcessor processor = new DatagramAcceptorProcessor();
-
-    private DatagramChannel channel;
-
     private final Queue<ServiceOperationFuture> registerQueue = new ConcurrentLinkedQueue<ServiceOperationFuture>();
-
     private final Queue<ServiceOperationFuture> cancelQueue = new ConcurrentLinkedQueue<ServiceOperationFuture>();
-
     private final Queue<NioDatagramSession> flushingSessions = new ConcurrentLinkedQueue<NioDatagramSession>();
 
+    private IoSessionRecycler sessionRecycler = DEFAULT_RECYCLER;
+
+    private DatagramChannel channel;
     private Worker worker;
+    private long lastIdleCheckTime;
 
     /**
      * Creates a new instance.
@@ -277,10 +272,11 @@
     private class Worker implements Runnable {
         public void run() {
             Thread.currentThread().setName("DatagramAcceptor-" + id);
+            lastIdleCheckTime = System.currentTimeMillis();
 
             for (; ;) {
                 try {
-                    int nKeys = selector.select();
+                    int nKeys = selector.select(1000);
 
                     registerNew();
 
@@ -291,6 +287,8 @@
                     flushSessions();
                     cancelKeys();
 
+                    notifyIdleSessions();
+
                     if (selector.keys().isEmpty()) {
                         synchronized (NioDatagramAcceptor.this) {
                             if (selector.keys().isEmpty()
@@ -507,6 +505,17 @@
                 getListeners().fireServiceDeactivated();
                 request.setDone();
             }
+        }
+    }
+
+    private void notifyIdleSessions() {
+        // process idle sessions
+        long currentTime = System.currentTimeMillis();
+        if (currentTime - lastIdleCheckTime >= 1000) {
+            lastIdleCheckTime = currentTime;
+            IdleStatusChecker.notifyIdleSessions(
+                    getListeners().getManagedSessions().iterator(),
+                    currentTime);
         }
     }
 }