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 2008/04/07 14:40:43 UTC

svn commit: r645480 - in /mina/trunk/core/src/main/java/org/apache/mina: common/AbstractIoService.java common/AbstractIoSession.java common/DummySession.java common/IoService.java transport/vmpipe/VmPipeSessionImpl.java

Author: trustin
Date: Mon Apr  7 05:40:40 2008
New Revision: 645480

URL: http://svn.apache.org/viewvc?rev=645480&view=rev
Log:
scheduledWriteBytes and scheduledWriteMessages cannot be greated than Integer.MAX.  Otherwise we will get OOM 100%.  Consequently, changed the type of scheduledWriteBytes and scheduledWriteMessages to AtomicInteger.


Modified:
    mina/trunk/core/src/main/java/org/apache/mina/common/AbstractIoService.java
    mina/trunk/core/src/main/java/org/apache/mina/common/AbstractIoSession.java
    mina/trunk/core/src/main/java/org/apache/mina/common/DummySession.java
    mina/trunk/core/src/main/java/org/apache/mina/common/IoService.java
    mina/trunk/core/src/main/java/org/apache/mina/transport/vmpipe/VmPipeSessionImpl.java

Modified: mina/trunk/core/src/main/java/org/apache/mina/common/AbstractIoService.java
URL: http://svn.apache.org/viewvc/mina/trunk/core/src/main/java/org/apache/mina/common/AbstractIoService.java?rev=645480&r1=645479&r2=645480&view=diff
==============================================================================
--- mina/trunk/core/src/main/java/org/apache/mina/common/AbstractIoService.java (original)
+++ mina/trunk/core/src/main/java/org/apache/mina/common/AbstractIoService.java Mon Apr  7 05:40:40 2008
@@ -102,8 +102,8 @@
     private long lastReadTime;
     private long lastWriteTime;
 
-    private final AtomicLong scheduledWriteBytes = new AtomicLong();
-    private final AtomicLong scheduledWriteMessages = new AtomicLong();
+    private final AtomicInteger scheduledWriteBytes = new AtomicInteger();
+    private final AtomicInteger scheduledWriteMessages = new AtomicInteger();
 
     private final Object throughputCalculationLock = new Object();
     private int throughputCalculationInterval = 3;
@@ -153,8 +153,8 @@
                     + getTransportMetadata().getSessionConfigType() + ")");
         }
 
-        this.listeners = new IoServiceListenerSupport(this);
-        this.listeners.add(serviceActivationListener);
+        listeners = new IoServiceListenerSupport(this);
+        listeners.add(serviceActivationListener);
         this.sessionConfig = sessionConfig;
 
         // Make JVM load the exception monitor before some transports
@@ -163,13 +163,13 @@
 
         if (executor == null) {
             this.executor = Executors.newCachedThreadPool();
-            this.createdExecutor = true;
+            createdExecutor = true;
         } else {
             this.executor = executor;
-            this.createdExecutor = false;
+            createdExecutor = false;
         }
 
-        this.threadName = getClass().getSimpleName() + '-' + id.incrementAndGet();
+        threadName = getClass().getSimpleName() + '-' + id.incrementAndGet();
 
         executeWorker(idleStatusChecker.getNotifyingTask(), "idleStatusChecker");
     }
@@ -438,15 +438,15 @@
         }
     }
 
-    public final long getScheduledWriteBytes() {
+    public final int getScheduledWriteBytes() {
         return scheduledWriteBytes.get();
     }
 
-    protected final void increaseScheduledWriteBytes(long increment) {
+    protected final void increaseScheduledWriteBytes(int increment) {
         scheduledWriteBytes.addAndGet(increment);
     }
 
-    public final long getScheduledWriteMessages() {
+    public final int getScheduledWriteMessages() {
         return scheduledWriteMessages.get();
     }
 

Modified: mina/trunk/core/src/main/java/org/apache/mina/common/AbstractIoSession.java
URL: http://svn.apache.org/viewvc/mina/trunk/core/src/main/java/org/apache/mina/common/AbstractIoSession.java?rev=645480&r1=645479&r2=645480&view=diff
==============================================================================
--- mina/trunk/core/src/main/java/org/apache/mina/common/AbstractIoSession.java (original)
+++ mina/trunk/core/src/main/java/org/apache/mina/common/AbstractIoSession.java Mon Apr  7 05:40:40 2008
@@ -28,7 +28,6 @@
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.mina.util.CircularQueue;
 
@@ -45,7 +44,7 @@
         new AttributeKey(AbstractIoSession.class, "readyReadFutures");
     private static final AttributeKey WAITING_READ_FUTURES =
         new AttributeKey(AbstractIoSession.class, "waitingReadFutures");
-    
+
     private static final IoFutureListener<CloseFuture> SCHEDULED_COUNTER_RESETTER =
         new IoFutureListener<CloseFuture>() {
             public void operationComplete(CloseFuture future) {
@@ -67,12 +66,12 @@
         new DefaultWriteRequest(new Object());
 
     private final Object lock = new Object();
-    
+
     private IoSessionAttributeMap attributes;
     private WriteRequestQueue writeRequestQueue;
     private WriteRequest currentWriteRequest;
     private final long creationTime;
-    
+
     /**
      * A future that will be set 'closed' when the connection is closed.
      */
@@ -83,7 +82,7 @@
 
     // Status variables
     private final AtomicBoolean scheduledForFlush = new AtomicBoolean();
-    private final AtomicLong scheduledWriteBytes = new AtomicLong();
+    private final AtomicInteger scheduledWriteBytes = new AtomicInteger();
     private final AtomicInteger scheduledWriteMessages = new AtomicInteger();
 
     private long readBytes;
@@ -92,7 +91,7 @@
     private long writtenMessages;
     private long lastReadTime;
     private long lastWriteTime;
-    
+
     private long lastThroughputCalculationTime;
     private long lastReadBytes;
     private long lastWrittenBytes;
@@ -114,13 +113,13 @@
     private boolean deferDecreaseReadBuffer = true;
 
     protected AbstractIoSession() {
-        creationTime = lastThroughputCalculationTime = 
+        creationTime = lastThroughputCalculationTime =
             lastReadTime = lastWriteTime =
             lastIdleTimeForBoth = lastIdleTimeForRead =
             lastIdleTimeForWrite = System.currentTimeMillis();
         closeFuture.addListener(SCHEDULED_COUNTER_RESETTER);
     }
-    
+
     public final long getId() {
         return hashCode() & 0xFFFFFFFFL;
     }
@@ -185,7 +184,7 @@
         if (!getConfig().isUseReadOperation()) {
             throw new IllegalStateException("useReadOperation is not enabled.");
         }
-        
+
         Queue<ReadFuture> readyReadFutures = getReadyReadFutures();
         ReadFuture future;
         synchronized (readyReadFutures) {
@@ -200,10 +199,10 @@
                 getWaitingReadFutures().offer(future);
             }
         }
-        
+
         return future;
     }
-    
+
     protected final void offerReadFuture(Object message) {
         newReadFuture().setRead(message);
     }
@@ -235,20 +234,20 @@
 
     @SuppressWarnings("unchecked")
     private Queue<ReadFuture> getReadyReadFutures() {
-        Queue<ReadFuture> readyReadFutures = 
+        Queue<ReadFuture> readyReadFutures =
             (Queue<ReadFuture>) getAttribute(READY_READ_FUTURES);
         if (readyReadFutures == null) {
             readyReadFutures = new CircularQueue<ReadFuture>();
-            
+
             Queue<ReadFuture> oldReadyReadFutures =
                 (Queue<ReadFuture>) setAttributeIfAbsent(
                         READY_READ_FUTURES, readyReadFutures);
             if (oldReadyReadFutures != null) {
                 readyReadFutures = oldReadyReadFutures;
             }
-            
+
             // Initialize waitingReadFutures together.
-            Queue<ReadFuture> waitingReadFutures = 
+            Queue<ReadFuture> waitingReadFutures =
                 new CircularQueue<ReadFuture>();
             setAttributeIfAbsent(WAITING_READ_FUTURES, waitingReadFutures);
         }
@@ -372,7 +371,7 @@
     public final Set<Object> getAttributeKeys() {
         return attributes.getAttributeKeys(this);
     }
-    
+
     protected final IoSessionAttributeMap getAttributeMap() {
         return attributes;
     }
@@ -380,7 +379,7 @@
     protected final void setAttributeMap(IoSessionAttributeMap attributes) {
         this.attributes = attributes;
     }
-    
+
     protected final void setWriteRequestQueue(WriteRequestQueue writeRequestQueue) {
         this.writeRequestQueue =
             new CloseRequestAwareWriteRequestQueue(writeRequestQueue);
@@ -394,14 +393,14 @@
         if (trafficMask == null) {
             throw new NullPointerException("trafficMask");
         }
-        
+
         if (isClosing() || !isConnected()) {
             return;
         }
-        
+
         getFilterChain().fireFilterSetTrafficMask(trafficMask);
     }
-    
+
     protected final void setTrafficMaskNow(TrafficMask trafficMask) {
         this.trafficMask = trafficMask;
     }
@@ -453,38 +452,38 @@
     public final double getWrittenMessagesThroughput() {
         return writtenMessagesThroughput;
     }
-    
+
     /**
      * Update all statistical properties related with throughput assuming
      * the specified time is the current time.  By default this method returns
-     * silently without updating the throughput properties if they were 
-     * calculated already within last 
+     * silently without updating the throughput properties if they were
+     * calculated already within last
      * {@link IoSessionConfig#getThroughputCalculationInterval() calculation interval}.
-     * If, however, <tt>force</tt> is specified as <tt>true</tt>, this method 
+     * If, however, <tt>force</tt> is specified as <tt>true</tt>, this method
      * updates the throughput properties immediately.
 
      * @param currentTime the current time in milliseconds
      */
     protected final void updateThroughput(long currentTime, boolean force) {
         int interval = (int) (currentTime - lastThroughputCalculationTime);
-        
+
         long minInterval = getConfig().getThroughputCalculationIntervalInMillis();
         if (minInterval == 0 || interval < minInterval) {
             if (!force) {
                 return;
             }
         }
-        
+
         readBytesThroughput = (readBytes - lastReadBytes) * 1000.0 / interval;
         writtenBytesThroughput = (writtenBytes - lastWrittenBytes) * 1000.0 / interval;
         readMessagesThroughput = (readMessages - lastReadMessages) * 1000.0 / interval;
         writtenMessagesThroughput = (writtenMessages - lastWrittenMessages) * 1000.0 / interval;
-        
+
         lastReadBytes = readBytes;
         lastWrittenBytes = writtenBytes;
         lastReadMessages = readMessages;
         lastWrittenMessages = writtenMessages;
-        
+
         lastThroughputCalculationTime = currentTime;
     }
 
@@ -496,7 +495,7 @@
         return scheduledWriteMessages.get();
     }
 
-    protected void setScheduledWriteBytes(long byteCount){
+    protected void setScheduledWriteBytes(int byteCount){
         scheduledWriteBytes.set(byteCount);
     }
 
@@ -508,7 +507,7 @@
         if (increment <= 0) {
             return;
         }
-        
+
         readBytes += increment;
         lastReadTime = currentTime;
         idleCountForBoth = 0;
@@ -518,19 +517,19 @@
             ((AbstractIoService) getService()).increaseReadBytes(increment, currentTime);
         }
     }
-    
+
     protected final void increaseReadMessages(long currentTime) {
         readMessages++;
         lastReadTime = currentTime;
         idleCountForBoth = 0;
         idleCountForRead = 0;
-        
+
         if (getService() instanceof AbstractIoService) {
             ((AbstractIoService) getService()).increaseReadMessages(currentTime);
         }
     }
 
-    protected final void increaseWrittenBytes(long increment, long currentTime) {
+    protected final void increaseWrittenBytes(int increment, long currentTime) {
         if (increment <= 0) {
             return;
         }
@@ -566,7 +565,7 @@
         decreaseScheduledWriteMessages();
     }
 
-    protected final void increaseScheduledWriteBytes(long increment) {
+    protected final void increaseScheduledWriteBytes(int increment) {
         scheduledWriteBytes.addAndGet(increment);
         if (getService() instanceof AbstractIoService) {
             ((AbstractIoService) getService()).increaseScheduledWriteBytes(increment);
@@ -607,11 +606,11 @@
         }
         return writeRequestQueue;
     }
-    
+
     public final WriteRequest getCurrentWriteRequest() {
         return currentWriteRequest;
     }
-    
+
     public final Object getCurrentWriteMessage() {
         WriteRequest req = getCurrentWriteRequest();
         if (req == null) {
@@ -619,7 +618,7 @@
         }
         return req.getMessage();
     }
-    
+
     protected final void setCurrentWriteRequest(WriteRequest currentWriteRequest) {
         this.currentWriteRequest = currentWriteRequest;
     }
@@ -691,7 +690,7 @@
     public final boolean isWriterIdle() {
         return isIdle(IdleStatus.WRITER_IDLE);
     }
-    
+
     public final int getIdleCount(IdleStatus status) {
         if (getConfig().getIdleTime(status) == 0) {
             if (status == IdleStatus.BOTH_IDLE) {
@@ -806,7 +805,7 @@
                     getLocalAddress() + " => " + getRemoteAddress() + ')';
         }
     }
-    
+
     private String getIdAsString() {
         String id = Long.toHexString(getId()).toUpperCase();
 
@@ -816,7 +815,7 @@
             id = '0' + id; // padding
         }
         id = "0x" + id;
-        
+
         return id;
     }
 
@@ -828,11 +827,11 @@
             return tm.getProviderName() + ' ' + tm.getName();
         }
     }
-    
+
     private class CloseRequestAwareWriteRequestQueue implements WriteRequestQueue {
-        
+
         private final WriteRequestQueue q;
-        
+
         public CloseRequestAwareWriteRequestQueue(WriteRequestQueue q) {
             this.q = q;
         }
@@ -846,7 +845,7 @@
             }
             return answer;
         }
-        
+
         public void offer(IoSession session, WriteRequest e) {
             q.offer(session, e);
         }
@@ -854,7 +853,7 @@
         public boolean isEmpty(IoSession session) {
             return q.isEmpty(session);
         }
-        
+
         public void clear(IoSession session) {
             q.clear(session);
         }

Modified: mina/trunk/core/src/main/java/org/apache/mina/common/DummySession.java
URL: http://svn.apache.org/viewvc/mina/trunk/core/src/main/java/org/apache/mina/common/DummySession.java?rev=645480&r1=645479&r2=645480&view=diff
==============================================================================
--- mina/trunk/core/src/main/java/org/apache/mina/common/DummySession.java (original)
+++ mina/trunk/core/src/main/java/org/apache/mina/common/DummySession.java Mon Apr  7 05:40:40 2008
@@ -28,7 +28,7 @@
 /**
  * A dummy {@link IoSession} for unit-testing or non-network-use of
  * the classes that depends on {@link IoSession}.
- * 
+ *
  * <h2>Overriding I/O request methods</h2>
  * All I/O request methods (i.e. {@link #close()}, {@link #write(Object)} and
  * {@link #setTrafficMask(TrafficMask)}) are final and therefore cannot be
@@ -111,9 +111,9 @@
         // Set meaningless default values.
         acceptor.setHandler(new IoHandlerAdapter());
 
-        this.service = acceptor;
+        service = acceptor;
 
-        this.processor = new IoProcessor<IoSession>() {
+        processor = new IoProcessor<IoSession>() {
             public void add(IoSession session) {
             }
 
@@ -259,9 +259,9 @@
 
         this.transportMetadata = transportMetadata;
     }
-    
+
     @Override
-    public void setScheduledWriteBytes(long byteCount){
+    public void setScheduledWriteBytes(int byteCount){
         super.setScheduledWriteBytes(byteCount);
     }
 
@@ -273,7 +273,7 @@
     /**
      * Update all statistical properties related with throughput.  By default
      * this method returns silently without updating the throughput properties
-     * if they were calculated already within last 
+     * if they were calculated already within last
      * {@link IoSessionConfig#getThroughputCalculationInterval() calculation interval}.
      * If, however, <tt>force</tt> is specified as <tt>true</tt>, this method
      * updates the throughput properties immediately.

Modified: mina/trunk/core/src/main/java/org/apache/mina/common/IoService.java
URL: http://svn.apache.org/viewvc/mina/trunk/core/src/main/java/org/apache/mina/common/IoService.java?rev=645480&r1=645479&r2=645480&view=diff
==============================================================================
--- mina/trunk/core/src/main/java/org/apache/mina/common/IoService.java (original)
+++ mina/trunk/core/src/main/java/org/apache/mina/common/IoService.java Mon Apr  7 05:40:40 2008
@@ -46,22 +46,22 @@
      * related with this service.
      */
     void removeListener(IoServiceListener listener);
-    
+
     /**
      * Returns <tt>true</tt> if and if only {@link #dispose()} method has
      * been called.  Please note that this method will return <tt>true</tt>
      * even after all the related resources are released.
      */
     boolean isDisposing();
-    
+
     /**
      * Returns <tt>true</tt> if and if only all resources of this processor
      * have been disposed.
      */
     boolean isDisposed();
-    
+
     /**
-     * Releases any resources allocated by this service.  Please note that 
+     * Releases any resources allocated by this service.  Please note that
      * this method might block as long as there are any sessions managed by
      * this service.
      */
@@ -83,19 +83,19 @@
      * @return the sessions. An empty collection if there's no session.
      */
     Set<IoSession> getManagedSessions();
-    
+
     /**
      * Returns the number of all sessions which are currently managed by this
      * service.
      */
     int getManagedSessionCount();
-    
+
     /**
      * Returns the maximum number of sessions which were being managed at the
      * same time.
      */
     int getLargestManagedSessionCount();
-    
+
     /**
      * Returns the cumulative number of sessions which were managed (or are
      * being managed) by this service, which means 'currently managed session
@@ -169,25 +169,25 @@
      * Returns the time in millis when write operation occurred lastly.
      */
     long getLastWriteTime();
-    
+
     /**
      * Returns <code>true</code> if this service is idle for the specified
      * {@link IdleStatus}.
      */
     boolean isIdle(IdleStatus status);
-    
+
     /**
      * Returns <code>true</code> if this service is {@link IdleStatus#READER_IDLE}.
      * @see #isIdle(IdleStatus)
      */
     boolean isReaderIdle();
-    
+
     /**
      * Returns <code>true</code> if this service is {@link IdleStatus#WRITER_IDLE}.
      * @see #isIdle(IdleStatus)
      */
     boolean isWriterIdle();
-    
+
     /**
      * Returns <code>true</code> if this service is {@link IdleStatus#BOTH_IDLE}.
      * @see #isIdle(IdleStatus)
@@ -219,34 +219,34 @@
      * @see #getIdleCount(IdleStatus)
      */
     int getWriterIdleCount();
-    
+
     /**
      * Returns the number of the fired continuous <tt>serviceIdle</tt> events
      * for {@link IdleStatus#BOTH_IDLE}.
      * @see #getIdleCount(IdleStatus)
      */
     int getBothIdleCount();
-    
+
     /**
      * Returns the time in milliseconds when the last <tt>serviceIdle</tt> event
      * is fired for the specified {@link IdleStatus}.
      */
     long getLastIdleTime(IdleStatus status);
-    
+
     /**
      * Returns the time in milliseconds when the last <tt>serviceIdle</tt> event
      * is fired for {@link IdleStatus#READER_IDLE}.
      * @see #getLastIdleTime(IdleStatus)
      */
     long getLastReaderIdleTime();
-    
+
     /**
      * Returns the time in milliseconds when the last <tt>serviceIdle</tt> event
      * is fired for {@link IdleStatus#WRITER_IDLE}.
      * @see #getLastIdleTime(IdleStatus)
      */
     long getLastWriterIdleTime();
-    
+
     /**
      * Returns the time in milliseconds when the last <tt>serviceIdle</tt> event
      * is fired for {@link IdleStatus#BOTH_IDLE}.
@@ -258,7 +258,7 @@
      * Returns idle time for the specified type of idleness in seconds.
      */
     int getIdleTime(IdleStatus status);
-    
+
     /**
      * Returns idle time for the specified type of idleness in milliseconds.
      */
@@ -268,47 +268,47 @@
      * Sets idle time for the specified type of idleness in seconds.
      */
     void setIdleTime(IdleStatus status, int idleTime);
-    
+
     /**
      * Returns idle time for {@link IdleStatus#READER_IDLE} in seconds.
      */
     int getReaderIdleTime();
-    
+
     /**
      * Returns idle time for {@link IdleStatus#READER_IDLE} in milliseconds.
      */
     long getReaderIdleTimeInMillis();
-    
+
     /**
      * Sets idle time for {@link IdleStatus#READER_IDLE} in seconds.
      */
     void setReaderIdleTime(int idleTime);
-    
+
     /**
      * Returns idle time for {@link IdleStatus#WRITER_IDLE} in seconds.
      */
     int getWriterIdleTime();
-    
+
     /**
      * Returns idle time for {@link IdleStatus#WRITER_IDLE} in milliseconds.
      */
     long getWriterIdleTimeInMillis();
-    
+
     /**
      * Sets idle time for {@link IdleStatus#WRITER_IDLE} in seconds.
      */
     void setWriterIdleTime(int idleTime);
-    
+
     /**
      * Returns idle time for {@link IdleStatus#BOTH_IDLE} in seconds.
      */
     int getBothIdleTime();
-    
+
     /**
      * Returns idle time for {@link IdleStatus#BOTH_IDLE} in milliseconds.
      */
     long getBothIdleTimeInMillis();
-    
+
     /**
      * Sets idle time for {@link IdleStatus#WRITER_IDLE} in seconds.
      */
@@ -345,7 +345,7 @@
      * 	The number of messages this service has written
      */
     long getWrittenMessages();
-    
+
     /**
      * Returns the number of read bytes per second.
      */
@@ -355,32 +355,32 @@
      * Returns the number of written bytes per second.
      */
     double getWrittenBytesThroughput();
-    
+
     /**
      * Returns the number of read messages per second.
      */
     double getReadMessagesThroughput();
-    
+
     /**
      * Returns the number of written messages per second.
      */
     double getWrittenMessagesThroughput();
-    
+
     /**
      * Returns the maximum of the {@link #getReadBytesThroughput() readBytesThroughput}.
      */
     double getLargestReadBytesThroughput();
-    
+
     /**
      * Returns the maximum of the {@link #getWrittenBytesThroughput() writtenBytesThroughput}.
      */
     double getLargestWrittenBytesThroughput();
-    
+
     /**
      * Returns the maximum of the {@link #getReadMessagesThroughput() readMessagesThroughput}.
      */
     double getLargestReadMessagesThroughput();
-    
+
     /**
      * Returns the maximum of the {@link #getWrittenMessagesThroughput() writtenMessagesThroughput}.
      */
@@ -391,13 +391,13 @@
      * The default value is <tt>3</tt> seconds.
      */
     int getThroughputCalculationInterval();
-    
+
     /**
      * Returns the interval (milliseconds) between each throughput calculation.
      * The default value is <tt>3</tt> seconds.
      */
     long getThroughputCalculationIntervalInMillis();
-    
+
     /**
      * Sets the interval (seconds) between each throughput calculation.  The
      * default value is <tt>3</tt> seconds.
@@ -410,7 +410,7 @@
      * @return
      * 	The number of bytes scheduled to be written
      */
-    long getScheduledWriteBytes();
+    int getScheduledWriteBytes();
 
     /**
      * Returns the number of messages scheduled to be written
@@ -418,7 +418,7 @@
      * @return
      * 	The number of messages scheduled to be written
      */
-    long getScheduledWriteMessages();
+    int getScheduledWriteMessages();
 
     /**
      * Writes the specified {@code message} to all the {@link IoSession}s
@@ -426,13 +426,13 @@
      * {@link IoUtil#broadcast(Object, Collection)}.
      */
     Set<WriteFuture> broadcast(Object message);
-    
+
     /**
      * Returns the {@link IoSessionDataStructureFactory} that provides
      * related data structures for a new session created by this service.
      */
     IoSessionDataStructureFactory getSessionDataStructureFactory();
-    
+
     /**
      * Sets the {@link IoSessionDataStructureFactory} that provides
      * related data structures for a new session created by this service.

Modified: mina/trunk/core/src/main/java/org/apache/mina/transport/vmpipe/VmPipeSessionImpl.java
URL: http://svn.apache.org/viewvc/mina/trunk/core/src/main/java/org/apache/mina/transport/vmpipe/VmPipeSessionImpl.java?rev=645480&r1=645479&r2=645480&view=diff
==============================================================================
--- mina/trunk/core/src/main/java/org/apache/mina/transport/vmpipe/VmPipeSessionImpl.java (original)
+++ mina/trunk/core/src/main/java/org/apache/mina/transport/vmpipe/VmPipeSessionImpl.java Mon Apr  7 05:40:40 2008
@@ -80,12 +80,12 @@
                       VmPipeAddress localAddress, IoHandler handler, VmPipe remoteEntry) {
         this.service = service;
         this.serviceListeners = serviceListeners;
-        this.lock = new ReentrantLock();
+        lock = new ReentrantLock();
         this.localAddress = localAddress;
-        this.remoteAddress = this.serviceAddress = remoteEntry.getAddress();
+        remoteAddress = serviceAddress = remoteEntry.getAddress();
         this.handler = handler;
-        this.filterChain = new VmPipeFilterChain(this);
-        this.receivedMessageQueue = new LinkedBlockingQueue<Object>();
+        filterChain = new VmPipeFilterChain(this);
+        receivedMessageQueue = new LinkedBlockingQueue<Object>();
 
         remoteSession = new VmPipeSessionImpl(this, remoteEntry);
     }
@@ -94,15 +94,15 @@
      * Constructor for server-side session.
      */
     private VmPipeSessionImpl(VmPipeSessionImpl remoteSession, VmPipe entry) {
-        this.service = entry.getAcceptor();
-        this.serviceListeners = entry.getListeners();
-        this.lock = remoteSession.lock;
-        this.localAddress = this.serviceAddress = remoteSession.remoteAddress;
-        this.remoteAddress = remoteSession.localAddress;
-        this.handler = entry.getHandler();
-        this.filterChain = new VmPipeFilterChain(this);
+        service = entry.getAcceptor();
+        serviceListeners = entry.getListeners();
+        lock = remoteSession.lock;
+        localAddress = serviceAddress = remoteSession.remoteAddress;
+        remoteAddress = remoteSession.localAddress;
+        handler = entry.getHandler();
+        filterChain = new VmPipeFilterChain(this);
         this.remoteSession = remoteSession;
-        this.receivedMessageQueue = new LinkedBlockingQueue<Object>();
+        receivedMessageQueue = new LinkedBlockingQueue<Object>();
     }
 
     public IoService getService() {
@@ -150,8 +150,8 @@
     public VmPipeAddress getServiceAddress() {
         return serviceAddress;
     }
-    
-    void increaseWrittenBytes0(long increment, long currentTime) {
+
+    void increaseWrittenBytes0(int increment, long currentTime) {
         super.increaseWrittenBytes(increment, currentTime);
     }