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

Heng Chen commented on HBASE-15278:
-----------------------------------

It seems testRTEDuringConnectionSetup inject one exception in handler pipeLine, 
 but we catch all throwable in handler and close the channel,  So we got 
unexpected exception in catch scope like below

{code}
   } catch (Exception e) {
      LOG.info("Caught expected exception: " + e.toString());
      assertTrue(StringUtils.stringifyException(e).contains("Injected fault")); 
 // Test case failed here
    }
{code}

I can see some logs 

{code}
2016-02-18 11:27:27,182 INFO  [main] ipc.AbstractTestIPC(298): Caught expected 
exception: org.apache.hadoop.hbase.exceptions.ConnectionClosingException: Call 
id=0 on server /127.0.0.1:63478 aborted: connection is closing
{code}

IMO We can change the code like below

{code}
      if (!(e instanceof ConnectionClosingException)) {
        LOG.info("Caught expected exception: " + e.toString());
        assertTrue(StringUtils.stringifyException(e).contains("Injected 
fault"));
      }
{code}


> AsyncRPCClient hangs if Connection closes before RPC call response 
> -------------------------------------------------------------------
>
>                 Key: HBASE-15278
>                 URL: https://issues.apache.org/jira/browse/HBASE-15278
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Enis Soztutar
>            Priority: Blocker
>             Fix For: 2.0.0
>
>         Attachments: hbase-15278_v00.patch
>
>
> The test for HBASE-15212 discovered an issue with Async RPC Client. 
> In that test, we are closing the connection if an RPC call writes a call 
> larger than max allowed size, the server closes the connection. However the 
> async client does not seem to handle connection closes with outstanding RPC 
> calls. The client just hangs. 
> Marking this blocker against 2.0 since it is default there. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to