You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@bookkeeper.apache.org by "Rakesh R (JIRA)" <ji...@apache.org> on 2013/10/04 12:01:44 UTC

[jira] [Commented] (BOOKKEEPER-688) NPE exception in PerChannelBookieClient

    [ https://issues.apache.org/jira/browse/BOOKKEEPER-688?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13786042#comment-13786042 ] 

Rakesh R commented on BOOKKEEPER-688:
-------------------------------------


IMO, will try to avoid nullifying the channel. I feel it would be more readable if we could utilize ConnectionStates. As an initial step, I just tried to draw state transition diagram and  attaching the same, please feel free to correct.

When seeing the code, I got one doubt - is there any case where channelfuture.isSuccess() and future.getChannel() is returning a null reference ?
Otw I feel 'state' is sufficient to control the channel logic.

{code}
PerChannelBookieClient.java:

                    if (future.isSuccess() && state == ConnectionState.CONNECTING) {
                        LOG.info("Successfully connected to bookie: {}", future.getChannel());
                        rc = BKException.Code.OK;
                        channel = future.getChannel();
                        state = ConnectionState.CONNECTED;
{code}

Also, if any operations comes to the channel after its closure, it would throw 'java.nio.channels.ClosedChannelException'. I think this is fine.

> NPE exception in PerChannelBookieClient
> ---------------------------------------
>
>                 Key: BOOKKEEPER-688
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-688
>             Project: Bookkeeper
>          Issue Type: Bug
>    Affects Versions: 4.2.0
>            Reporter: Rakesh R
>            Assignee: Rakesh R
>         Attachments: PerChannelBookieClient-ConnectionState-diagram.png
>
>
> NPE exception in PerChannelBookieClient:
> {code}
> 2013-10-04 11:56:34,526 - INFO  - [NIOServerFactory-15099:NIOServerFactory$Cnxn@246] - Peer closed connection. rc=-1 java.nio.channels.SocketChannel[connected local=/10.18.170.130:15099 remote=/10.18.170.130:53945]
> 2013-10-04 11:56:34,526 - INFO  - [Thread-93:PerChannelBookieClient@493] - Disconnected from bookie channel [id: 0x006287d3, /10.18.170.130:53945 :> /10.18.170.130:15099]
> 2013-10-04 11:56:34,526 - INFO  - [New I/O client worker #90-3:PerChannelBookieClient$1@137] - Successfully connected to bookie: [id: 0x01964fe8, /10.18.170.130:53951 => /10.18.170.130:15100]
> 2013-10-04 11:56:34,542 - INFO  - [NIOServerFactory-15100:NIOServerFactory$Cnxn@246] - Peer closed connection. rc=-1 java.nio.channels.SocketChannel[connected local=/10.18.170.130:15100 remote=/10.18.170.130:53951]
> 2013-10-04 11:56:34,542 - INFO  - [Thread-93:PerChannelBookieClient@493] - Disconnected from bookie channel [id: 0x01964fe8, /10.18.170.130:53951 :> /10.18.170.130:15100]
> 2013-10-04 11:56:34,542 - WARN  - [New I/O client worker #90-3:PerChannelBookieClient@274] - Add entry operation failed
> java.lang.NullPointerException
> 	at org.apache.bookkeeper.proto.PerChannelBookieClient.addEntry(PerChannelBookieClient.java:258)
> 	at org.apache.bookkeeper.proto.BookieClient$2.operationComplete(BookieClient.java:138)
> 	at org.apache.bookkeeper.proto.BookieClient$2.operationComplete(BookieClient.java:1)
> 	at org.apache.bookkeeper.proto.PerChannelBookieClient$1.operationComplete(PerChannelBookieClient.java:173)
> 	at org.jboss.netty.channel.DefaultChannelFuture.notifyListener(DefaultChannelFuture.java:381)
> 	at org.jboss.netty.channel.DefaultChannelFuture.notifyListeners(DefaultChannelFuture.java:372)
> 	at org.jboss.netty.channel.DefaultChannelFuture.setSuccess(DefaultChannelFuture.java:316)
> 	at org.jboss.netty.channel.socket.nio.NioWorker$RegisterTask.run(NioWorker.java:767)
> 	at org.jboss.netty.channel.socket.nio.NioWorker.processRegisterTaskQueue(NioWorker.java:256)
> 	at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:198)
> 	at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
> 	at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:44)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:885)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
> 	at java.lang.Thread.run(Thread.java:619)
> 4 Oct, 2013 11:56:34 AM org.jboss.netty.channel.DefaultChannelFuture
> WARNING: An exception was thrown by ChannelFutureListener.
> {code}
> Here the operation which is performed is
> step-1 addEntry asynchronously
> step-2 Immediately after adding the entry, close the bookie client



--
This message was sent by Atlassian JIRA
(v6.1#6144)