We used to have group commit in HBase until Todd did the work to do
the same thing the DFSClient so we removed it.

https://issues.apache.org/jira/browse/HBASE-1939

J-D

On Fri, Jun 14, 2013 at 5:25 PM, lars hofhansl <[email protected]> wrote:
> While looking at the HLog code I saw some funny methods that are *only* 
> called from tests.
> For example:
>
>   public long append(HRegionInfo regionInfo, HLogKey logKey, WALEdit logEdit,
>                      HTableDescriptor htd, boolean doSync)
>
>
> which seems to be (almost) a carbon copy of:
>
>
>   private long append(HRegionInfo info, byte [] tableName, WALEdit edits, 
> UUID clusterId,
>       final long now, HTableDescriptor htd, boolean doSync)
>
>
> I plan to remove the former.
>
> Second:
> Should we think about the notion of "group commits". Many relational 
> databases delay the commit of a transaction a bit in order to group multiple 
> commits together.
> Only after all such grouped transactions are committed are the respective 
> clients notified.
>
> HBase does a bit of this. If Puts hit the same RegionServer at the same time, 
> the edits get added to WAL and all outstanding edits are sync'ed at the end 
> of the Put.
> We could widen the window by delaying this by (say) a second or so, and thus 
> reduce the number of hflushes (or more importantly hsyncs, once I finally get 
> to adding them to HBase).
> Or is the current logic good enough?
>
>
> -- Lars
>

Reply via email to