You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mina.apache.org by jv...@apache.org on 2011/05/20 15:47:01 UTC

svn commit: r1125385 - in /mina/branches/3.0/core/src/main/java/org/apache/mina: ./ service/ session/ transport/tcp/

Author: jvermillard
Date: Fri May 20 13:47:01 2011
New Revision: 1125385

URL: http://svn.apache.org/viewvc?rev=1125385&view=rev
Log:
write queues structures, still need to be processed in the selector

Added:
    mina/branches/3.0/core/src/main/java/org/apache/mina/session/DefaultWriteQueue.java   (with props)
    mina/branches/3.0/core/src/main/java/org/apache/mina/session/DefaultWriteRequest.java   (with props)
    mina/branches/3.0/core/src/main/java/org/apache/mina/session/WriteQueue.java   (with props)
    mina/branches/3.0/core/src/main/java/org/apache/mina/session/WriteRequest.java   (with props)
Modified:
    mina/branches/3.0/core/src/main/java/org/apache/mina/IoSession.java
    mina/branches/3.0/core/src/main/java/org/apache/mina/service/SelectorProcessor.java
    mina/branches/3.0/core/src/main/java/org/apache/mina/session/AbstractIoSession.java
    mina/branches/3.0/core/src/main/java/org/apache/mina/transport/tcp/NioSelectorProcessor.java

Modified: mina/branches/3.0/core/src/main/java/org/apache/mina/IoSession.java
URL: http://svn.apache.org/viewvc/mina/branches/3.0/core/src/main/java/org/apache/mina/IoSession.java?rev=1125385&r1=1125384&r2=1125385&view=diff
==============================================================================
--- mina/branches/3.0/core/src/main/java/org/apache/mina/IoSession.java (original)
+++ mina/branches/3.0/core/src/main/java/org/apache/mina/IoSession.java Fri May 20 13:47:01 2011
@@ -22,6 +22,8 @@ package org.apache.mina;
 import java.net.SocketAddress;
 import java.util.Set;
 
+import org.apache.mina.service.SelectorProcessor;
+
 /**
  * A handle which represents a connection between two end-points regardless of
  * transport types.
@@ -49,206 +51,233 @@ import java.util.Set;
  */
 public interface IoSession {
 
-    /**
-     * The unique identifier of this session.
-     * 
-     * @return the session's unique identifier 
-     */
-    long getId();
-
-    /* ADDRESSES */
-
-    /**
-     * Returns the socket address of remote peer.
-     * 
-     * @return the remote socket address
-     */
-    SocketAddress getRemoteAddress();
-
-    /**
-     * Gets the local address of the local peer.
-     * 
-     * @return the socket address of local machine which is associated with this
-     * session.
-     */
-    SocketAddress getLocalAddress();
-
-    /**
-     * Gets the service this session is attached to.
-     * 
-     * @return the {@link IoService} which provides {@link IoSession} to this
-     * session.
-     */
-    IoService getService();
-
-    /* READ / WRITE / CLOSE */
-    /**
-     * Tells if the session is currently connected and able to process incoming
-     * requests and to send outgoing responses.
-     * 
-     * @return <code>true</code> if this session is connected with remote peer.
-     */
-    boolean isConnected();
-
-    /**
-     * Tells if the session is being closed, but is not yet in Closed state.
-     * 
-     * @return <code>true</tt> if and only if this session is being closed
-     * (but not disconnected yet) or is closed.
-     */
-    boolean isClosing();
-
-    /**
-     * Closes this session immediately or after all queued write requests are
-     * flushed. This operation is asynchronous. Wait for the returned
-     * {@link CloseFuture} if you want to wait for the session actually closed.
-     * Once this method has been called, no incoming request will be accepted.
-     * 
-     * @param immediately {@code true} to close this session immediately. {@code false}
-     * to close this session after all queued write requests are flushed.
-     * @return A {@link CloseFuture} that will contains the session's state
-     */
-    CloseFuture close(boolean immediately);
-
-    /* READ/WRITE PAUSE MANAGEMENT */
-    /**
-     * Suspends read operations for this session.
-     */
-    void suspendRead();
-
-    /**
-     * Suspends write operations for this session.
-     */
-    void suspendWrite();
-
-    /**
-     * Resumes read operations for this session.
-     */
-    void resumeRead();
-
-    /**
-     * Resumes write operations for this session.
-     */
-    void resumeWrite();
-
-    /**
-     * Is read operation is suspended for this session.
-     * 
-     * @return <code>true</code> if suspended
-     */
-    boolean isReadSuspended();
-
-    /**
-     * Is write operation is suspended for this session.
-     * 
-     * @return <code>true</code> if suspended
-     */
-    boolean isWriteSuspended();
-
-    /* BASIC STATS */
-    /**
-     * Gets the total number of bytes read for this session since it was created.
-     * 
-     * Returns the total number of bytes which were read from this session.
-     */
-    long getReadBytes();
-
-    /**
-     * Gets the total number of bytes written for this session since it was created.
-     * 
-     * @return the total number of bytes which were written to this session.
-     */
-    long getWrittenBytes();
-
-    /* IDLE management */
-    /**
-     * Gets the session configuration, it where the idle timeout are set and
-     * other transport specific configuration.
-     * 
-     * @return the session's configuration
-     */
-    IoSessionConfig getConfig();
-
-    /**
-     * The session's creation time.
-     * 
-     * @return the session's creation time in milliseconds
-     */
-    long getCreationTime();
-
-    /**
-     * Returns the time in millisecond when I/O occurred lastly (either read or write).
-     * 
-     * @return the time of the last read or write done for this session
-     */
-    long getLastIoTime();
-
-    /**
-     * Returns the time in millisecond when the last I/O read occurred.
-     * 
-     * Returns the time in millisecond when read operation occurred lastly.
-     */
-    long getLastReadTime();
-
-    /**
-     * Returns the time in millisecond when the last I/O write occurred.
-     * 
-     * Returns the time in millisecond when write operation occurred lastly.
-     */
-    long getLastWriteTime();
-
-    /* Session context management */
-    /**
-     * Returns the value of the user-defined attribute for this session.
-     * 
-     * @param name the attribute's name
-     * @return <tt>null</tt> if there is no attribute with the specified name
-     */
-    Object getAttribute(Object name);
-
-    /**
-     * Sets a user-defined attribute.
-     * 
-     * @param name the attribute's name
-     * @param value the attribute's value
-     * @return The old attribute's value. <tt>null</tt> if there is no previous value
-     * or if the value is null
-     */
-    Object setAttribute(Object name, Object value);
-
-    /**
-     * Removes a user-defined attribute with the specified name.
-     * 
-     * @param name the attribute's name
-     * @return The old attribute's value. <tt>null</tt> if not found or if the 
-     * attribute had no value
-     */
-    Object removeAttribute(Object name);
-
-    /**
-     * Tells if the session has an attached attribute.
-     * 
-     * @return <tt>true</tt> if this session contains the attribute with the
-     * specified <tt>name</tt>.
-     */
-    boolean containsAttribute(Object name);
-
-    /**
-     * Gets the set of attributes stored within the session.
-     * 
-     * @return the set of names of all user-defined attributes.
-     */
-    Set<Object> getAttributeNames();
-    
-    
-    SessionState getState();
-    
-    /**
-     * State of a {@link IoSession}
-     * @author <a href="http://mina.apache.org">Apache MINA Project</a>
-     *
-     */
-    public enum SessionState {
-        CREATED,CONNECTED,CLOSING,CLOSED
-    }
+	/**
+	 * The unique identifier of this session.
+	 * 
+	 * @return the session's unique identifier
+	 */
+	long getId();
+
+	/* ADDRESSES */
+
+	/**
+	 * Returns the socket address of remote peer.
+	 * 
+	 * @return the remote socket address
+	 */
+	SocketAddress getRemoteAddress();
+
+	/**
+	 * Gets the local address of the local peer.
+	 * 
+	 * @return the socket address of local machine which is associated with this
+	 *         session.
+	 */
+	SocketAddress getLocalAddress();
+
+	/**
+	 * Gets the service this session is attached to.
+	 * 
+	 * @return the {@link IoService} which provides {@link IoSession} to this
+	 *         session.
+	 */
+	IoService getService();
+
+	/* READ / WRITE / CLOSE */
+	/**
+	 * Tells if the session is currently connected and able to process incoming
+	 * requests and to send outgoing responses.
+	 * 
+	 * @return <code>true</code> if this session is connected with remote peer.
+	 */
+	boolean isConnected();
+
+	/**
+	 * Tells if the session is being closed, but is not yet in Closed state.
+	 * 
+	 * @return <code>true</tt> if and only if this session is being closed
+	 * (but not disconnected yet) or is closed.
+	 */
+	boolean isClosing();
+
+	/**
+	 * Closes this session immediately or after all queued write requests are
+	 * flushed. This operation is asynchronous. Wait for the returned
+	 * {@link CloseFuture} if you want to wait for the session actually closed.
+	 * Once this method has been called, no incoming request will be accepted.
+	 * 
+	 * @param immediately
+	 *            {@code true} to close this session immediately. {@code false}
+	 *            to close this session after all queued write requests are
+	 *            flushed.
+	 * @return A {@link CloseFuture} that will contains the session's state
+	 */
+	CloseFuture close(boolean immediately);
+
+	/* READ/WRITE PAUSE MANAGEMENT */
+	/**
+	 * Suspends read operations for this session.
+	 */
+	void suspendRead();
+
+	/**
+	 * Suspends write operations for this session.
+	 */
+	void suspendWrite();
+
+	/**
+	 * Resumes read operations for this session.
+	 */
+	void resumeRead();
+
+	/**
+	 * Resumes write operations for this session.
+	 */
+	void resumeWrite();
+
+	/**
+	 * Is read operation is suspended for this session.
+	 * 
+	 * @return <code>true</code> if suspended
+	 */
+	boolean isReadSuspended();
+
+	/**
+	 * Is write operation is suspended for this session.
+	 * 
+	 * @return <code>true</code> if suspended
+	 */
+	boolean isWriteSuspended();
+
+	/* BASIC STATS */
+	/**
+	 * Gets the total number of bytes read for this session since it was
+	 * created.
+	 * 
+	 * Returns the total number of bytes which were read from this session.
+	 */
+	long getReadBytes();
+
+	/**
+	 * Gets the total number of bytes written for this session since it was
+	 * created.
+	 * 
+	 * @return the total number of bytes which were written to this session.
+	 */
+	long getWrittenBytes();
+
+	/* IDLE management */
+	/**
+	 * Gets the session configuration, it where the idle timeout are set and
+	 * other transport specific configuration.
+	 * 
+	 * @return the session's configuration
+	 */
+	IoSessionConfig getConfig();
+
+	/**
+	 * The session's creation time.
+	 * 
+	 * @return the session's creation time in milliseconds
+	 */
+	long getCreationTime();
+
+	/**
+	 * Returns the time in millisecond when I/O occurred lastly (either read or
+	 * write).
+	 * 
+	 * @return the time of the last read or write done for this session
+	 */
+	long getLastIoTime();
+
+	/**
+	 * Returns the time in millisecond when the last I/O read occurred.
+	 * 
+	 * Returns the time in millisecond when read operation occurred lastly.
+	 */
+	long getLastReadTime();
+
+	/**
+	 * Returns the time in millisecond when the last I/O write occurred.
+	 * 
+	 * Returns the time in millisecond when write operation occurred lastly.
+	 */
+	long getLastWriteTime();
+
+	/* Session context management */
+	/**
+	 * Returns the value of the user-defined attribute for this session.
+	 * 
+	 * @param name
+	 *            the attribute's name
+	 * @return <tt>null</tt> if there is no attribute with the specified name
+	 */
+	Object getAttribute(Object name);
+
+	/**
+	 * Sets a user-defined attribute.
+	 * 
+	 * @param name
+	 *            the attribute's name
+	 * @param value
+	 *            the attribute's value
+	 * @return The old attribute's value. <tt>null</tt> if there is no previous
+	 *         value or if the value is null
+	 */
+	Object setAttribute(Object name, Object value);
+
+	/**
+	 * Removes a user-defined attribute with the specified name.
+	 * 
+	 * @param name
+	 *            the attribute's name
+	 * @return The old attribute's value. <tt>null</tt> if not found or if the
+	 *         attribute had no value
+	 */
+	Object removeAttribute(Object name);
+
+	/**
+	 * Tells if the session has an attached attribute.
+	 * 
+	 * @return <tt>true</tt> if this session contains the attribute with the
+	 *         specified <tt>name</tt>.
+	 */
+	boolean containsAttribute(Object name);
+
+	/**
+	 * Gets the set of attributes stored within the session.
+	 * 
+	 * @return the set of names of all user-defined attributes.
+	 */
+	Set<Object> getAttributeNames();
+
+	SessionState getState();
+
+	/**
+	 * State of a {@link IoSession}
+	 * 
+	 * @author <a href="http://mina.apache.org">Apache MINA Project</a>
+	 * 
+	 */
+	public enum SessionState {
+		CREATED, CONNECTED, CLOSING, CLOSED
+	}
+
+	/* SESSION WRITING */
+	/**
+	 * Enqueue a message for writing. This method wont block ! The message will
+	 * by asynchronously processed by the filter chain and wrote to socket by
+	 * the {@link SelectorProcessor}.
+	 * 
+	 */
+	public void write(Object message);
+
+	/**
+	 * Same as {@link IoSession#write(Object)}, but provide a
+	 * {@link WriteFuture} for tracking the completion of this write.
+	 * 
+	 * @param message the message to be processed and written
+	 * @return the {@link WriteFuture} for tracking this asynchronous operation
+	 */
+	public WriteFuture writeWithFuture(Object message);
 }
\ No newline at end of file

Modified: mina/branches/3.0/core/src/main/java/org/apache/mina/service/SelectorProcessor.java
URL: http://svn.apache.org/viewvc/mina/branches/3.0/core/src/main/java/org/apache/mina/service/SelectorProcessor.java?rev=1125385&r1=1125384&r2=1125385&view=diff
==============================================================================
--- mina/branches/3.0/core/src/main/java/org/apache/mina/service/SelectorProcessor.java (original)
+++ mina/branches/3.0/core/src/main/java/org/apache/mina/service/SelectorProcessor.java Fri May 20 13:47:01 2011
@@ -25,6 +25,7 @@ import java.net.SocketAddress;
 
 import org.apache.mina.IoServer;
 import org.apache.mina.IoService;
+import org.apache.mina.IoSession;
 
 /**
  * A processor in charge of a group of client session and server sockets.
@@ -54,4 +55,10 @@ public interface SelectorProcessor {
      * @throws IOException exception thrown if any problem occurs while unbinding
      */
     void unbind(SocketAddress address) throws IOException;
+    
+    /**
+     * Schedule a session for flushing, to be called after a session write.
+     * @param session the session to flush
+     */
+    void flush(IoSession session);
 }

Modified: mina/branches/3.0/core/src/main/java/org/apache/mina/session/AbstractIoSession.java
URL: http://svn.apache.org/viewvc/mina/branches/3.0/core/src/main/java/org/apache/mina/session/AbstractIoSession.java?rev=1125385&r1=1125384&r2=1125385&view=diff
==============================================================================
--- mina/branches/3.0/core/src/main/java/org/apache/mina/session/AbstractIoSession.java (original)
+++ mina/branches/3.0/core/src/main/java/org/apache/mina/session/AbstractIoSession.java Fri May 20 13:47:01 2011
@@ -26,6 +26,8 @@ import java.util.concurrent.atomic.Atomi
 
 import org.apache.mina.IoService;
 import org.apache.mina.IoSession;
+import org.apache.mina.WriteFuture;
+import org.apache.mina.service.SelectorProcessor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -65,16 +67,18 @@ public abstract class AbstractIoSession 
 
     /** unique identifier generator */
     private static final AtomicLong NEXT_ID = new AtomicLong(0);
-
     
+    protected Object stateMonitor = new Object();
     protected SessionState state;
     
+    /** the queue of pending writes for the session, to be dequeued by the {@link SelectorProcessor} */
+    private WriteQueue writeQueue = new DefaultWriteQueue();
+    
     /**
      * Create an {@link IoSession} with a unique identifier (
      * {@link IoSession#getId()}) and an associated {@link IoService}
      * 
-     * @param the
-     *            service this session is associated with
+     * @param service the service this session is associated with
      */
     public AbstractIoSession(IoService service) {
         // generated a unique id
@@ -82,7 +86,9 @@ public abstract class AbstractIoSession 
         creationTime = System.currentTimeMillis();
         this.service = service;
         LOG.debug("Created new session with id : {}", id);
-        this.state=SessionState.CREATED;
+        synchronized (stateMonitor) {
+            this.state=SessionState.CREATED;
+		}
     }
 
     public SessionState getState() {
@@ -191,4 +197,26 @@ public abstract class AbstractIoSession 
     public Set<Object> getAttributeNames() {
         return attributes.keySet();
     }
-}
+    
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void write(Object message) {
+    	if (state == SessionState.CLOSED || state == SessionState.CLOSING) {
+    		// TODO actually we just just shallow the message if the session is closed/closing
+    		LOG.error("writing to closed or cloing session");
+    		return;
+    	}
+    	writeQueue.add(new DefaultWriteRequest(message));
+    	
+    	// mark as interested to write
+    }
+    
+    @Override
+    public WriteFuture writeWithFuture(Object message) {
+    	write(message);
+    	// TODO implements WriteFuture
+    	return null;
+    }
+}
\ No newline at end of file

Added: mina/branches/3.0/core/src/main/java/org/apache/mina/session/DefaultWriteQueue.java
URL: http://svn.apache.org/viewvc/mina/branches/3.0/core/src/main/java/org/apache/mina/session/DefaultWriteQueue.java?rev=1125385&view=auto
==============================================================================
--- mina/branches/3.0/core/src/main/java/org/apache/mina/session/DefaultWriteQueue.java (added)
+++ mina/branches/3.0/core/src/main/java/org/apache/mina/session/DefaultWriteQueue.java Fri May 20 13:47:01 2011
@@ -0,0 +1,37 @@
+/*
+ *  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.mina.session;
+
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+import org.apache.mina.IoSession;
+
+/**
+ * The default implementation of the WriteRequest queue for storing writes to
+ * process in the {@link IoSession}.
+ * 
+ * @author <a href="http://mina.apache.org">Apache MINA Project</a>
+ * 
+ */
+public class DefaultWriteQueue extends ConcurrentLinkedQueue<WriteRequest> implements WriteQueue {
+
+	private static final long serialVersionUID = 1L;
+
+}

Propchange: mina/branches/3.0/core/src/main/java/org/apache/mina/session/DefaultWriteQueue.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: mina/branches/3.0/core/src/main/java/org/apache/mina/session/DefaultWriteRequest.java
URL: http://svn.apache.org/viewvc/mina/branches/3.0/core/src/main/java/org/apache/mina/session/DefaultWriteRequest.java?rev=1125385&view=auto
==============================================================================
--- mina/branches/3.0/core/src/main/java/org/apache/mina/session/DefaultWriteRequest.java (added)
+++ mina/branches/3.0/core/src/main/java/org/apache/mina/session/DefaultWriteRequest.java Fri May 20 13:47:01 2011
@@ -0,0 +1,39 @@
+/*
+ *  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.mina.session;
+
+/**
+ * 
+ * @author <a href="http://mina.apache.org">Apache MINA Project</a>
+ * 
+ */
+public class DefaultWriteRequest implements WriteRequest {
+
+	private Object message;
+
+	public DefaultWriteRequest(Object message) {
+		this.message = message;
+	}
+
+	@Override
+	public Object getMessage() {
+		return message;
+	}
+}
\ No newline at end of file

Propchange: mina/branches/3.0/core/src/main/java/org/apache/mina/session/DefaultWriteRequest.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: mina/branches/3.0/core/src/main/java/org/apache/mina/session/WriteQueue.java
URL: http://svn.apache.org/viewvc/mina/branches/3.0/core/src/main/java/org/apache/mina/session/WriteQueue.java?rev=1125385&view=auto
==============================================================================
--- mina/branches/3.0/core/src/main/java/org/apache/mina/session/WriteQueue.java (added)
+++ mina/branches/3.0/core/src/main/java/org/apache/mina/session/WriteQueue.java Fri May 20 13:47:01 2011
@@ -0,0 +1,31 @@
+/*
+ *  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.mina.session;
+
+import java.util.Queue;
+
+/**
+ * A {@link Queue} of {@link WriteRequest} being associated with each session.
+ * 
+ * @author <a href="http://mina.apache.org">Apache MINA Project</a>
+ */
+public interface WriteQueue extends Queue<WriteRequest> {
+
+}
\ No newline at end of file

Propchange: mina/branches/3.0/core/src/main/java/org/apache/mina/session/WriteQueue.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: mina/branches/3.0/core/src/main/java/org/apache/mina/session/WriteRequest.java
URL: http://svn.apache.org/viewvc/mina/branches/3.0/core/src/main/java/org/apache/mina/session/WriteRequest.java?rev=1125385&view=auto
==============================================================================
--- mina/branches/3.0/core/src/main/java/org/apache/mina/session/WriteRequest.java (added)
+++ mina/branches/3.0/core/src/main/java/org/apache/mina/session/WriteRequest.java Fri May 20 13:47:01 2011
@@ -0,0 +1,38 @@
+/*
+ *  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.mina.session;
+
+import org.apache.mina.IoSession;
+
+/**
+ * The write request created by the {@link IoSession#write} method, travel around the filter chain and finish as a 
+ * socket write.
+ * 
+ * @author <a href="http://mina.apache.org">Apache MINA Project</a>
+ */
+public interface WriteRequest {
+	
+	/**
+	 * Get the message of this request.
+	 * 
+	 * @return the contained message
+	 */
+	Object getMessage();
+}
\ No newline at end of file

Propchange: mina/branches/3.0/core/src/main/java/org/apache/mina/session/WriteRequest.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: mina/branches/3.0/core/src/main/java/org/apache/mina/transport/tcp/NioSelectorProcessor.java
URL: http://svn.apache.org/viewvc/mina/branches/3.0/core/src/main/java/org/apache/mina/transport/tcp/NioSelectorProcessor.java?rev=1125385&r1=1125384&r2=1125385&view=diff
==============================================================================
--- mina/branches/3.0/core/src/main/java/org/apache/mina/transport/tcp/NioSelectorProcessor.java (original)
+++ mina/branches/3.0/core/src/main/java/org/apache/mina/transport/tcp/NioSelectorProcessor.java Fri May 20 13:47:01 2011
@@ -36,6 +36,7 @@ import java.util.concurrent.ConcurrentLi
 
 import org.apache.mina.IoServer;
 import org.apache.mina.IoService;
+import org.apache.mina.IoSession;
 import org.apache.mina.service.SelectorProcessor;
 import org.apache.mina.service.SelectorStrategy;
 import org.apache.mina.transport.tcp.nio.NioTcpServer;
@@ -161,7 +162,7 @@ public class NioSelectorProcessor implem
         // map for finding the keys associated with a given server
         private Map<ServerSocketChannel, SelectionKey> serverKey = new HashMap<ServerSocketChannel, SelectionKey>();
 
-        // map for fining keys associated with a given session
+        // map for finding keys associated with a given session
         private Map<NioTcpSession, SelectionKey> sessionKey = new HashMap<NioTcpSession, SelectionKey>();
 
         @Override
@@ -291,4 +292,10 @@ public class NioSelectorProcessor implem
             }
         }
     }
+
+	@Override
+	public void flush(IoSession session) {
+		// add the session to the list of session to be registered for writing
+		// wake the selector
+	}
 }