You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@avro.apache.org by Alex Miller <al...@puredanger.com> on 2011/06/13 18:37:59 UTC

SaslSocketServer.close() doesn't close?

Hello all,

I've been having an issue in some tests where it appears that closing
a SaslSocketServer does not actually cause the socket to be closed.
The code below reproduces roughly what happens in the course of a much
more complicated test.  Basically I start a socket server, close() it,
then try to start another on the same port which fails to bind as the
address is in use.

>From trolling through the code, it is not immediately obvious that any
code exists to actually close the socket.  The nio thread group gets
interrupted which from debugging into this, does seem to cause the
Connection threads to break out of their loop and close but the main
socket listener (which is not a member of the thread group) and the
socket channel do not get closed.

Am I crazy?



import java.net.InetAddress;
import java.net.InetSocketAddress;

import org.apache.avro.Protocol;
import org.apache.avro.Schema;
import org.apache.avro.Protocol.Message;
import org.apache.avro.io.Decoder;
import org.apache.avro.io.Encoder;
import org.apache.avro.ipc.Responder;
import org.apache.avro.ipc.SaslSocketServer;
import org.apache.avro.ipc.Server;

public class SocketDoesntClose extends Responder {

  public static void main(String[] args) throws Exception {

    Protocol prot = new Protocol("c", "a.b");

    // start on port 9999
    InetSocketAddress addr = new
InetSocketAddress(InetAddress.getLocalHost(), 9999);
    Server server = new SaslSocketServer(new SocketDoesntClose(prot), addr);
    server.start();

    // stop -- I would expect this to fully release the socket
    server.close();

    Thread.sleep(10000);

    // start on same socket again -> address already in use
    server = new SaslSocketServer(new SocketDoesntClose(prot), addr);
  }

  // dummy implementation
  public SocketDoesntClose(Protocol local) {
    super(local);
  }

  public Object readRequest(Schema actual, Schema expected, Decoder in) {
    return null;
  }

  public Object respond(Message message, Object request) {
    return null;
  }

  public void writeError(Schema schema, Object error, Encoder out) {
  }

  public void writeResponse(Schema schema, Object response, Encoder out) {
  }
}

Re: SaslSocketServer.close() doesn't close?

Posted by James Baldassari <jb...@gmail.com>.
Hi Alex,

The test you posted works fine for me, even using a sleep time as short as
5ms between closing and reopening.

Looking at trunk, the code to close the socket is in the finally block
inside SocketServer.run().  When close() is called it sends an interrupt to
the SocketServer thread, which causes the channel.accept() call to unblock
and throw java.nio.channels.ClosedByInterruptException.
ClosedByInterruptException is a ClosedChannelException, which is caught
inside the channel.accept() try/catch block and causes the run() method to
return.  Before the method returns, the finally block fires, which closes
the channel.  You should be able to verify that the finally block is called
when you see the "INFO org.apache.avro.ipc.SocketServer - stopping" log
message.  One improvement that could be made to this code is modifying the
close() method to block until the socket has finished closing.  An
overloaded close(Boolean) could be added that takes a flag indicating
whether the call should return immediately or block until the socket has
closed.

10 seconds is a long time, but does your test pass if you increase the sleep
time?  Do you ever see the "stopping" log message?  Can you step through
your test in a debugger and see what's happening inside SocketServer when
you call close()?  What is your environment like?  JVM version, Linux distro
and kernel, etc.

-James


On Mon, Jun 13, 2011 at 12:37 PM, Alex Miller <al...@puredanger.com> wrote:

> Hello all,
>
> I've been having an issue in some tests where it appears that closing
> a SaslSocketServer does not actually cause the socket to be closed.
> The code below reproduces roughly what happens in the course of a much
> more complicated test.  Basically I start a socket server, close() it,
> then try to start another on the same port which fails to bind as the
> address is in use.
>
> From trolling through the code, it is not immediately obvious that any
> code exists to actually close the socket.  The nio thread group gets
> interrupted which from debugging into this, does seem to cause the
> Connection threads to break out of their loop and close but the main
> socket listener (which is not a member of the thread group) and the
> socket channel do not get closed.
>
> Am I crazy?
>
>
>
> import java.net.InetAddress;
> import java.net.InetSocketAddress;
>
> import org.apache.avro.Protocol;
> import org.apache.avro.Schema;
> import org.apache.avro.Protocol.Message;
> import org.apache.avro.io.Decoder;
> import org.apache.avro.io.Encoder;
> import org.apache.avro.ipc.Responder;
> import org.apache.avro.ipc.SaslSocketServer;
> import org.apache.avro.ipc.Server;
>
> public class SocketDoesntClose extends Responder {
>
>  public static void main(String[] args) throws Exception {
>
>    Protocol prot = new Protocol("c", "a.b");
>
>    // start on port 9999
>    InetSocketAddress addr = new
> InetSocketAddress(InetAddress.getLocalHost(), 9999);
>    Server server = new SaslSocketServer(new SocketDoesntClose(prot), addr);
>    server.start();
>
>    // stop -- I would expect this to fully release the socket
>    server.close();
>
>    Thread.sleep(10000);
>
>    // start on same socket again -> address already in use
>    server = new SaslSocketServer(new SocketDoesntClose(prot), addr);
>  }
>
>  // dummy implementation
>  public SocketDoesntClose(Protocol local) {
>    super(local);
>  }
>
>  public Object readRequest(Schema actual, Schema expected, Decoder in) {
>    return null;
>  }
>
>  public Object respond(Message message, Object request) {
>    return null;
>  }
>
>  public void writeError(Schema schema, Object error, Encoder out) {
>  }
>
>  public void writeResponse(Schema schema, Object response, Encoder out) {
>  }
> }
>

Re: SaslSocketServer.close() doesn't close?

Posted by Holger Hoffstätte <ho...@googlemail.com>.
On 14.06.2011 08:49, Holger Hoffstätte mistakenly wrote:
> Both your test and Alex' original code fail for me on Windows7 with JDK
> 1.6.25. Your test hangs, Alex' code fails on re-bind()ing. I'm running
> against svn trunk. Could this be a platform-dependenant behaviour?

No, the problem is that I was NOT running against trunk but rather
against 1.5.1. trunk works as expected since it has the missing
this.interrupt() in SocketServer.close().

yay!

-h

Re: SaslSocketServer.close() doesn't close?

Posted by James Baldassari <jb...@gmail.com>.
I ran the test on Linux, so yes, it could be a platform-specific problem.  I
think this is the more likely cause, though:

https://issues.apache.org/jira/browse/AVRO-776

Maybe that fix isn't in the version of Avro that you guys are
using/testing.  That would certainly explain the symptoms.  Maybe it works
for me because I was running the test against trunk rather than a tag.

-James


2011/6/14 Holger Hoffstätte <ho...@googlemail.com>

>
> On 14.06.2011 08:32, James Baldassari wrote:
> > I made a JUnit test from Alex's example and modified it to join on the
> > SocketServer after calling close(), as you suggested.  This doesn't hang
> > for me, and it doesn't throw any exceptions when re-binding to the same
> > port.  The test I ran is here:
> >
> > http://pastebin.com/zWPv3MXF
>
> Both your test and Alex' original code fail for me on Windows7 with JDK
> 1.6.25. Your test hangs, Alex' code fails on re-bind()ing. I'm running
> against svn trunk. Could this be a platform-dependenant behaviour?
> Curiously I haven't noticed any problems with this in my own tests, but
> in those I do accept connections/exchange data etc.
>
> -h
>

Re: SaslSocketServer.close() doesn't close?

Posted by Holger Hoffstätte <ho...@googlemail.com>.
On 14.06.2011 08:32, James Baldassari wrote:
> I made a JUnit test from Alex's example and modified it to join on the
> SocketServer after calling close(), as you suggested.  This doesn't hang
> for me, and it doesn't throw any exceptions when re-binding to the same
> port.  The test I ran is here:
> 
> http://pastebin.com/zWPv3MXF

Both your test and Alex' original code fail for me on Windows7 with JDK
1.6.25. Your test hangs, Alex' code fails on re-bind()ing. I'm running
against svn trunk. Could this be a platform-dependenant behaviour?
Curiously I haven't noticed any problems with this in my own tests, but
in those I do accept connections/exchange data etc.

-h

Re: SaslSocketServer.close() doesn't close?

Posted by James Baldassari <jb...@gmail.com>.
Hi Sudhan,

The parts of SocketServer that we need to examine are the run() and close()
methods.  Removing everything but the core functionality, these methods look
something like this (in trunk):

public void run() {
  try {
    while (true) {
      try {
        new Connection(channel.accept());
      } catch (ClosedChannelException e) {
        return;  // caught when the SocketServer thread is interrupted in
close()
      }
    }
  } finally {
    channel.close(); // channel is always closed before run() returns
  }
}

public void close() {
  this.interrupt(); // interrupts the SocketServer
  group.interrupt(); // interrupts the connection thread group
}

The first statement in the close() method interrupts the SocketServer
thread, which causes that thread to break out of the channel.accept() call
and the while loop in the run() method (after catching
ClosedChannelException).  When the finally block runs, the channel will be
closed.  The SocketServer thread does not need to be added to the thread
group because there is a separate interrupt call for the SocketServer thread
(this.interrupt()).

I made a JUnit test from Alex's example and modified it to join on the
SocketServer after calling close(), as you suggested.  This doesn't hang for
me, and it doesn't throw any exceptions when re-binding to the same port.
The test I ran is here:

http://pastebin.com/zWPv3MXF

Can you try running that to see if you get the same result?  By the way,
what version of Avro are you using?  Is it something other than trunk?
Maybe that explains the difference in behavior.

-James


On Tue, Jun 14, 2011 at 1:38 AM, Sudharsan Sampath <su...@gmail.com>wrote:

> I don't see the daemon SocketServer thread that was created as part of the
> server.start() call added to the 'group' ThreadGroup. Only any new
> connections that the socket accepts are added to the threadGroup and only
> these are the ones that will get interrupted on the close call.
>
> In our example with no clients requesting access, the thread group array
> remains empty and send no interrupt at all. Could this be the issue for the
> server to hang indefinitely?
>
> -Sudhan S
>
>
> On Tue, Jun 14, 2011 at 10:35 AM, Sudharsan Sampath <su...@gmail.com>wrote:
>
>> Hi,
>>
>> I tried but have the same issue. I added server.join() after the close
>> call instead of waiting for a pre-defined time. But it hangs indefinitely.
>>
>> -Sudhan S
>>
>>
>> On Mon, Jun 13, 2011 at 10:07 PM, Alex Miller <al...@puredanger.com>wrote:
>>
>>> Hello all,
>>>
>>> I've been having an issue in some tests where it appears that closing
>>> a SaslSocketServer does not actually cause the socket to be closed.
>>> The code below reproduces roughly what happens in the course of a much
>>> more complicated test.  Basically I start a socket server, close() it,
>>> then try to start another on the same port which fails to bind as the
>>> address is in use.
>>>
>>> From trolling through the code, it is not immediately obvious that any
>>> code exists to actually close the socket.  The nio thread group gets
>>> interrupted which from debugging into this, does seem to cause the
>>> Connection threads to break out of their loop and close but the main
>>> socket listener (which is not a member of the thread group) and the
>>> socket channel do not get closed.
>>>
>>> Am I crazy?
>>>
>>>
>>>
>>> import java.net.InetAddress;
>>> import java.net.InetSocketAddress;
>>>
>>> import org.apache.avro.Protocol;
>>> import org.apache.avro.Schema;
>>> import org.apache.avro.Protocol.Message;
>>> import org.apache.avro.io.Decoder;
>>> import org.apache.avro.io.Encoder;
>>> import org.apache.avro.ipc.Responder;
>>> import org.apache.avro.ipc.SaslSocketServer;
>>> import org.apache.avro.ipc.Server;
>>>
>>> public class SocketDoesntClose extends Responder {
>>>
>>>  public static void main(String[] args) throws Exception {
>>>
>>>    Protocol prot = new Protocol("c", "a.b");
>>>
>>>    // start on port 9999
>>>    InetSocketAddress addr = new
>>> InetSocketAddress(InetAddress.getLocalHost(), 9999);
>>>    Server server = new SaslSocketServer(new SocketDoesntClose(prot),
>>> addr);
>>>    server.start();
>>>
>>>    // stop -- I would expect this to fully release the socket
>>>    server.close();
>>>
>>>    Thread.sleep(10000);
>>>
>>>    // start on same socket again -> address already in use
>>>    server = new SaslSocketServer(new SocketDoesntClose(prot), addr);
>>>  }
>>>
>>>  // dummy implementation
>>>  public SocketDoesntClose(Protocol local) {
>>>    super(local);
>>>  }
>>>
>>>  public Object readRequest(Schema actual, Schema expected, Decoder in) {
>>>    return null;
>>>  }
>>>
>>>  public Object respond(Message message, Object request) {
>>>    return null;
>>>  }
>>>
>>>  public void writeError(Schema schema, Object error, Encoder out) {
>>>  }
>>>
>>>  public void writeResponse(Schema schema, Object response, Encoder out) {
>>>  }
>>> }
>>>
>>
>>
>

Re: SaslSocketServer.close() doesn't close?

Posted by Sudharsan Sampath <su...@gmail.com>.
I don't see the daemon SocketServer thread that was created as part of the
server.start() call added to the 'group' ThreadGroup. Only any new
connections that the socket accepts are added to the threadGroup and only
these are the ones that will get interrupted on the close call.

In our example with no clients requesting access, the thread group array
remains empty and send no interrupt at all. Could this be the issue for the
server to hang indefinitely?

-Sudhan S

On Tue, Jun 14, 2011 at 10:35 AM, Sudharsan Sampath <su...@gmail.com>wrote:

> Hi,
>
> I tried but have the same issue. I added server.join() after the close call
> instead of waiting for a pre-defined time. But it hangs indefinitely.
>
> -Sudhan S
>
>
> On Mon, Jun 13, 2011 at 10:07 PM, Alex Miller <al...@puredanger.com> wrote:
>
>> Hello all,
>>
>> I've been having an issue in some tests where it appears that closing
>> a SaslSocketServer does not actually cause the socket to be closed.
>> The code below reproduces roughly what happens in the course of a much
>> more complicated test.  Basically I start a socket server, close() it,
>> then try to start another on the same port which fails to bind as the
>> address is in use.
>>
>> From trolling through the code, it is not immediately obvious that any
>> code exists to actually close the socket.  The nio thread group gets
>> interrupted which from debugging into this, does seem to cause the
>> Connection threads to break out of their loop and close but the main
>> socket listener (which is not a member of the thread group) and the
>> socket channel do not get closed.
>>
>> Am I crazy?
>>
>>
>>
>> import java.net.InetAddress;
>> import java.net.InetSocketAddress;
>>
>> import org.apache.avro.Protocol;
>> import org.apache.avro.Schema;
>> import org.apache.avro.Protocol.Message;
>> import org.apache.avro.io.Decoder;
>> import org.apache.avro.io.Encoder;
>> import org.apache.avro.ipc.Responder;
>> import org.apache.avro.ipc.SaslSocketServer;
>> import org.apache.avro.ipc.Server;
>>
>> public class SocketDoesntClose extends Responder {
>>
>>  public static void main(String[] args) throws Exception {
>>
>>    Protocol prot = new Protocol("c", "a.b");
>>
>>    // start on port 9999
>>    InetSocketAddress addr = new
>> InetSocketAddress(InetAddress.getLocalHost(), 9999);
>>    Server server = new SaslSocketServer(new SocketDoesntClose(prot),
>> addr);
>>    server.start();
>>
>>    // stop -- I would expect this to fully release the socket
>>    server.close();
>>
>>    Thread.sleep(10000);
>>
>>    // start on same socket again -> address already in use
>>    server = new SaslSocketServer(new SocketDoesntClose(prot), addr);
>>  }
>>
>>  // dummy implementation
>>  public SocketDoesntClose(Protocol local) {
>>    super(local);
>>  }
>>
>>  public Object readRequest(Schema actual, Schema expected, Decoder in) {
>>    return null;
>>  }
>>
>>  public Object respond(Message message, Object request) {
>>    return null;
>>  }
>>
>>  public void writeError(Schema schema, Object error, Encoder out) {
>>  }
>>
>>  public void writeResponse(Schema schema, Object response, Encoder out) {
>>  }
>> }
>>
>
>

Re: SaslSocketServer.close() doesn't close?

Posted by Sudharsan Sampath <su...@gmail.com>.
Hi,

I tried but have the same issue. I added server.join() after the close call
instead of waiting for a pre-defined time. But it hangs indefinitely.

-Sudhan S

On Mon, Jun 13, 2011 at 10:07 PM, Alex Miller <al...@puredanger.com> wrote:

> Hello all,
>
> I've been having an issue in some tests where it appears that closing
> a SaslSocketServer does not actually cause the socket to be closed.
> The code below reproduces roughly what happens in the course of a much
> more complicated test.  Basically I start a socket server, close() it,
> then try to start another on the same port which fails to bind as the
> address is in use.
>
> From trolling through the code, it is not immediately obvious that any
> code exists to actually close the socket.  The nio thread group gets
> interrupted which from debugging into this, does seem to cause the
> Connection threads to break out of their loop and close but the main
> socket listener (which is not a member of the thread group) and the
> socket channel do not get closed.
>
> Am I crazy?
>
>
>
> import java.net.InetAddress;
> import java.net.InetSocketAddress;
>
> import org.apache.avro.Protocol;
> import org.apache.avro.Schema;
> import org.apache.avro.Protocol.Message;
> import org.apache.avro.io.Decoder;
> import org.apache.avro.io.Encoder;
> import org.apache.avro.ipc.Responder;
> import org.apache.avro.ipc.SaslSocketServer;
> import org.apache.avro.ipc.Server;
>
> public class SocketDoesntClose extends Responder {
>
>  public static void main(String[] args) throws Exception {
>
>    Protocol prot = new Protocol("c", "a.b");
>
>    // start on port 9999
>    InetSocketAddress addr = new
> InetSocketAddress(InetAddress.getLocalHost(), 9999);
>    Server server = new SaslSocketServer(new SocketDoesntClose(prot), addr);
>    server.start();
>
>    // stop -- I would expect this to fully release the socket
>    server.close();
>
>    Thread.sleep(10000);
>
>    // start on same socket again -> address already in use
>    server = new SaslSocketServer(new SocketDoesntClose(prot), addr);
>  }
>
>  // dummy implementation
>  public SocketDoesntClose(Protocol local) {
>    super(local);
>  }
>
>  public Object readRequest(Schema actual, Schema expected, Decoder in) {
>    return null;
>  }
>
>  public Object respond(Message message, Object request) {
>    return null;
>  }
>
>  public void writeError(Schema schema, Object error, Encoder out) {
>  }
>
>  public void writeResponse(Schema schema, Object response, Encoder out) {
>  }
> }
>