[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-12-13 Thread stack (JIRA)

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

stack updated HBASE-8755:
-

   Resolution: Fixed
Fix Version/s: 0.99.0
 Assignee: Feng Honghua  (was: stack)
 Release Note: Redo of thread model writing edits to the WAL; slower when 
few clients but as concurrency rises, it makes for better throughput.
 Hadoop Flags: Reviewed
   Status: Resolved  (was: Patch Available)

Committed to trunk.  [~andrew.purt...@gmail.com] Do you want this in 0.98?

Thanks for the hard work [~fenghh] and persistence getting this in.

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: Performance, wal
Reporter: Feng Honghua
Assignee: Feng Honghua
Priority: Critical
 Fix For: 0.99.0

 Attachments: 8755-syncer.patch, 8755trunkV2.txt, 8755v8.txt, 
 8755v9.txt, HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch, 
 HBASE-8755-0.96-v0.patch, HBASE-8755-trunk-V0.patch, 
 HBASE-8755-trunk-V1.patch, HBASE-8755-trunk-v4.patch, 
 HBASE-8755-trunk-v6.patch, HBASE-8755-trunk-v7.patch, HBASE-8755-v5.patch, 
 thread.out


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-12-13 Thread stack (JIRA)

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

stack updated HBASE-8755:
-

Fix Version/s: 0.98.0

Committed to 0.98 also.

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: Performance, wal
Reporter: Feng Honghua
Assignee: Feng Honghua
Priority: Critical
 Fix For: 0.98.0, 0.99.0

 Attachments: 8755-syncer.patch, 8755trunkV2.txt, 8755v8.txt, 
 8755v9.txt, HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch, 
 HBASE-8755-0.96-v0.patch, HBASE-8755-trunk-V0.patch, 
 HBASE-8755-trunk-V1.patch, HBASE-8755-trunk-v4.patch, 
 HBASE-8755-trunk-v6.patch, HBASE-8755-trunk-v7.patch, HBASE-8755-v5.patch, 
 thread.out


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-12-12 Thread stack (JIRA)

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

stack updated HBASE-8755:
-

Attachment: thread.out

A couple of thread dumps w/ WAL hang in them.

Sorry, should have attached this last night [~fenghh]

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: Performance, wal
Reporter: Feng Honghua
Assignee: stack
Priority: Critical
 Attachments: 8755-syncer.patch, 8755trunkV2.txt, 
 HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch, HBASE-8755-0.96-v0.patch, 
 HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch, 
 HBASE-8755-trunk-v4.patch, HBASE-8755-trunk-v6.patch, 
 HBASE-8755-trunk-v7.patch, HBASE-8755-v5.patch, thread.out


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-12-12 Thread stack (JIRA)

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

stack updated HBASE-8755:
-

Attachment: 8755v8.txt

Same as [~fenghh]'s patch only it checks for null writer before using it -- 
this is currently in the code and seems to make this patch work again (i'm 
testing)  -- and adds this on tail of each Async* thread:

+  } catch (Exception e) {
+LOG.error(UNEXPECTED, e);
   } finally {

Also renames the threads from AsyncHLog* to WAL.Async.  Minor.

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: Performance, wal
Reporter: Feng Honghua
Assignee: stack
Priority: Critical
 Attachments: 8755-syncer.patch, 8755trunkV2.txt, 8755v8.txt, 
 HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch, HBASE-8755-0.96-v0.patch, 
 HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch, 
 HBASE-8755-trunk-v4.patch, HBASE-8755-trunk-v6.patch, 
 HBASE-8755-trunk-v7.patch, HBASE-8755-v5.patch, thread.out


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-12-12 Thread stack (JIRA)

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

stack updated HBASE-8755:
-

Attachment: 8755v9.txt

Fix compile issue.

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: Performance, wal
Reporter: Feng Honghua
Assignee: stack
Priority: Critical
 Attachments: 8755-syncer.patch, 8755trunkV2.txt, 8755v8.txt, 
 8755v9.txt, HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch, 
 HBASE-8755-0.96-v0.patch, HBASE-8755-trunk-V0.patch, 
 HBASE-8755-trunk-V1.patch, HBASE-8755-trunk-v4.patch, 
 HBASE-8755-trunk-v6.patch, HBASE-8755-trunk-v7.patch, HBASE-8755-v5.patch, 
 thread.out


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-12-10 Thread Feng Honghua (JIRA)

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

Feng Honghua updated HBASE-8755:


Attachment: HBASE-8755-trunk-v7.patch

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: Performance, wal
Reporter: Feng Honghua
Assignee: stack
Priority: Critical
 Attachments: 8755-syncer.patch, 8755trunkV2.txt, 
 HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch, HBASE-8755-0.96-v0.patch, 
 HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch, 
 HBASE-8755-trunk-v4.patch, HBASE-8755-trunk-v6.patch, 
 HBASE-8755-trunk-v7.patch, HBASE-8755-v5.patch


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-12-09 Thread Feng Honghua (JIRA)

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

Feng Honghua updated HBASE-8755:


Attachment: HBASE-8755-trunk-v6.patch

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: Performance, wal
Reporter: Feng Honghua
Assignee: stack
Priority: Critical
 Attachments: 8755-syncer.patch, 8755trunkV2.txt, 
 HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch, HBASE-8755-0.96-v0.patch, 
 HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch, 
 HBASE-8755-trunk-v4.patch, HBASE-8755-trunk-v6.patch, HBASE-8755-v5.patch


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-12-09 Thread stack (JIRA)

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

stack updated HBASE-8755:
-

Status: Patch Available  (was: Open)

See what hadoopqa thinks of [~fenghh] 's patch

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: Performance, wal
Reporter: Feng Honghua
Assignee: stack
Priority: Critical
 Attachments: 8755-syncer.patch, 8755trunkV2.txt, 
 HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch, HBASE-8755-0.96-v0.patch, 
 HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch, 
 HBASE-8755-trunk-v4.patch, HBASE-8755-trunk-v6.patch, HBASE-8755-v5.patch


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-12-08 Thread Himanshu Vashishtha (JIRA)

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

Himanshu Vashishtha updated HBASE-8755:
---

Attachment: HBASE-8755-v5.patch

Patch is remarkably stale after 10010 and 10048 went in; attaching a rebased 
patch of the original version.

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: Performance, wal
Reporter: Feng Honghua
Assignee: stack
Priority: Critical
 Attachments: 8755-syncer.patch, 8755trunkV2.txt, 
 HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch, HBASE-8755-0.96-v0.patch, 
 HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch, 
 HBASE-8755-trunk-v4.patch, HBASE-8755-v5.patch


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-12-05 Thread Himanshu Vashishtha (JIRA)

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

Himanshu Vashishtha updated HBASE-8755:
---

Attachment: 8755-syncer.patch

Attached is the prototype patch (HLogPE change inclusive). Thanks.

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: Performance, wal
Reporter: Feng Honghua
Assignee: stack
Priority: Critical
 Attachments: 8755-syncer.patch, 8755trunkV2.txt, 
 HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch, HBASE-8755-0.96-v0.patch, 
 HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch, 
 HBASE-8755-trunk-v4.patch


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)



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


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-11-21 Thread Feng Honghua (JIRA)

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

Feng Honghua updated HBASE-8755:


Attachment: HBASE-8755-trunk-v4.patch
HBASE-8755-0.96-v0.patch

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: Performance, wal
Reporter: Feng Honghua
Assignee: stack
Priority: Critical
 Attachments: 8755trunkV2.txt, HBASE-8755-0.94-V0.patch, 
 HBASE-8755-0.94-V1.patch, HBASE-8755-0.96-v0.patch, 
 HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch, 
 HBASE-8755-trunk-v4.patch


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)



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


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-11-11 Thread stack (JIRA)

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

stack updated HBASE-8755:
-

Fix Version/s: (was: 0.96.1)

Moving out of 0.96.1.  We'll get it when we get it.

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: Performance, wal
Reporter: Feng Honghua
Assignee: stack
Priority: Critical
 Attachments: 8755trunkV2.txt, HBASE-8755-0.94-V0.patch, 
 HBASE-8755-0.94-V1.patch, HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)



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


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-09-07 Thread stack (JIRA)

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

stack updated HBASE-8755:
-

Attachment: 8755trunkV2.txt

Rebase of trunk.

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: Performance, wal
Reporter: Feng Honghua
Assignee: stack
Priority: Critical
 Fix For: 0.96.1

 Attachments: 8755trunkV2.txt, HBASE-8755-0.94-V0.patch, 
 HBASE-8755-0.94-V1.patch, HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-09-05 Thread stack (JIRA)

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

stack updated HBASE-8755:
-

  Component/s: Performance
 Priority: Critical  (was: Major)
Fix Version/s: 0.96.1
 Assignee: stack

Let me try this.

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: Performance, wal
Reporter: Feng Honghua
Assignee: stack
Priority: Critical
 Fix For: 0.96.1

 Attachments: HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch, 
 HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-07-23 Thread Feng Honghua (JIRA)

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

Feng Honghua updated HBASE-8755:


Attachment: HBASE-8755-trunk-V1.patch

updated patch rebased on latest trunk code base

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: wal
Reporter: Feng Honghua
 Attachments: HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch, 
 HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-06-18 Thread Feng Honghua (JIRA)

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

Feng Honghua updated HBASE-8755:


Attachment: HBASE-8755-trunk-V0.patch

new write thread model patch based on trunk

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: wal
Reporter: Feng Honghua
 Attachments: HBASE-8755-0.94-V0.patch, HBASE-8755-trunk-V0.patch


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-06-18 Thread Feng Honghua (JIRA)

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

Feng Honghua updated HBASE-8755:


Attachment: HBASE-8755-0.94-V1.patch

an update patch based on 0.94 according to Ted/stack's review attached

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: wal
Reporter: Feng Honghua
 Attachments: HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch, 
 HBASE-8755-trunk-V0.patch


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput

2013-06-17 Thread Feng Honghua (JIRA)

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

Feng Honghua updated HBASE-8755:


Attachment: HBASE-8755-0.94-V0.patch

the patch HBASE-8755-0.94-V0.patch is based on 
http://svn.apache.org/repos/asf/hbase/branches/0.94

 A new write thread model for HLog to improve the overall HBase write 
 throughput
 ---

 Key: HBASE-8755
 URL: https://issues.apache.org/jira/browse/HBASE-8755
 Project: HBase
  Issue Type: Improvement
  Components: wal
Reporter: Feng Honghua
 Attachments: HBASE-8755-0.94-V0.patch


 In current write model, each write handler thread (executing put()) will 
 individually go through a full 'append (hlog local buffer) = HLog writer 
 append (write to hdfs) = HLog writer sync (sync hdfs)' cycle for each write, 
 which incurs heavy race condition on updateLock and flushLock.
 The only optimization where checking if current syncTillHere  txid in 
 expectation for other thread help write/sync its own txid to hdfs and 
 omitting the write/sync actually help much less than expectation.
 Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi 
 proposed a new write thread model for writing hdfs sequence file and the 
 prototype implementation shows a 4X improvement for throughput (from 17000 to 
 7+). 
 I apply this new write thread model in HLog and the performance test in our 
 test cluster shows about 3X throughput improvement (from 12150 to 31520 for 1 
 RS, from 22000 to 7 for 5 RS), the 1 RS write throughput (1K row-size) 
 even beats the one of BigTable (Precolator published in 2011 says Bigtable's 
 write throughput then is 31002). I can provide the detailed performance test 
 results if anyone is interested.
 The change for new write thread model is as below:
  1 All put handler threads append the edits to HLog's local pending buffer; 
 (it notifies AsyncWriter thread that there is new edits in local buffer)
  2 All put handler threads wait in HLog.syncer() function for underlying 
 threads to finish the sync that contains its txid;
  3 An single AsyncWriter thread is responsible for retrieve all the buffered 
 edits in HLog's local pending buffer and write to the hdfs 
 (hlog.writer.append); (it notifies AsyncFlusher thread that there is new 
 writes to hdfs that needs a sync)
  4 An single AsyncFlusher thread is responsible for issuing a sync to hdfs 
 to persist the writes by AsyncWriter; (it notifies the AsyncNotifier thread 
 that sync watermark increases)
  5 An single AsyncNotifier thread is responsible for notifying all pending 
 put handler threads which are waiting in the HLog.syncer() function
  6 No LogSyncer thread any more (since there is always 
 AsyncWriter/AsyncFlusher threads do the same job it does)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira