You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by Stanislav Barton <st...@internetmemory.net> on 2013/07/05 19:02:13 UTC

Bulk loading HFiles via LoadIncrementalHFiles fails at a region that is being compacted, a bug?

We are running HBase version 0.94.6-cdh4.3.0 and I have noticed that
quite often while bulk loading HFiles created with a MR job fails with
following Exception:

13/06/26 18:28:15 ERROR mapreduce.LoadIncrementalHFiles: Encountered
unrecoverable error from region server
org.apache.hadoop.hbase.client.RetriesExhaustedException: Failed after
attempts=10, exceptions:
Wed Jun 26 18:18:35 CEST 2013,
org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles$3@650e938c,
org.apache.hadoop.hbase.RegionTooBusyException:
org.apache.hadoop.hbase.RegionTooBusyException: failed to get a lock
in 60000ms
        at org.apache.hadoop.hbase.regionserver.HRegion.lock(HRegion.java:5770)
        at org.apache.hadoop.hbase.regionserver.HRegion.lock(HRegion.java:5756)
        at org.apache.hadoop.hbase.regionserver.HRegion.startBulkRegionOperation(HRegion.java:5715)
        at org.apache.hadoop.hbase.regionserver.HRegion.bulkLoadHFiles(HRegion.java:3526)
        at org.apache.hadoop.hbase.regionserver.HRegion.bulkLoadHFiles(HRegion.java:3509)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.bulkLoadHFiles(HRegionServer.java:2793)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.hbase.ipc.WritableRpcEngine$Server.call(WritableRpcEngine.java:320)
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1426)



I have firstly suspected that having a scanners on the region actually
blocks the acquiring of the lock to bulk load. But after stopping all
the reader clients, the load was still failing. I have then localized
the HFile that was failing to load and went to see what the region
server was doing at that time via the UI and found out that it was
compacting exactly the region to which the HFile would fit.

We have been runnning this workflow for a long  time without any
problem on a previous version of HBase bundled in CDH4.1.2 i.e. HBase
0.92, even under high load (compacting, splitting). I suppose that
bulkloading while compacting should not affect each other so it seems
to me that this is a bug. Can someone please confirm to me this
behaviour so I can take actions? (wait for a fix or implement a safety
ne on my side).

Thanks and Cheers,

Stan Barton

Re: Bulk loading HFiles via LoadIncrementalHFiles fails at a region that is being compacted, a bug?

Posted by Stack <st...@duboce.net>.
Hey Stan:

60seconds is a long time.

If you try upping it the wait, does it still fail still? (looking at how
long to wait is calculated, it is a little hairy figuring what to change).

Any chance of thread dump while it is hung up?  Might tell us something?

Good on you Stan,
St.Ack



On Fri, Jul 5, 2013 at 10:02 AM, Stanislav Barton <
stanislav.barton@internetmemory.net> wrote:

> We are running HBase version 0.94.6-cdh4.3.0 and I have noticed that
> quite often while bulk loading HFiles created with a MR job fails with
> following Exception:
>
> 13/06/26 18:28:15 ERROR mapreduce.LoadIncrementalHFiles: Encountered
> unrecoverable error from region server
> org.apache.hadoop.hbase.client.RetriesExhaustedException: Failed after
> attempts=10, exceptions:
> Wed Jun 26 18:18:35 CEST 2013,
> org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles$3@650e938c,
> org.apache.hadoop.hbase.RegionTooBusyException:
> org.apache.hadoop.hbase.RegionTooBusyException: failed to get a lock
> in 60000ms
>         at
> org.apache.hadoop.hbase.regionserver.HRegion.lock(HRegion.java:5770)
>         at
> org.apache.hadoop.hbase.regionserver.HRegion.lock(HRegion.java:5756)
>         at
> org.apache.hadoop.hbase.regionserver.HRegion.startBulkRegionOperation(HRegion.java:5715)
>         at
> org.apache.hadoop.hbase.regionserver.HRegion.bulkLoadHFiles(HRegion.java:3526)
>         at
> org.apache.hadoop.hbase.regionserver.HRegion.bulkLoadHFiles(HRegion.java:3509)
>         at
> org.apache.hadoop.hbase.regionserver.HRegionServer.bulkLoadHFiles(HRegionServer.java:2793)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>         at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at
> org.apache.hadoop.hbase.ipc.WritableRpcEngine$Server.call(WritableRpcEngine.java:320)
>         at
> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1426)
>
>
>
> I have firstly suspected that having a scanners on the region actually
> blocks the acquiring of the lock to bulk load. But after stopping all
> the reader clients, the load was still failing. I have then localized
> the HFile that was failing to load and went to see what the region
> server was doing at that time via the UI and found out that it was
> compacting exactly the region to which the HFile would fit.
>
> We have been runnning this workflow for a long  time without any
> problem on a previous version of HBase bundled in CDH4.1.2 i.e. HBase
> 0.92, even under high load (compacting, splitting). I suppose that
> bulkloading while compacting should not affect each other so it seems
> to me that this is a bug. Can someone please confirm to me this
> behaviour so I can take actions? (wait for a fix or implement a safety
> ne on my side).
>
> Thanks and Cheers,
>
> Stan Barton
>