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

Todd Lipcon commented on HBASE-2467:
------------------------------------

bq. We don't need it anymore, but having deferred log flush could still be 
useful. Should be in the same issue.

There is one case in which group commit is still helpful - the way sync works 
is that it always sends the last "chunk" of the file, even if some of the bytes 
were already sent. The chunks here are 512 bytes (ie checksum boundaries). So, 
if you're always syncing very small edits (ie <512 bytes) then there is a 
benefit to waiting a milli or two to cross that 512-byte boundary. Otherwise 
with each write we will re-send the previous writes as well during the sync.

We can do a little better on the HDFS side to get this "for free", though, so 
probably not worth worrying about in this patch.

> Concurrent flushers in HLog sync using HDFS-895
> -----------------------------------------------
>
>                 Key: HBASE-2467
>                 URL: https://issues.apache.org/jira/browse/HBASE-2467
>             Project: HBase
>          Issue Type: Improvement
>          Components: regionserver
>            Reporter: Todd Lipcon
>            Assignee: Todd Lipcon
>             Fix For: 0.90.0
>
>         Attachments: HBASE-2467-v2.patch, hbase-2467.txt
>
>
> HDFS-895 changes hflush() to be able to run concurrently from multiple 
> threads, where flushes can be concurrent with further writes to the same file.
> We need to rip out/amend the group commit code a bit to take advantage of 
> this.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to