You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by Jordan Zimmerman <jo...@jordanzimmerman.com> on 2013/03/21 01:51:38 UTC

Is this a bug?

I got a bug report for Curator today and while debugging it I found out that if the current thread is interrupted, ZK node creation will succeed on the server, but the create() method will throw an exception. The InterruptedException gets thrown by ClientCnxn.submitRequest() when packet.wait() is called. This makes the lock recipe even trickier. If an app uses a single ZK handle in a multi-threaded app and is shutting down one of the threads, the thread could potentially start a lock recipe before it realizes the thread is interrupted. That thread will try to create an EPHEMERAL_SEQUENTIAL node which will succeed on the server, but the thread will throw an exception. The client has no way of knowing what it's node is and, because it's an interrupted exception, might not even know to delete it.

Curator handles KeeperExceptions in its lock recipe by inserting a GUID in the node name and calling getChildren when a KeeperException is caught during EPHEMERAL_SEQUENTIAL node create. Of course, InterruptedException is not in the category of recoverable exception. I'd appreciate this ml's thoughts on this. Curator could handle this by checking if the current thread is interrupted before attempting to create the EPHEMERAL_SEQUENTIAL node.

Here's sample code, FYI:

        String connectString = server.getConnectString();
        final CountDownLatch latch = new CountDownLatch(1);
        Watcher watcher = new Watcher()
        {
            @Override
            public void process(WatchedEvent event)
            {
                if ( event.getState() == Event.KeeperState.SyncConnected )
                {
                    latch.countDown();
                }
            }
        };
        ZooKeeper zk = new ZooKeeper(connectString, 10000, watcher);
        latch.await();
        try
        {
            zk.create("/test", new byte[]{}, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
            Thread.currentThread().interrupt();
            zk.create("/test/x-", new byte[]{}, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL);
        }
        catch ( InterruptedException e )
        {
            Thread.interrupted();   // clear
        }
        System.out.println(zk.getChildren("/test", false));
        zk.close();


Re: Is this a bug?

Posted by Camille Fournier <ca...@apache.org>.
I think in general if you want Curator to behave well in the environment
you describe, where threads may be interrupted as part of a partial or
complete shut down, you might need to implement some sort of lifecycle
notion that they can tie into their framework. You're essentially providing
a transactional resource in this as a lock manager, if people want to use
it as such, that resource needs to know when things are shutting down. I'd
expect that the end result of whatever is calling interrupt to be shutting
down the process (thus closing the ZK connection), but it might hang while
some other part of the shutdown lifecycle completes. So Curator needs to
know that "close" should be called when the shutdown lifecycle is initiated
so even if it got a lock during that process it would immediately release
it by closing the client.

I don't know if it makes sense to try and handle interrupted exceptions
beyond lifecycle management... if someone calls interrupt they probably
need you to check the state of some flag that indicates a state change for
the whole Curator.

Does that make sense?


On Wed, Mar 20, 2013 at 11:55 PM, Benjamin Reed <br...@apache.org> wrote:

> do you know why the thread was interrupted? i don't think it helps to
> check if the thread is interrupted before you start since you may not
> be interrupted when you start, but later receive an interrupt when you
> are waiting for a response.
>
> interrupts are a pain to handle unless things are shutting down (in
> which case you are going away anyway). you could easily handle them
> when the connectionloss code path, but that kind of defeats the point
> of interrupting...
>
> ben
>
> On Wed, Mar 20, 2013 at 5:51 PM, Jordan Zimmerman
> <jo...@jordanzimmerman.com> wrote:
> > I got a bug report for Curator today and while debugging it I found out
> that if the current thread is interrupted, ZK node creation will succeed on
> the server, but the create() method will throw an exception. The
> InterruptedException gets thrown by ClientCnxn.submitRequest() when
> packet.wait() is called. This makes the lock recipe even trickier. If an
> app uses a single ZK handle in a multi-threaded app and is shutting down
> one of the threads, the thread could potentially start a lock recipe before
> it realizes the thread is interrupted. That thread will try to create an
> EPHEMERAL_SEQUENTIAL node which will succeed on the server, but the thread
> will throw an exception. The client has no way of knowing what it's node is
> and, because it's an interrupted exception, might not even know to delete
> it.
> >
> > Curator handles KeeperExceptions in its lock recipe by inserting a GUID
> in the node name and calling getChildren when a KeeperException is caught
> during EPHEMERAL_SEQUENTIAL node create. Of course, InterruptedException is
> not in the category of recoverable exception. I'd appreciate this ml's
> thoughts on this. Curator could handle this by checking if the current
> thread is interrupted before attempting to create the EPHEMERAL_SEQUENTIAL
> node.
> >
> > Here's sample code, FYI:
> >
> >         String connectString = server.getConnectString();
> >         final CountDownLatch latch = new CountDownLatch(1);
> >         Watcher watcher = new Watcher()
> >         {
> >             @Override
> >             public void process(WatchedEvent event)
> >             {
> >                 if ( event.getState() == Event.KeeperState.SyncConnected
> )
> >                 {
> >                     latch.countDown();
> >                 }
> >             }
> >         };
> >         ZooKeeper zk = new ZooKeeper(connectString, 10000, watcher);
> >         latch.await();
> >         try
> >         {
> >             zk.create("/test", new byte[]{},
> ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
> >             Thread.currentThread().interrupt();
> >             zk.create("/test/x-", new byte[]{},
> ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL);
> >         }
> >         catch ( InterruptedException e )
> >         {
> >             Thread.interrupted();   // clear
> >         }
> >         System.out.println(zk.getChildren("/test", false));
> >         zk.close();
> >
>

Re: Is this a bug?

Posted by Benjamin Reed <br...@apache.org>.
do you know why the thread was interrupted? i don't think it helps to
check if the thread is interrupted before you start since you may not
be interrupted when you start, but later receive an interrupt when you
are waiting for a response.

interrupts are a pain to handle unless things are shutting down (in
which case you are going away anyway). you could easily handle them
when the connectionloss code path, but that kind of defeats the point
of interrupting...

ben

On Wed, Mar 20, 2013 at 5:51 PM, Jordan Zimmerman
<jo...@jordanzimmerman.com> wrote:
> I got a bug report for Curator today and while debugging it I found out that if the current thread is interrupted, ZK node creation will succeed on the server, but the create() method will throw an exception. The InterruptedException gets thrown by ClientCnxn.submitRequest() when packet.wait() is called. This makes the lock recipe even trickier. If an app uses a single ZK handle in a multi-threaded app and is shutting down one of the threads, the thread could potentially start a lock recipe before it realizes the thread is interrupted. That thread will try to create an EPHEMERAL_SEQUENTIAL node which will succeed on the server, but the thread will throw an exception. The client has no way of knowing what it's node is and, because it's an interrupted exception, might not even know to delete it.
>
> Curator handles KeeperExceptions in its lock recipe by inserting a GUID in the node name and calling getChildren when a KeeperException is caught during EPHEMERAL_SEQUENTIAL node create. Of course, InterruptedException is not in the category of recoverable exception. I'd appreciate this ml's thoughts on this. Curator could handle this by checking if the current thread is interrupted before attempting to create the EPHEMERAL_SEQUENTIAL node.
>
> Here's sample code, FYI:
>
>         String connectString = server.getConnectString();
>         final CountDownLatch latch = new CountDownLatch(1);
>         Watcher watcher = new Watcher()
>         {
>             @Override
>             public void process(WatchedEvent event)
>             {
>                 if ( event.getState() == Event.KeeperState.SyncConnected )
>                 {
>                     latch.countDown();
>                 }
>             }
>         };
>         ZooKeeper zk = new ZooKeeper(connectString, 10000, watcher);
>         latch.await();
>         try
>         {
>             zk.create("/test", new byte[]{}, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
>             Thread.currentThread().interrupt();
>             zk.create("/test/x-", new byte[]{}, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL);
>         }
>         catch ( InterruptedException e )
>         {
>             Thread.interrupted();   // clear
>         }
>         System.out.println(zk.getChildren("/test", false));
>         zk.close();
>