[jira] [Updated] (HBASE-16698) Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload

2017-01-22 Thread Yu Li (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-16698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Yu Li updated HBASE-16698:
--
  Resolution: Fixed
Release Note: 
Assign sequenceid to an edit before we go on the ringbuffer; undoes contention 
on WALKey latch. Adds a new config "hbase.hregion.mvcc.preassign" which 
defaults to true: i.e. this speedup is enabled.

User could set this per-table level, like:
create 
'table',{NAME=>'f1',CONFIGURATION=>{'hbase.hregion.mvcc.preassign'=>'false'}}

  was:Assign sequenceid to an edit before we go on the ringbuffer; undoes 
contention on WALKey latch. Adds a new config "hbase.hregion.mvcc.preassign" 
which defaults to true: i.e. this speedup is enabled.

  Status: Resolved  (was: Patch Available)

Closing this JIRA per above comments.

> Performance issue: handlers stuck waiting for CountDownLatch inside 
> WALKey#getWriteEntry under high writing workload
> 
>
> Key: HBASE-16698
> URL: https://issues.apache.org/jira/browse/HBASE-16698
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.2.3
>Reporter: Yu Li
>Assignee: Yu Li
> Fix For: 2.0.0, 1.4.0
>
> Attachments: hadoop0495.et2.jstack, HBASE-16698.branch-1.patch, 
> HBASE-16698.branch-1.v2.patch, HBASE-16698.branch-1.v2.patch, 
> HBASE-16698.patch, HBASE-16698.v2.patch
>
>
> As titled, on our production environment we observed 98 out of 128 handlers 
> get stuck waiting for the CountDownLatch {{seqNumAssignedLatch}} inside 
> {{WALKey#getWriteEntry}} under a high writing workload.
> After digging into the problem, we found that the problem is mainly caused by 
> advancing mvcc in the append logic. Below is some detailed analysis:
> Under current branch-1 code logic, all batch puts will call 
> {{WALKey#getWriteEntry}} after appending edit to WAL, and 
> {{seqNumAssignedLatch}} is only released when the relative append call is 
> handled by RingBufferEventHandler (see {{FSWALEntry#stampRegionSequenceId}}). 
> Because currently we're using a single event handler for the ringbuffer, the 
> append calls are handled one by one (actually lot's of our current logic 
> depending on this sequential dealing logic), and this becomes a bottleneck 
> under high writing workload.
> The worst part is that by default we only use one WAL per RS, so appends on 
> all regions are dealt with in sequential, which causes contention among 
> different regions...
> To fix this, we could also take use of the "sequential appends" mechanism, 
> that we could grab the WriteEntry before publishing append onto ringbuffer 
> and use it as sequence id, only that we need to add a lock to make "grab 
> WriteEntry" and "append edit" a transaction. This will still cause contention 
> inside a region but could avoid contention between different regions. This 
> solution is already verified in our online environment and proved to be 
> effective.
> Notice that for master (2.0) branch since we already change the write 
> pipeline to sync before writing memstore (HBASE-15158), this issue only 
> exists for the ASYNC_WAL writes scenario.



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


[jira] [Updated] (HBASE-16698) Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload

2016-11-01 Thread Yu Li (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-16698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Yu Li updated HBASE-16698:
--
Fix Version/s: 1.4.0

> Performance issue: handlers stuck waiting for CountDownLatch inside 
> WALKey#getWriteEntry under high writing workload
> 
>
> Key: HBASE-16698
> URL: https://issues.apache.org/jira/browse/HBASE-16698
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.2.3
>Reporter: Yu Li
>Assignee: Yu Li
> Fix For: 2.0.0, 1.4.0
>
> Attachments: HBASE-16698.branch-1.patch, 
> HBASE-16698.branch-1.v2.patch, HBASE-16698.branch-1.v2.patch, 
> HBASE-16698.patch, HBASE-16698.v2.patch, hadoop0495.et2.jstack
>
>
> As titled, on our production environment we observed 98 out of 128 handlers 
> get stuck waiting for the CountDownLatch {{seqNumAssignedLatch}} inside 
> {{WALKey#getWriteEntry}} under a high writing workload.
> After digging into the problem, we found that the problem is mainly caused by 
> advancing mvcc in the append logic. Below is some detailed analysis:
> Under current branch-1 code logic, all batch puts will call 
> {{WALKey#getWriteEntry}} after appending edit to WAL, and 
> {{seqNumAssignedLatch}} is only released when the relative append call is 
> handled by RingBufferEventHandler (see {{FSWALEntry#stampRegionSequenceId}}). 
> Because currently we're using a single event handler for the ringbuffer, the 
> append calls are handled one by one (actually lot's of our current logic 
> depending on this sequential dealing logic), and this becomes a bottleneck 
> under high writing workload.
> The worst part is that by default we only use one WAL per RS, so appends on 
> all regions are dealt with in sequential, which causes contention among 
> different regions...
> To fix this, we could also take use of the "sequential appends" mechanism, 
> that we could grab the WriteEntry before publishing append onto ringbuffer 
> and use it as sequence id, only that we need to add a lock to make "grab 
> WriteEntry" and "append edit" a transaction. This will still cause contention 
> inside a region but could avoid contention between different regions. This 
> solution is already verified in our online environment and proved to be 
> effective.
> Notice that for master (2.0) branch since we already change the write 
> pipeline to sync before writing memstore (HBASE-15158), this issue only 
> exists for the ASYNC_WAL writes scenario.



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


[jira] [Updated] (HBASE-16698) Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload

2016-10-18 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-16698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-16698:
--
Attachment: HBASE-16698.branch-1.v2.patch

Why the FindBugs warning? There is a finally to undo the reentrant lock. Retry 
the patch.

> Performance issue: handlers stuck waiting for CountDownLatch inside 
> WALKey#getWriteEntry under high writing workload
> 
>
> Key: HBASE-16698
> URL: https://issues.apache.org/jira/browse/HBASE-16698
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.2.3
>Reporter: Yu Li
>Assignee: Yu Li
> Fix For: 2.0.0
>
> Attachments: HBASE-16698.branch-1.patch, 
> HBASE-16698.branch-1.v2.patch, HBASE-16698.branch-1.v2.patch, 
> HBASE-16698.patch, HBASE-16698.v2.patch, hadoop0495.et2.jstack
>
>
> As titled, on our production environment we observed 98 out of 128 handlers 
> get stuck waiting for the CountDownLatch {{seqNumAssignedLatch}} inside 
> {{WALKey#getWriteEntry}} under a high writing workload.
> After digging into the problem, we found that the problem is mainly caused by 
> advancing mvcc in the append logic. Below is some detailed analysis:
> Under current branch-1 code logic, all batch puts will call 
> {{WALKey#getWriteEntry}} after appending edit to WAL, and 
> {{seqNumAssignedLatch}} is only released when the relative append call is 
> handled by RingBufferEventHandler (see {{FSWALEntry#stampRegionSequenceId}}). 
> Because currently we're using a single event handler for the ringbuffer, the 
> append calls are handled one by one (actually lot's of our current logic 
> depending on this sequential dealing logic), and this becomes a bottleneck 
> under high writing workload.
> The worst part is that by default we only use one WAL per RS, so appends on 
> all regions are dealt with in sequential, which causes contention among 
> different regions...
> To fix this, we could also take use of the "sequential appends" mechanism, 
> that we could grab the WriteEntry before publishing append onto ringbuffer 
> and use it as sequence id, only that we need to add a lock to make "grab 
> WriteEntry" and "append edit" a transaction. This will still cause contention 
> inside a region but could avoid contention between different regions. This 
> solution is already verified in our online environment and proved to be 
> effective.
> Notice that for master (2.0) branch since we already change the write 
> pipeline to sync before writing memstore (HBASE-15158), this issue only 
> exists for the ASYNC_WAL writes scenario.



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


[jira] [Updated] (HBASE-16698) Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload

2016-10-16 Thread Yu Li (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-16698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Yu Li updated HBASE-16698:
--
Status: Patch Available  (was: Reopened)

Submit patch so HadoopQA could check the new branch-1 patch

> Performance issue: handlers stuck waiting for CountDownLatch inside 
> WALKey#getWriteEntry under high writing workload
> 
>
> Key: HBASE-16698
> URL: https://issues.apache.org/jira/browse/HBASE-16698
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.2.3
>Reporter: Yu Li
>Assignee: Yu Li
> Fix For: 2.0.0
>
> Attachments: HBASE-16698.branch-1.patch, 
> HBASE-16698.branch-1.v2.patch, HBASE-16698.patch, HBASE-16698.v2.patch, 
> hadoop0495.et2.jstack
>
>
> As titled, on our production environment we observed 98 out of 128 handlers 
> get stuck waiting for the CountDownLatch {{seqNumAssignedLatch}} inside 
> {{WALKey#getWriteEntry}} under a high writing workload.
> After digging into the problem, we found that the problem is mainly caused by 
> advancing mvcc in the append logic. Below is some detailed analysis:
> Under current branch-1 code logic, all batch puts will call 
> {{WALKey#getWriteEntry}} after appending edit to WAL, and 
> {{seqNumAssignedLatch}} is only released when the relative append call is 
> handled by RingBufferEventHandler (see {{FSWALEntry#stampRegionSequenceId}}). 
> Because currently we're using a single event handler for the ringbuffer, the 
> append calls are handled one by one (actually lot's of our current logic 
> depending on this sequential dealing logic), and this becomes a bottleneck 
> under high writing workload.
> The worst part is that by default we only use one WAL per RS, so appends on 
> all regions are dealt with in sequential, which causes contention among 
> different regions...
> To fix this, we could also take use of the "sequential appends" mechanism, 
> that we could grab the WriteEntry before publishing append onto ringbuffer 
> and use it as sequence id, only that we need to add a lock to make "grab 
> WriteEntry" and "append edit" a transaction. This will still cause contention 
> inside a region but could avoid contention between different regions. This 
> solution is already verified in our online environment and proved to be 
> effective.
> Notice that for master (2.0) branch since we already change the write 
> pipeline to sync before writing memstore (HBASE-15158), this issue only 
> exists for the ASYNC_WAL writes scenario.



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


[jira] [Updated] (HBASE-16698) Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload

2016-10-14 Thread Yu Li (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-16698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Yu Li updated HBASE-16698:
--
Attachment: HBASE-16698.branch-1.v2.patch

Update patch for branch-1 after code rebase. Also make it able to work together 
with HBASE-16768 (we should also call {{HRegion#updateSequenceId}} when mvcc is 
preassigned after HBASE-16768, please refer to patch for the reason. I found 
this problem because {{TestHRegion#testReverseScanner_StackOverflow}} failed 
w/o the new changes)

[~stack] could you help review the new branch-1 patch? Many thanks.

> Performance issue: handlers stuck waiting for CountDownLatch inside 
> WALKey#getWriteEntry under high writing workload
> 
>
> Key: HBASE-16698
> URL: https://issues.apache.org/jira/browse/HBASE-16698
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.2.3
>Reporter: Yu Li
>Assignee: Yu Li
> Fix For: 2.0.0
>
> Attachments: HBASE-16698.branch-1.patch, 
> HBASE-16698.branch-1.v2.patch, HBASE-16698.patch, HBASE-16698.v2.patch, 
> hadoop0495.et2.jstack
>
>
> As titled, on our production environment we observed 98 out of 128 handlers 
> get stuck waiting for the CountDownLatch {{seqNumAssignedLatch}} inside 
> {{WALKey#getWriteEntry}} under a high writing workload.
> After digging into the problem, we found that the problem is mainly caused by 
> advancing mvcc in the append logic. Below is some detailed analysis:
> Under current branch-1 code logic, all batch puts will call 
> {{WALKey#getWriteEntry}} after appending edit to WAL, and 
> {{seqNumAssignedLatch}} is only released when the relative append call is 
> handled by RingBufferEventHandler (see {{FSWALEntry#stampRegionSequenceId}}). 
> Because currently we're using a single event handler for the ringbuffer, the 
> append calls are handled one by one (actually lot's of our current logic 
> depending on this sequential dealing logic), and this becomes a bottleneck 
> under high writing workload.
> The worst part is that by default we only use one WAL per RS, so appends on 
> all regions are dealt with in sequential, which causes contention among 
> different regions...
> To fix this, we could also take use of the "sequential appends" mechanism, 
> that we could grab the WriteEntry before publishing append onto ringbuffer 
> and use it as sequence id, only that we need to add a lock to make "grab 
> WriteEntry" and "append edit" a transaction. This will still cause contention 
> inside a region but could avoid contention between different regions. This 
> solution is already verified in our online environment and proved to be 
> effective.
> Notice that for master (2.0) branch since we already change the write 
> pipeline to sync before writing memstore (HBASE-15158), this issue only 
> exists for the ASYNC_WAL writes scenario.



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


[jira] [Updated] (HBASE-16698) Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload

2016-10-14 Thread Yu Li (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-16698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Yu Li updated HBASE-16698:
--
Affects Version/s: (was: 1.1.6)

Ok, let me explain.

One of the main difference between our customized 1.1.2 and current branch-1.1 
is that I've backported HBASE-14465 (Allow rowlock to be read/write in 
branch-1) to improve the non-conditional put performance. HBASE-14465 only goes 
in branch-1.2+, and this is what cause the confusion. Please check the latest 
branch-1 code and you'll find out what I'm trying to fix in this JIRA 
[~allan163]

Removing 1.1.6 from "Affects Versions" to avoid the confusion, and thanks for 
pointing this out [~allan163].

> Performance issue: handlers stuck waiting for CountDownLatch inside 
> WALKey#getWriteEntry under high writing workload
> 
>
> Key: HBASE-16698
> URL: https://issues.apache.org/jira/browse/HBASE-16698
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.2.3
>Reporter: Yu Li
>Assignee: Yu Li
> Fix For: 2.0.0
>
> Attachments: HBASE-16698.branch-1.patch, HBASE-16698.patch, 
> HBASE-16698.v2.patch, hadoop0495.et2.jstack
>
>
> As titled, on our production environment we observed 98 out of 128 handlers 
> get stuck waiting for the CountDownLatch {{seqNumAssignedLatch}} inside 
> {{WALKey#getWriteEntry}} under a high writing workload.
> After digging into the problem, we found that the problem is mainly caused by 
> advancing mvcc in the append logic. Below is some detailed analysis:
> Under current branch-1 code logic, all batch puts will call 
> {{WALKey#getWriteEntry}} after appending edit to WAL, and 
> {{seqNumAssignedLatch}} is only released when the relative append call is 
> handled by RingBufferEventHandler (see {{FSWALEntry#stampRegionSequenceId}}). 
> Because currently we're using a single event handler for the ringbuffer, the 
> append calls are handled one by one (actually lot's of our current logic 
> depending on this sequential dealing logic), and this becomes a bottleneck 
> under high writing workload.
> The worst part is that by default we only use one WAL per RS, so appends on 
> all regions are dealt with in sequential, which causes contention among 
> different regions...
> To fix this, we could also take use of the "sequential appends" mechanism, 
> that we could grab the WriteEntry before publishing append onto ringbuffer 
> and use it as sequence id, only that we need to add a lock to make "grab 
> WriteEntry" and "append edit" a transaction. This will still cause contention 
> inside a region but could avoid contention between different regions. This 
> solution is already verified in our online environment and proved to be 
> effective.
> Notice that for master (2.0) branch since we already change the write 
> pipeline to sync before writing memstore (HBASE-15158), this issue only 
> exists for the ASYNC_WAL writes scenario.



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


[jira] [Updated] (HBASE-16698) Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload

2016-10-13 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-16698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-16698:
--
   Resolution: Fixed
 Hadoop Flags: Reviewed
Fix Version/s: (was: 1.4.0)
   (was: 1.3.0)
 Release Note: Assign sequenceid to an edit before we go on the ringbuffer; 
undoes contention on WALKey latch. Adds a new config 
"hbase.hregion.mvcc.preassign" which defaults to true: i.e. this speedup is 
enabled.
   Status: Resolved  (was: Patch Available)

Pushed ot master with this speedup enabled by default. The numbers posted look 
good and [~carp84] is running it in production these last two months. I did not 
backport to branch-1. It does not apply cleanly. Thanks for the clever patch 
[~carp84]

> Performance issue: handlers stuck waiting for CountDownLatch inside 
> WALKey#getWriteEntry under high writing workload
> 
>
> Key: HBASE-16698
> URL: https://issues.apache.org/jira/browse/HBASE-16698
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.1.6, 1.2.3
>Reporter: Yu Li
>Assignee: Yu Li
> Fix For: 2.0.0
>
> Attachments: HBASE-16698.branch-1.patch, HBASE-16698.patch, 
> HBASE-16698.v2.patch, hadoop0495.et2.jstack
>
>
> As titled, on our production environment we observed 98 out of 128 handlers 
> get stuck waiting for the CountDownLatch {{seqNumAssignedLatch}} inside 
> {{WALKey#getWriteEntry}} under a high writing workload.
> After digging into the problem, we found that the problem is mainly caused by 
> advancing mvcc in the append logic. Below is some detailed analysis:
> Under current branch-1 code logic, all batch puts will call 
> {{WALKey#getWriteEntry}} after appending edit to WAL, and 
> {{seqNumAssignedLatch}} is only released when the relative append call is 
> handled by RingBufferEventHandler (see {{FSWALEntry#stampRegionSequenceId}}). 
> Because currently we're using a single event handler for the ringbuffer, the 
> append calls are handled one by one (actually lot's of our current logic 
> depending on this sequential dealing logic), and this becomes a bottleneck 
> under high writing workload.
> The worst part is that by default we only use one WAL per RS, so appends on 
> all regions are dealt with in sequential, which causes contention among 
> different regions...
> To fix this, we could also take use of the "sequential appends" mechanism, 
> that we could grab the WriteEntry before publishing append onto ringbuffer 
> and use it as sequence id, only that we need to add a lock to make "grab 
> WriteEntry" and "append edit" a transaction. This will still cause contention 
> inside a region but could avoid contention between different regions. This 
> solution is already verified in our online environment and proved to be 
> effective.
> Notice that for master (2.0) branch since we already change the write 
> pipeline to sync before writing memstore (HBASE-15158), this issue only 
> exists for the ASYNC_WAL writes scenario.



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


[jira] [Updated] (HBASE-16698) Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload

2016-10-12 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-16698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-16698:
--
Fix Version/s: 1.4.0
   1.3.0
   2.0.0

> Performance issue: handlers stuck waiting for CountDownLatch inside 
> WALKey#getWriteEntry under high writing workload
> 
>
> Key: HBASE-16698
> URL: https://issues.apache.org/jira/browse/HBASE-16698
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.1.6, 1.2.3
>Reporter: Yu Li
>Assignee: Yu Li
> Fix For: 2.0.0, 1.3.0, 1.4.0
>
> Attachments: HBASE-16698.branch-1.patch, HBASE-16698.patch, 
> HBASE-16698.v2.patch, hadoop0495.et2.jstack
>
>
> As titled, on our production environment we observed 98 out of 128 handlers 
> get stuck waiting for the CountDownLatch {{seqNumAssignedLatch}} inside 
> {{WALKey#getWriteEntry}} under a high writing workload.
> After digging into the problem, we found that the problem is mainly caused by 
> advancing mvcc in the append logic. Below is some detailed analysis:
> Under current branch-1 code logic, all batch puts will call 
> {{WALKey#getWriteEntry}} after appending edit to WAL, and 
> {{seqNumAssignedLatch}} is only released when the relative append call is 
> handled by RingBufferEventHandler (see {{FSWALEntry#stampRegionSequenceId}}). 
> Because currently we're using a single event handler for the ringbuffer, the 
> append calls are handled one by one (actually lot's of our current logic 
> depending on this sequential dealing logic), and this becomes a bottleneck 
> under high writing workload.
> The worst part is that by default we only use one WAL per RS, so appends on 
> all regions are dealt with in sequential, which causes contention among 
> different regions...
> To fix this, we could also take use of the "sequential appends" mechanism, 
> that we could grab the WriteEntry before publishing append onto ringbuffer 
> and use it as sequence id, only that we need to add a lock to make "grab 
> WriteEntry" and "append edit" a transaction. This will still cause contention 
> inside a region but could avoid contention between different regions. This 
> solution is already verified in our online environment and proved to be 
> effective.
> Notice that for master (2.0) branch since we already change the write 
> pipeline to sync before writing memstore (HBASE-15158), this issue only 
> exists for the ASYNC_WAL writes scenario.



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


[jira] [Updated] (HBASE-16698) Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload

2016-10-02 Thread Yu Li (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-16698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Yu Li updated HBASE-16698:
--
Attachment: HBASE-16698.branch-1.patch

Uploading patch for branch-1, will upload perf number later. Sorry for the lag.

> Performance issue: handlers stuck waiting for CountDownLatch inside 
> WALKey#getWriteEntry under high writing workload
> 
>
> Key: HBASE-16698
> URL: https://issues.apache.org/jira/browse/HBASE-16698
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.1.6, 1.2.3
>Reporter: Yu Li
>Assignee: Yu Li
> Attachments: HBASE-16698.branch-1.patch, HBASE-16698.patch, 
> HBASE-16698.v2.patch, hadoop0495.et2.jstack
>
>
> As titled, on our production environment we observed 98 out of 128 handlers 
> get stuck waiting for the CountDownLatch {{seqNumAssignedLatch}} inside 
> {{WALKey#getWriteEntry}} under a high writing workload.
> After digging into the problem, we found that the problem is mainly caused by 
> advancing mvcc in the append logic. Below is some detailed analysis:
> Under current branch-1 code logic, all batch puts will call 
> {{WALKey#getWriteEntry}} after appending edit to WAL, and 
> {{seqNumAssignedLatch}} is only released when the relative append call is 
> handled by RingBufferEventHandler (see {{FSWALEntry#stampRegionSequenceId}}). 
> Because currently we're using a single event handler for the ringbuffer, the 
> append calls are handled one by one (actually lot's of our current logic 
> depending on this sequential dealing logic), and this becomes a bottleneck 
> under high writing workload.
> The worst part is that by default we only use one WAL per RS, so appends on 
> all regions are dealt with in sequential, which causes contention among 
> different regions...
> To fix this, we could also take use of the "sequential appends" mechanism, 
> that we could grab the WriteEntry before publishing append onto ringbuffer 
> and use it as sequence id, only that we need to add a lock to make "grab 
> WriteEntry" and "append edit" a transaction. This will still cause contention 
> inside a region but could avoid contention between different regions. This 
> solution is already verified in our online environment and proved to be 
> effective.
> Notice that for master (2.0) branch since we already change the write 
> pipeline to sync before writing memstore (HBASE-15158), this issue only 
> exists for the ASYNC_WAL writes scenario.



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


[jira] [Updated] (HBASE-16698) Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload

2016-09-23 Thread Yu Li (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-16698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Yu Li updated HBASE-16698:
--
Attachment: HBASE-16698.v2.patch

Addressing review comments, moving constants from {{HConstants}} to {{HRegion}}

btw, we could enable/disable this mvcc preassign feature by setting 
{{hbase.hregion.mvcc.preassign}}, it could be set per-table, so if we know that 
workload won't introduce much write contention, we could set it to false to 
prevent the cost of preAssignMvccLock. This is specially useful if we use 
multiple WALs and some WALs are exclusive for specific application.

> Performance issue: handlers stuck waiting for CountDownLatch inside 
> WALKey#getWriteEntry under high writing workload
> 
>
> Key: HBASE-16698
> URL: https://issues.apache.org/jira/browse/HBASE-16698
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.1.6, 1.2.3
>Reporter: Yu Li
>Assignee: Yu Li
> Attachments: HBASE-16698.patch, HBASE-16698.v2.patch, 
> hadoop0495.et2.jstack
>
>
> As titled, on our production environment we observed 98 out of 128 handlers 
> get stuck waiting for the CountDownLatch {{seqNumAssignedLatch}} inside 
> {{WALKey#getWriteEntry}} under a high writing workload.
> After digging into the problem, we found that the problem is mainly caused by 
> advancing mvcc in the append logic. Below is some detailed analysis:
> Under current branch-1 code logic, all batch puts will call 
> {{WALKey#getWriteEntry}} after appending edit to WAL, and 
> {{seqNumAssignedLatch}} is only released when the relative append call is 
> handled by RingBufferEventHandler (see {{FSWALEntry#stampRegionSequenceId}}). 
> Because currently we're using a single event handler for the ringbuffer, the 
> append calls are handled one by one (actually lot's of our current logic 
> depending on this sequential dealing logic), and this becomes a bottleneck 
> under high writing workload.
> The worst part is that by default we only use one WAL per RS, so appends on 
> all regions are dealt with in sequential, which causes contention among 
> different regions...
> To fix this, we could also take use of the "sequential appends" mechanism, 
> that we could grab the WriteEntry before publishing append onto ringbuffer 
> and use it as sequence id, only that we need to add a lock to make "grab 
> WriteEntry" and "append edit" a transaction. This will still cause contention 
> inside a region but could avoid contention between different regions. This 
> solution is already verified in our online environment and proved to be 
> effective.
> Notice that for master (2.0) branch since we already change the write 
> pipeline to sync before writing memstore (HBASE-15158), this issue only 
> exists for the ASYNC_WAL writes scenario.



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


[jira] [Updated] (HBASE-16698) Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload

2016-09-23 Thread Yu Li (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-16698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Yu Li updated HBASE-16698:
--
Attachment: hadoop0495.et2.jstack

Uploading the jstack demonstrating the problem mentioned here.

> Performance issue: handlers stuck waiting for CountDownLatch inside 
> WALKey#getWriteEntry under high writing workload
> 
>
> Key: HBASE-16698
> URL: https://issues.apache.org/jira/browse/HBASE-16698
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.1.6, 1.2.3
>Reporter: Yu Li
>Assignee: Yu Li
> Attachments: HBASE-16698.patch, hadoop0495.et2.jstack
>
>
> As titled, on our production environment we observed 98 out of 128 handlers 
> get stuck waiting for the CountDownLatch {{seqNumAssignedLatch}} inside 
> {{WALKey#getWriteEntry}} under a high writing workload.
> After digging into the problem, we found that the problem is mainly caused by 
> advancing mvcc in the append logic. Below is some detailed analysis:
> Under current branch-1 code logic, all batch puts will call 
> {{WALKey#getWriteEntry}} after appending edit to WAL, and 
> {{seqNumAssignedLatch}} is only released when the relative append call is 
> handled by RingBufferEventHandler (see {{FSWALEntry#stampRegionSequenceId}}). 
> Because currently we're using a single event handler for the ringbuffer, the 
> append calls are handled one by one (actually lot's of our current logic 
> depending on this sequential dealing logic), and this becomes a bottleneck 
> under high writing workload.
> The worst part is that by default we only use one WAL per RS, so appends on 
> all regions are dealt with in sequential, which causes contention among 
> different regions...
> To fix this, we could also take use of the "sequential appends" mechanism, 
> that we could grab the WriteEntry before publishing append onto ringbuffer 
> and use it as sequence id, only that we need to add a lock to make "grab 
> WriteEntry" and "append edit" a transaction. This will still cause contention 
> inside a region but could avoid contention between different regions. This 
> solution is already verified in our online environment and proved to be 
> effective.
> Notice that for master (2.0) branch since we already change the write 
> pipeline to sync before writing memstore (HBASE-15158), this issue only 
> exists for the ASYNC_WAL writes scenario.



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


[jira] [Updated] (HBASE-16698) Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload

2016-09-23 Thread Yu Li (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-16698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Yu Li updated HBASE-16698:
--
Status: Patch Available  (was: Open)

Have ran several core tests in local and all passed, submit to get a HadoopQA 
report

> Performance issue: handlers stuck waiting for CountDownLatch inside 
> WALKey#getWriteEntry under high writing workload
> 
>
> Key: HBASE-16698
> URL: https://issues.apache.org/jira/browse/HBASE-16698
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.2.3, 1.1.6
>Reporter: Yu Li
>Assignee: Yu Li
> Attachments: HBASE-16698.patch
>
>
> As titled, on our production environment we observed 98 out of 128 handlers 
> get stuck waiting for the CountDownLatch {{seqNumAssignedLatch}} inside 
> {{WALKey#getWriteEntry}} under a high writing workload.
> After digging into the problem, we found that the problem is mainly caused by 
> advancing mvcc in the append logic. Below is some detailed analysis:
> Under current branch-1 code logic, all batch puts will call 
> {{WALKey#getWriteEntry}} after appending edit to WAL, and 
> {{seqNumAssignedLatch}} is only released when the relative append call is 
> handled by RingBufferEventHandler (see {{FSWALEntry#stampRegionSequenceId}}). 
> Because currently we're using a single event handler for the ringbuffer, the 
> append calls are handled one by one (actually lot's of our current logic 
> depending on this sequential dealing logic), and this becomes a bottleneck 
> under high writing workload.
> The worst part is that by default we only use one WAL per RS, so appends on 
> all regions are dealt with in sequential, which causes contention among 
> different regions...
> To fix this, we could also take use of the "sequential appends" mechanism, 
> that we could grab the WriteEntry before publishing append onto ringbuffer 
> and use it as sequence id, only that we need to add a lock to make "grab 
> WriteEntry" and "append edit" a transaction. This will still cause contention 
> inside a region but could avoid contention between different regions. This 
> solution is already verified in our online environment and proved to be 
> effective.
> Notice that for master (2.0) branch since we already change the write 
> pipeline to sync before writing memstore (HBASE-15158), this issue only 
> exists for the ASYNC_WAL writes scenario.



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


[jira] [Updated] (HBASE-16698) Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload

2016-09-23 Thread Yu Li (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-16698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Yu Li updated HBASE-16698:
--
Attachment: HBASE-16698.patch

Patch for master branch to better show the idea of the fix, will upload a patch 
for branch-1 later after review.

> Performance issue: handlers stuck waiting for CountDownLatch inside 
> WALKey#getWriteEntry under high writing workload
> 
>
> Key: HBASE-16698
> URL: https://issues.apache.org/jira/browse/HBASE-16698
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.1.6, 1.2.3
>Reporter: Yu Li
>Assignee: Yu Li
> Attachments: HBASE-16698.patch
>
>
> As titled, on our production environment we observed 98 out of 128 handlers 
> get stuck waiting for the CountDownLatch {{seqNumAssignedLatch}} inside 
> {{WALKey#getWriteEntry}} under a high writing workload.
> After digging into the problem, we found that the problem is mainly caused by 
> advancing mvcc in the append logic. Below is some detailed analysis:
> Under current branch-1 code logic, all batch puts will call 
> {{WALKey#getWriteEntry}} after appending edit to WAL, and 
> {{seqNumAssignedLatch}} is only released when the relative append call is 
> handled by RingBufferEventHandler (see {{FSWALEntry#stampRegionSequenceId}}). 
> Because currently we're using a single event handler for the ringbuffer, the 
> append calls are handled one by one (actually lot's of our current logic 
> depending on this sequential dealing logic), and this becomes a bottleneck 
> under high writing workload.
> The worst part is that by default we only use one WAL per RS, so appends on 
> all regions are dealt with in sequential, which causes contention among 
> different regions...
> To fix this, we could also take use of the "sequential appends" mechanism, 
> that we could grab the WriteEntry before publishing append onto ringbuffer 
> and use it as sequence id, only that we need to add a lock to make "grab 
> WriteEntry" and "append edit" a transaction. This will still cause contention 
> inside a region but could avoid contention between different regions. This 
> solution is already verified in our online environment and proved to be 
> effective.
> Notice that for master (2.0) branch since we already change the write 
> pipeline to sync before writing memstore (HBASE-15158), this issue only 
> exists for the ASYNC_WAL writes scenario.



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


[jira] [Updated] (HBASE-16698) Performance issue: handlers stuck waiting for CountDownLatch inside WALKey#getWriteEntry under high writing workload

2016-09-23 Thread Yu Li (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-16698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Yu Li updated HBASE-16698:
--
Summary: Performance issue: handlers stuck waiting for CountDownLatch 
inside WALKey#getWriteEntry under high writing workload  (was: Handlers stuck 
waiting for CountDownLatch inside WALKey#getWriteEntry under high writing 
workload)

> Performance issue: handlers stuck waiting for CountDownLatch inside 
> WALKey#getWriteEntry under high writing workload
> 
>
> Key: HBASE-16698
> URL: https://issues.apache.org/jira/browse/HBASE-16698
> Project: HBase
>  Issue Type: Improvement
>  Components: Performance
>Affects Versions: 1.1.6, 1.2.3
>Reporter: Yu Li
>Assignee: Yu Li
>
> As titled, on our production environment we observed 98 out of 128 handlers 
> get stuck waiting for the CountDownLatch {{seqNumAssignedLatch}} inside 
> {{WALKey#getWriteEntry}} under a high writing workload.
> After digging into the problem, we found that the problem is mainly caused by 
> advancing mvcc in the append logic. Below is some detailed analysis:
> Under current branch-1 code logic, all batch puts will call 
> {{WALKey#getWriteEntry}} after appending edit to WAL, and 
> {{seqNumAssignedLatch}} is only released when the relative append call is 
> handled by RingBufferEventHandler (see {{FSWALEntry#stampRegionSequenceId}}). 
> Because currently we're using a single event handler for the ringbuffer, the 
> append calls are handled one by one (actually lot's of our current logic 
> depending on this sequential dealing logic), and this becomes a bottleneck 
> under high writing workload.
> The worst part is that by default we only use one WAL per RS, so appends on 
> all regions are dealt with in sequential, which causes contention among 
> different regions...
> To fix this, we could also take use of the "sequential appends" mechanism, 
> that we could grab the WriteEntry before publishing append onto ringbuffer 
> and use it as sequence id, only that we need to add a lock to make "grab 
> WriteEntry" and "append edit" a transaction. This will still cause contention 
> inside a region but could avoid contention between different regions. This 
> solution is already verified in our online environment and proved to be 
> effective.
> Notice that for master (2.0) branch since we already change the write 
> pipeline to sync before writing memstore (HBASE-15158), this issue only 
> exists for the ASYNC_WAL writes scenario.



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