[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2018-04-05 Thread stack (JIRA)

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

stack commented on HBASE-16635:
---

FYI: Got pushed w/ wrong JIRA ID

commit c5b8aababe18f65f5db979128a62d8a0686b9dc5
Author: Ramkrishna 
Date:   Mon Sep 19 16:12:15 2016 +0530

HBASE-16335 RpcClient under heavy load leaks some netty bytebuf (Ram)

> RpcClient under heavy load leaks some netty bytebuf
> ---
>
> Key: HBASE-16635
> URL: https://issues.apache.org/jira/browse/HBASE-16635
> Project: HBase
>  Issue Type: Bug
>Reporter: ramkrishna.s.vasudevan
>Assignee: ramkrishna.s.vasudevan
>Priority: Minor
> Fix For: 2.0.0
>
> Attachments: HBASE-16635.patch, HBASE-16635_1.patch
>
>
> Yet to analyse the actual root cause. 
> But the case is that when we run a PE tool with 50 threads under heavy load 
> when the writes are clogged I think we have some netty Bytebuf leak. Not sure 
> if it is a serious issue but we get this log
> {code}
> 2016-09-14 19:37:09,767 ERROR [Default-IPC-NioEventLoopGroup-1-16] 
> util.ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's 
> garbage-collected. Enable advanced leak reporting to find out where the leak 
> occurred. To enable advanced leak reporting, specify the JVM option 
> '-Dio.netty.leakDetection.level=advanced' or call 
> ResourceLeakDetector.setLevel() See 
> http://netty.io/wiki/reference-counted-objects.html for more information.
> {code}
> So reading the given link it is because of some ByteBuf that was not released 
> properly by the client and hence it gets GCed automatically. Netty provides 
> tips and tricks to find the root cause. Will get back here.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2016-09-19 Thread Hadoop QA (JIRA)

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

Hadoop QA commented on HBASE-16635:
---

| (x) *{color:red}-1 overall{color}* |
\\
\\
|| Vote || Subsystem || Runtime || Comment ||
| {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 56m 32s 
{color} | {color:blue} Docker mode activated. {color} |
| {color:green}+1{color} | {color:green} @author {color} | {color:green} 0m 0s 
{color} | {color:green} The patch does not contain any @author tags. {color} |
| {color:red}-1{color} | {color:red} test4tests {color} | {color:red} 0m 0s 
{color} | {color:red} The patch doesn't appear to include any new or modified 
tests. Please justify why no new tests are needed for this patch. Also please 
list what manual steps were performed to verify this patch. {color} |
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 11m 
7s {color} | {color:green} master passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 17s 
{color} | {color:green} master passed {color} |
| {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 0m 
43s {color} | {color:green} master passed {color} |
| {color:green}+1{color} | {color:green} mvneclipse {color} | {color:green} 0m 
43s {color} | {color:green} master passed {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green} 0m 
47s {color} | {color:green} master passed {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 29s 
{color} | {color:green} master passed {color} |
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 0m 
21s {color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 17s 
{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 17s 
{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 0m 
24s {color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} mvneclipse {color} | {color:green} 0m 
12s {color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} whitespace {color} | {color:green} 0m 
0s {color} | {color:green} The patch has no whitespace issues. {color} |
| {color:green}+1{color} | {color:green} hadoopcheck {color} | {color:green} 
34m 18s {color} | {color:green} Patch does not cause any errors with Hadoop 
2.4.0 2.4.1 2.5.0 2.5.1 2.5.2 2.6.1 2.6.2 2.6.3 2.7.1. {color} |
| {color:green}+1{color} | {color:green} hbaseprotoc {color} | {color:green} 0m 
14s {color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green} 1m 8s 
{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 18s 
{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} unit {color} | {color:green} 1m 9s 
{color} | {color:green} hbase-client in the patch passed. {color} |
| {color:green}+1{color} | {color:green} asflicense {color} | {color:green} 0m 
12s {color} | {color:green} The patch does not generate ASF License warnings. 
{color} |
| {color:black}{color} | {color:black} {color} | {color:black} 110m 3s {color} 
| {color:black} {color} |
\\
\\
|| Subsystem || Report/Notes ||
| Docker | Client=1.11.2 Server=1.11.2 Image:yetus/hbase:7bda515 |
| JIRA Patch URL | 
https://issues.apache.org/jira/secure/attachment/12829159/HBASE-16635_1.patch |
| JIRA Issue | HBASE-16635 |
| Optional Tests |  asflicense  javac  javadoc  unit  findbugs  hadoopcheck  
hbaseanti  checkstyle  compile  |
| uname | Linux c252777e9314 3.13.0-92-generic #139-Ubuntu SMP Tue Jun 28 
20:42:26 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux |
| Build tool | maven |
| Personality | 
/home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/component/dev-support/hbase-personality.sh
 |
| git revision | master / 6eb6225 |
| Default Java | 1.8.0_101 |
| findbugs | v3.0.0 |
|  Test Results | 
https://builds.apache.org/job/PreCommit-HBASE-Build/3599/testReport/ |
| modules | C: hbase-client U: hbase-client |
| Console output | 
https://builds.apache.org/job/PreCommit-HBASE-Build/3599/console |
| Powered by | Apache Yetus 0.3.0   http://yetus.apache.org |


This message was automatically generated.



> RpcClient under heavy load leaks some netty bytebuf
> ---
>
> Key: HBASE-16635
> URL: https://issues.apache.org/jira/browse/HBASE-16635
> Project: HBase
>  Issue Type: Bug
>Reporter: ramkrishna.s.vasudevan
>Assignee: ramkrishna.s.vasudevan
>Priority: Minor
>  

[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2016-09-19 Thread Duo Zhang (JIRA)

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

Duo Zhang commented on HBASE-16635:
---

+1.

There are still some bad practices but not your fault. I will file other issues 
to fix it.

Thanks.

> RpcClient under heavy load leaks some netty bytebuf
> ---
>
> Key: HBASE-16635
> URL: https://issues.apache.org/jira/browse/HBASE-16635
> Project: HBase
>  Issue Type: Bug
>Reporter: ramkrishna.s.vasudevan
>Assignee: ramkrishna.s.vasudevan
>Priority: Minor
> Fix For: 2.0.0
>
> Attachments: HBASE-16635.patch, HBASE-16635_1.patch
>
>
> Yet to analyse the actual root cause. 
> But the case is that when we run a PE tool with 50 threads under heavy load 
> when the writes are clogged I think we have some netty Bytebuf leak. Not sure 
> if it is a serious issue but we get this log
> {code}
> 2016-09-14 19:37:09,767 ERROR [Default-IPC-NioEventLoopGroup-1-16] 
> util.ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's 
> garbage-collected. Enable advanced leak reporting to find out where the leak 
> occurred. To enable advanced leak reporting, specify the JVM option 
> '-Dio.netty.leakDetection.level=advanced' or call 
> ResourceLeakDetector.setLevel() See 
> http://netty.io/wiki/reference-counted-objects.html for more information.
> {code}
> So reading the given link it is because of some ByteBuf that was not released 
> properly by the client and hence it gets GCed automatically. Netty provides 
> tips and tricks to find the root cause. Will get back here.



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


[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2016-09-19 Thread Duo Zhang (JIRA)

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

Duo Zhang commented on HBASE-16635:
---

The connect method will called when you call Bootstrap.connect. So I think the 
connect method of SaslWrapHandler will not be called because we only add it to 
the pipeline after we finish connect.

> RpcClient under heavy load leaks some netty bytebuf
> ---
>
> Key: HBASE-16635
> URL: https://issues.apache.org/jira/browse/HBASE-16635
> Project: HBase
>  Issue Type: Bug
>Reporter: ramkrishna.s.vasudevan
>Assignee: ramkrishna.s.vasudevan
>Priority: Minor
> Fix For: 2.0.0
>
> Attachments: HBASE-16635.patch
>
>
> Yet to analyse the actual root cause. 
> But the case is that when we run a PE tool with 50 threads under heavy load 
> when the writes are clogged I think we have some netty Bytebuf leak. Not sure 
> if it is a serious issue but we get this log
> {code}
> 2016-09-14 19:37:09,767 ERROR [Default-IPC-NioEventLoopGroup-1-16] 
> util.ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's 
> garbage-collected. Enable advanced leak reporting to find out where the leak 
> occurred. To enable advanced leak reporting, specify the JVM option 
> '-Dio.netty.leakDetection.level=advanced' or call 
> ResourceLeakDetector.setLevel() See 
> http://netty.io/wiki/reference-counted-objects.html for more information.
> {code}
> So reading the given link it is because of some ByteBuf that was not released 
> properly by the client and hence it gets GCed automatically. Netty provides 
> tips and tricks to find the root cause. Will get back here.



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


[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2016-09-19 Thread Hadoop QA (JIRA)

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

Hadoop QA commented on HBASE-16635:
---

| (x) *{color:red}-1 overall{color}* |
\\
\\
|| Vote || Subsystem || Runtime || Comment ||
| {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 0m 52s 
{color} | {color:blue} Docker mode activated. {color} |
| {color:green}+1{color} | {color:green} @author {color} | {color:green} 0m 0s 
{color} | {color:green} The patch does not contain any @author tags. {color} |
| {color:red}-1{color} | {color:red} test4tests {color} | {color:red} 0m 0s 
{color} | {color:red} The patch doesn't appear to include any new or modified 
tests. Please justify why no new tests are needed for this patch. Also please 
list what manual steps were performed to verify this patch. {color} |
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 3m 
41s {color} | {color:green} master passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 16s 
{color} | {color:green} master passed {color} |
| {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 0m 
23s {color} | {color:green} master passed {color} |
| {color:green}+1{color} | {color:green} mvneclipse {color} | {color:green} 0m 
10s {color} | {color:green} master passed {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green} 0m 
50s {color} | {color:green} master passed {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 16s 
{color} | {color:green} master passed {color} |
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 0m 
17s {color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 16s 
{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 16s 
{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 0m 
22s {color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} mvneclipse {color} | {color:green} 0m 
9s {color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} whitespace {color} | {color:green} 0m 
0s {color} | {color:green} The patch has no whitespace issues. {color} |
| {color:green}+1{color} | {color:green} hadoopcheck {color} | {color:green} 
28m 19s {color} | {color:green} Patch does not cause any errors with Hadoop 
2.4.0 2.4.1 2.5.0 2.5.1 2.5.2 2.6.1 2.6.2 2.6.3 2.7.1. {color} |
| {color:green}+1{color} | {color:green} hbaseprotoc {color} | {color:green} 0m 
10s {color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green} 0m 
58s {color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 16s 
{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} unit {color} | {color:green} 0m 58s 
{color} | {color:green} hbase-client in the patch passed. {color} |
| {color:green}+1{color} | {color:green} asflicense {color} | {color:green} 0m 
9s {color} | {color:green} The patch does not generate ASF License warnings. 
{color} |
| {color:black}{color} | {color:black} {color} | {color:black} 38m 43s {color} 
| {color:black} {color} |
\\
\\
|| Subsystem || Report/Notes ||
| Docker | Client=1.11.2 Server=1.11.2 Image:yetus/hbase:7bda515 |
| JIRA Patch URL | 
https://issues.apache.org/jira/secure/attachment/12829141/HBASE-16635.patch |
| JIRA Issue | HBASE-16635 |
| Optional Tests |  asflicense  javac  javadoc  unit  findbugs  hadoopcheck  
hbaseanti  checkstyle  compile  |
| uname | Linux fa8b179e7b2c 3.13.0-92-generic #139-Ubuntu SMP Tue Jun 28 
20:42:26 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux |
| Build tool | maven |
| Personality | 
/home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/component/dev-support/hbase-personality.sh
 |
| git revision | master / 6eb6225 |
| Default Java | 1.8.0_101 |
| findbugs | v3.0.0 |
|  Test Results | 
https://builds.apache.org/job/PreCommit-HBASE-Build/3595/testReport/ |
| modules | C: hbase-client U: hbase-client |
| Console output | 
https://builds.apache.org/job/PreCommit-HBASE-Build/3595/console |
| Powered by | Apache Yetus 0.3.0   http://yetus.apache.org |


This message was automatically generated.



> RpcClient under heavy load leaks some netty bytebuf
> ---
>
> Key: HBASE-16635
> URL: https://issues.apache.org/jira/browse/HBASE-16635
> Project: HBase
>  Issue Type: Bug
>Reporter: ramkrishna.s.vasudevan
>Assignee: ramkrishna.s.vasudevan
>Priority: Minor
> 

[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2016-09-19 Thread ramkrishna.s.vasudevan (JIRA)

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

ramkrishna.s.vasudevan commented on HBASE-16635:


Thanks for the comments. I was reading thro the interfaces in netty bytebuf to 
see what these APIs does.
One question on SaslWrapHandler
Where does the connect() happen in case of SaslWrapHandler? We pass on a 
SaslClient to it and we have write, handlerAdded and flush(). 
The context we have and we add the saslWrapHandler at the head of the pipeline. 
When will the connect() of ChannelOutboundHandlerAdapter be called? Am missing 
something but not sure on this framework usage.

> RpcClient under heavy load leaks some netty bytebuf
> ---
>
> Key: HBASE-16635
> URL: https://issues.apache.org/jira/browse/HBASE-16635
> Project: HBase
>  Issue Type: Bug
>Reporter: ramkrishna.s.vasudevan
>Assignee: ramkrishna.s.vasudevan
>Priority: Minor
> Fix For: 2.0.0
>
> Attachments: HBASE-16635.patch
>
>
> Yet to analyse the actual root cause. 
> But the case is that when we run a PE tool with 50 threads under heavy load 
> when the writes are clogged I think we have some netty Bytebuf leak. Not sure 
> if it is a serious issue but we get this log
> {code}
> 2016-09-14 19:37:09,767 ERROR [Default-IPC-NioEventLoopGroup-1-16] 
> util.ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's 
> garbage-collected. Enable advanced leak reporting to find out where the leak 
> occurred. To enable advanced leak reporting, specify the JVM option 
> '-Dio.netty.leakDetection.level=advanced' or call 
> ResourceLeakDetector.setLevel() See 
> http://netty.io/wiki/reference-counted-objects.html for more information.
> {code}
> So reading the given link it is because of some ByteBuf that was not released 
> properly by the client and hence it gets GCed automatically. Netty provides 
> tips and tricks to find the root cause. Will get back here.



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


[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2016-09-19 Thread Duo Zhang (JIRA)

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

Duo Zhang commented on HBASE-16635:
---

Ah we should override the close method, not disconnect, sorry...

> RpcClient under heavy load leaks some netty bytebuf
> ---
>
> Key: HBASE-16635
> URL: https://issues.apache.org/jira/browse/HBASE-16635
> Project: HBase
>  Issue Type: Bug
>Reporter: ramkrishna.s.vasudevan
>Assignee: ramkrishna.s.vasudevan
>Priority: Minor
> Fix For: 2.0.0
>
> Attachments: HBASE-16635.patch
>
>
> Yet to analyse the actual root cause. 
> But the case is that when we run a PE tool with 50 threads under heavy load 
> when the writes are clogged I think we have some netty Bytebuf leak. Not sure 
> if it is a serious issue but we get this log
> {code}
> 2016-09-14 19:37:09,767 ERROR [Default-IPC-NioEventLoopGroup-1-16] 
> util.ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's 
> garbage-collected. Enable advanced leak reporting to find out where the leak 
> occurred. To enable advanced leak reporting, specify the JVM option 
> '-Dio.netty.leakDetection.level=advanced' or call 
> ResourceLeakDetector.setLevel() See 
> http://netty.io/wiki/reference-counted-objects.html for more information.
> {code}
> So reading the given link it is because of some ByteBuf that was not released 
> properly by the client and hence it gets GCed automatically. Netty provides 
> tips and tricks to find the root cause. Will get back here.



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


[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2016-09-18 Thread Duo Zhang (JIRA)

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

Duo Zhang commented on HBASE-16635:
---

{quote}
so you mean all the ByteBuf in the queue has to removed in the disconnect and 
not this particular wrapped byteBuf?
{quote}

Two things

1. The ByteBuf created by calling queue.remove should be release manually in 
the method as we do not pass it to the subsequent handlers.
2. When channel disconnect, there maybe some ByteBufs left in the queue, so we 
need to release them all by calling queue.releaseAndFailAll.

Thanks.

> RpcClient under heavy load leaks some netty bytebuf
> ---
>
> Key: HBASE-16635
> URL: https://issues.apache.org/jira/browse/HBASE-16635
> Project: HBase
>  Issue Type: Bug
>Reporter: ramkrishna.s.vasudevan
>Assignee: ramkrishna.s.vasudevan
>Priority: Minor
> Fix For: 2.0.0
>
> Attachments: HBASE-16635.patch
>
>
> Yet to analyse the actual root cause. 
> But the case is that when we run a PE tool with 50 threads under heavy load 
> when the writes are clogged I think we have some netty Bytebuf leak. Not sure 
> if it is a serious issue but we get this log
> {code}
> 2016-09-14 19:37:09,767 ERROR [Default-IPC-NioEventLoopGroup-1-16] 
> util.ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's 
> garbage-collected. Enable advanced leak reporting to find out where the leak 
> occurred. To enable advanced leak reporting, specify the JVM option 
> '-Dio.netty.leakDetection.level=advanced' or call 
> ResourceLeakDetector.setLevel() See 
> http://netty.io/wiki/reference-counted-objects.html for more information.
> {code}
> So reading the given link it is because of some ByteBuf that was not released 
> properly by the client and hence it gets GCed automatically. Netty provides 
> tips and tricks to find the root cause. Will get back here.



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


[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2016-09-18 Thread ramkrishna.s.vasudevan (JIRA)

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

ramkrishna.s.vasudevan commented on HBASE-16635:


bq.The cleanupConnection does not need to throw Exception as the release method 
does not throw any exception
I agree but generally in API signature for methods like connectionCleanup we 
generally add throws clause. Hence did that. I can remove it.
bq.And we also need to override the disconnect method to release the ByteBuf 
buffered in queue.
Ok. so you mean all the ByteBuf in the queue has to removed in the disconnect 
and not this particular wrapped byteBuf?


> RpcClient under heavy load leaks some netty bytebuf
> ---
>
> Key: HBASE-16635
> URL: https://issues.apache.org/jira/browse/HBASE-16635
> Project: HBase
>  Issue Type: Bug
>Reporter: ramkrishna.s.vasudevan
>Assignee: ramkrishna.s.vasudevan
>Priority: Minor
> Fix For: 2.0.0
>
> Attachments: HBASE-16635.patch
>
>
> Yet to analyse the actual root cause. 
> But the case is that when we run a PE tool with 50 threads under heavy load 
> when the writes are clogged I think we have some netty Bytebuf leak. Not sure 
> if it is a serious issue but we get this log
> {code}
> 2016-09-14 19:37:09,767 ERROR [Default-IPC-NioEventLoopGroup-1-16] 
> util.ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's 
> garbage-collected. Enable advanced leak reporting to find out where the leak 
> occurred. To enable advanced leak reporting, specify the JVM option 
> '-Dio.netty.leakDetection.level=advanced' or call 
> ResourceLeakDetector.setLevel() See 
> http://netty.io/wiki/reference-counted-objects.html for more information.
> {code}
> So reading the given link it is because of some ByteBuf that was not released 
> properly by the client and hence it gets GCed automatically. Netty provides 
> tips and tricks to find the root cause. Will get back here.



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


[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2016-09-18 Thread Duo Zhang (JIRA)

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

Duo Zhang commented on HBASE-16635:
---

The cleanupConnection does not need to throw Exception as the release method 
does not throw any exception. And you can use ReferenceCountUtil.safeRelease if 
you want to make sure  there is no exception thrown.

And for SaslWrapHandler, we need to release the ByteBuf create by calling 
queue.remove, not the wrapped ByteBuf. The wrapped ByteBuf will be released 
when actually writing out. And we also need to override the disconnect method 
to release the ByteBuf buffered in queue.

Thanks.

> RpcClient under heavy load leaks some netty bytebuf
> ---
>
> Key: HBASE-16635
> URL: https://issues.apache.org/jira/browse/HBASE-16635
> Project: HBase
>  Issue Type: Bug
>Reporter: ramkrishna.s.vasudevan
>Assignee: ramkrishna.s.vasudevan
>Priority: Minor
> Fix For: 2.0.0
>
> Attachments: HBASE-16635.patch
>
>
> Yet to analyse the actual root cause. 
> But the case is that when we run a PE tool with 50 threads under heavy load 
> when the writes are clogged I think we have some netty Bytebuf leak. Not sure 
> if it is a serious issue but we get this log
> {code}
> 2016-09-14 19:37:09,767 ERROR [Default-IPC-NioEventLoopGroup-1-16] 
> util.ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's 
> garbage-collected. Enable advanced leak reporting to find out where the leak 
> occurred. To enable advanced leak reporting, specify the JVM option 
> '-Dio.netty.leakDetection.level=advanced' or call 
> ResourceLeakDetector.setLevel() See 
> http://netty.io/wiki/reference-counted-objects.html for more information.
> {code}
> So reading the given link it is because of some ByteBuf that was not released 
> properly by the client and hence it gets GCed automatically. Netty provides 
> tips and tricks to find the root cause. Will get back here.



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


[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2016-09-18 Thread ramkrishna.s.vasudevan (JIRA)

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

ramkrishna.s.vasudevan commented on HBASE-16635:


Tested the patch. I could not get that netty leak log in the client side after 
this. 

> RpcClient under heavy load leaks some netty bytebuf
> ---
>
> Key: HBASE-16635
> URL: https://issues.apache.org/jira/browse/HBASE-16635
> Project: HBase
>  Issue Type: Bug
>Reporter: ramkrishna.s.vasudevan
>Assignee: ramkrishna.s.vasudevan
>Priority: Minor
> Fix For: 2.0.0
>
> Attachments: HBASE-16635.patch
>
>
> Yet to analyse the actual root cause. 
> But the case is that when we run a PE tool with 50 threads under heavy load 
> when the writes are clogged I think we have some netty Bytebuf leak. Not sure 
> if it is a serious issue but we get this log
> {code}
> 2016-09-14 19:37:09,767 ERROR [Default-IPC-NioEventLoopGroup-1-16] 
> util.ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's 
> garbage-collected. Enable advanced leak reporting to find out where the leak 
> occurred. To enable advanced leak reporting, specify the JVM option 
> '-Dio.netty.leakDetection.level=advanced' or call 
> ResourceLeakDetector.setLevel() See 
> http://netty.io/wiki/reference-counted-objects.html for more information.
> {code}
> So reading the given link it is because of some ByteBuf that was not released 
> properly by the client and hence it gets GCed automatically. Netty provides 
> tips and tricks to find the root cause. Will get back here.



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


[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2016-09-16 Thread ramkrishna.s.vasudevan (JIRA)

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

ramkrishna.s.vasudevan commented on HBASE-16635:


Thanks [~Apache9]
Yes in the above condition it is the admin connection that is actually closed() 
but the Bytebuf created for that connection is not released. And reading the 
code, you are right we cannot release the Bytebuf in shutdown() because there 
the connection is put back to the pool for reusing it. 

> RpcClient under heavy load leaks some netty bytebuf
> ---
>
> Key: HBASE-16635
> URL: https://issues.apache.org/jira/browse/HBASE-16635
> Project: HBase
>  Issue Type: Bug
>Reporter: ramkrishna.s.vasudevan
>Assignee: ramkrishna.s.vasudevan
>Priority: Minor
> Fix For: 2.0.0
>
>
> Yet to analyse the actual root cause. 
> But the case is that when we run a PE tool with 50 threads under heavy load 
> when the writes are clogged I think we have some netty Bytebuf leak. Not sure 
> if it is a serious issue but we get this log
> {code}
> 2016-09-14 19:37:09,767 ERROR [Default-IPC-NioEventLoopGroup-1-16] 
> util.ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's 
> garbage-collected. Enable advanced leak reporting to find out where the leak 
> occurred. To enable advanced leak reporting, specify the JVM option 
> '-Dio.netty.leakDetection.level=advanced' or call 
> ResourceLeakDetector.setLevel() See 
> http://netty.io/wiki/reference-counted-objects.html for more information.
> {code}
> So reading the given link it is because of some ByteBuf that was not released 
> properly by the client and hence it gets GCed automatically. Netty provides 
> tips and tricks to find the root cause. Will get back here.



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


[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2016-09-15 Thread Duo Zhang (JIRA)

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

Duo Zhang commented on HBASE-16635:
---

Go for it!

And some background, we can not release the ByteBufs in shutdown method because 
we may reconnect later. We can only release them when the connection is removed 
from the connection pool. So we need to add a method to the RpcConnection class 
to do this.

Thanks.

> RpcClient under heavy load leaks some netty bytebuf
> ---
>
> Key: HBASE-16635
> URL: https://issues.apache.org/jira/browse/HBASE-16635
> Project: HBase
>  Issue Type: Bug
>Reporter: ramkrishna.s.vasudevan
>Assignee: ramkrishna.s.vasudevan
>Priority: Minor
> Fix For: 2.0.0
>
>
> Yet to analyse the actual root cause. 
> But the case is that when we run a PE tool with 50 threads under heavy load 
> when the writes are clogged I think we have some netty Bytebuf leak. Not sure 
> if it is a serious issue but we get this log
> {code}
> 2016-09-14 19:37:09,767 ERROR [Default-IPC-NioEventLoopGroup-1-16] 
> util.ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's 
> garbage-collected. Enable advanced leak reporting to find out where the leak 
> occurred. To enable advanced leak reporting, specify the JVM option 
> '-Dio.netty.leakDetection.level=advanced' or call 
> ResourceLeakDetector.setLevel() See 
> http://netty.io/wiki/reference-counted-objects.html for more information.
> {code}
> So reading the given link it is because of some ByteBuf that was not released 
> properly by the client and hence it gets GCed automatically. Netty provides 
> tips and tricks to find the root cause. Will get back here.



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


[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2016-09-15 Thread ramkrishna.s.vasudevan (JIRA)

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

ramkrishna.s.vasudevan commented on HBASE-16635:


Thanks [~Apache9]
I think your guess i right. The two connection ByteBufs aren't released I 
believe.
{code}
2016-09-15 22:05:12,617 ERROR [Default-IPC-NioEventLoopGroup-1-26] 
util.ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's 
garbage-collected. See http://netty.io/wiki/reference-counted-objects.html for 
more information.
WARNING: 2 leak records were discarded because the leak record count is limited 
to 4. Use system property io.netty.leakDetection.maxRecords to increase the 
limit.
Recent access records: 5
#5:

io.netty.buffer.AdvancedLeakAwareByteBuf.release(AdvancedLeakAwareByteBuf.java:955)
io.netty.util.ReferenceCountUtil.release(ReferenceCountUtil.java:84)

io.netty.util.ReferenceCountUtil.safeRelease(ReferenceCountUtil.java:109)

io.netty.channel.ChannelOutboundBuffer.remove(ChannelOutboundBuffer.java:265)

io.netty.channel.ChannelOutboundBuffer.removeBytes(ChannelOutboundBuffer.java:346)

io.netty.channel.socket.nio.NioSocketChannel.doWrite(NioSocketChannel.java:426)

io.netty.channel.AbstractChannel$AbstractUnsafe.flush0(AbstractChannel.java:844)

io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.flush0(AbstractNioChannel.java:360)

io.netty.channel.AbstractChannel$AbstractUnsafe.flush(AbstractChannel.java:811)

io.netty.channel.DefaultChannelPipeline$HeadContext.flush(DefaultChannelPipeline.java:1276)

io.netty.channel.AbstractChannelHandlerContext.invokeFlush0(AbstractChannelHandlerContext.java:762)

io.netty.channel.AbstractChannelHandlerContext.invokeFlush(AbstractChannelHandlerContext.java:754)

io.netty.channel.AbstractChannelHandlerContext.flush(AbstractChannelHandlerContext.java:735)

io.netty.channel.ChannelDuplexHandler.flush(ChannelDuplexHandler.java:117)

io.netty.channel.AbstractChannelHandlerContext.invokeFlush0(AbstractChannelHandlerContext.java:762)

io.netty.channel.AbstractChannelHandlerContext.invokeWriteAndFlush(AbstractChannelHandlerContext.java:788)

io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:800)

io.netty.channel.AbstractChannelHandlerContext.writeAndFlush(AbstractChannelHandlerContext.java:780)

io.netty.channel.AbstractChannelHandlerContext.writeAndFlush(AbstractChannelHandlerContext.java:817)

io.netty.channel.DefaultChannelPipeline.writeAndFlush(DefaultChannelPipeline.java:1011)
io.netty.channel.AbstractChannel.writeAndFlush(AbstractChannel.java:289)

org.apache.hadoop.hbase.ipc.NettyRpcConnection$3.operationComplete(NettyRpcConnection.java:228)

org.apache.hadoop.hbase.ipc.NettyRpcConnection$3.operationComplete(NettyRpcConnection.java:218)

io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:518)

io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:511)

io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:490)

io.netty.util.concurrent.DefaultPromise.notifyListenersWithStackOverFlowProtection(DefaultPromise.java:431)

io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:420)

io.netty.util.concurrent.DefaultPromise.trySuccess(DefaultPromise.java:108)

io.netty.channel.DefaultChannelPromise.trySuccess(DefaultChannelPromise.java:82)

io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.fulfillConnectPromise(AbstractNioChannel.java:304)

io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.finishConnect(AbstractNioChannel.java:339)

io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:580)

io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:504)

io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:418)
io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:390)

io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:742)

io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:145)
java.lang.Thread.run(Thread.java:745)
#4:

io.netty.buffer.AdvancedLeakAwareByteBuf.internalNioBuffer(AdvancedLeakAwareByteBuf.java:735)

io.netty.channel.ChannelOutboundBuffer.nioBuffers(ChannelOutboundBuffer.java:420)

io.netty.channel.socket.nio.NioSocketChannel.doWrite(NioSocketChannel.java:379)

io.netty.channel.AbstractChannel$AbstractUnsafe.flush0(AbstractChannel.java:844)


[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2016-09-15 Thread Duo Zhang (JIRA)

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

Duo Zhang commented on HBASE-16635:
---

Ah after a careful review, I found some leaks...

First is in SaslWrapHandler, I forgot to release the ByteBuf after read all its 
content to a byte array.

Second, the two connection header ByteBufs in NettyRpcConnection. It is 
allocated by Unpooled so I ignored the release intentionally when writing down 
the code, maybe. But after reading the code of netty, I believe the ByteBufs 
allocated by Unpooled should also be released...

I do not know why I have an illusion that an unpooled ByteBuf does not need to 
be released. I had hit a 'buffer already released' problem when I forgot to use 
retainedDuplicate before writing out the header. This definitely means netty 
will release unpooled ByteBuf...

Please tell me if you have other findings. Thanks. My fault.

> RpcClient under heavy load leaks some netty bytebuf
> ---
>
> Key: HBASE-16635
> URL: https://issues.apache.org/jira/browse/HBASE-16635
> Project: HBase
>  Issue Type: Bug
>Reporter: ramkrishna.s.vasudevan
>Assignee: ramkrishna.s.vasudevan
>Priority: Minor
> Fix For: 2.0.0
>
>
> Yet to analyse the actual root cause. 
> But the case is that when we run a PE tool with 50 threads under heavy load 
> when the writes are clogged I think we have some netty Bytebuf leak. Not sure 
> if it is a serious issue but we get this log
> {code}
> 2016-09-14 19:37:09,767 ERROR [Default-IPC-NioEventLoopGroup-1-16] 
> util.ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's 
> garbage-collected. Enable advanced leak reporting to find out where the leak 
> occurred. To enable advanced leak reporting, specify the JVM option 
> '-Dio.netty.leakDetection.level=advanced' or call 
> ResourceLeakDetector.setLevel() See 
> http://netty.io/wiki/reference-counted-objects.html for more information.
> {code}
> So reading the given link it is because of some ByteBuf that was not released 
> properly by the client and hence it gets GCed automatically. Netty provides 
> tips and tricks to find the root cause. Will get back here.



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


[jira] [Commented] (HBASE-16635) RpcClient under heavy load leaks some netty bytebuf

2016-09-14 Thread Duo Zhang (JIRA)

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

Duo Zhang commented on HBASE-16635:
---

Can you add the -Dio.netty.leakDetection.level=advanced options and run again?

This will output some stacktrace infos which could help us find out where the 
ByteBufs are touched.

Thanks.

> RpcClient under heavy load leaks some netty bytebuf
> ---
>
> Key: HBASE-16635
> URL: https://issues.apache.org/jira/browse/HBASE-16635
> Project: HBase
>  Issue Type: Bug
>Reporter: ramkrishna.s.vasudevan
>Assignee: ramkrishna.s.vasudevan
>Priority: Minor
> Fix For: 2.0.0
>
>
> Yet to analyse the actual root cause. 
> But the case is that when we run a PE tool with 50 threads under heavy load 
> when the writes are clogged I think we have some netty Bytebuf leak. Not sure 
> if it is a serious issue but we get this log
> {code}
> 2016-09-14 19:37:09,767 ERROR [Default-IPC-NioEventLoopGroup-1-16] 
> util.ResourceLeakDetector: LEAK: ByteBuf.release() was not called before it's 
> garbage-collected. Enable advanced leak reporting to find out where the leak 
> occurred. To enable advanced leak reporting, specify the JVM option 
> '-Dio.netty.leakDetection.level=advanced' or call 
> ResourceLeakDetector.setLevel() See 
> http://netty.io/wiki/reference-counted-objects.html for more information.
> {code}
> So reading the given link it is because of some ByteBuf that was not released 
> properly by the client and hence it gets GCed automatically. Netty provides 
> tips and tricks to find the root cause. Will get back here.



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