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

ramkrishna vasudevan commented on HBASE-10499:
----------------------------------------------

https://issues.apache.org/jira/browse/HBASE-10499?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
causing RegionTooBusyException
----------------------------------------------------------------------------------------------
10499-v6.txt, HBASE-10499.patch, HBASE-10499_v5.patch,
compaction-queue.png, hbase-root-master-ip-10-157-0-229.zip,
hbase-root-regionserver-ip-10-93-128-92.zip, master_4e39.log,
master_576f.log, rs_4e39.log, rs_576f.log, t1.dump, t2.dump,
workloada_0.98.dat
to this version.  Doesn't seem so to me.
has 200 regions.  In one of the run with 0.98 server and 0.98 client I
faced this problem like the hlogs became more and the system requested
flushes for those many regions.
remained unflushed.  The ripple effect of this on the client side
org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException: Failed
54 actions: RegionTooBusyException: 54 times,
org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException: Failed
54 actions: RegionTooBusyException: 54 times,
org.apache.hadoop.hbase.client.AsyncProcess$BatchErrors.makeException(AsyncProcess.java:187)
org.apache.hadoop.hbase.client.AsyncProcess$BatchErrors.access$500(AsyncProcess.java:171)
org.apache.hadoop.hbase.client.AsyncProcess.getErrors(AsyncProcess.java:897)
org.apache.hadoop.hbase.client.HTable.backgroundFlushCommits(HTable.java:961)
org.apache.hadoop.hbase.client.HTable.flushCommits(HTable.java:1225)
wal.FSHLog: Too many hlogs: logs=38, maxlogs=32; forcing flush of 23
regions(s): 97d8ae2f78910cc5ded5fbb1ddad8492,
d396b8a1da05c871edcb68a15608fdf2, 01a68742a1be3a9705d574ad68fec1d7,
1250381046301e7465b6cf398759378e, 127c133f47d0419bd5ab66675aff76d4,
9f01c5d25ddc6675f750968873721253, 29c055b5690839c2fa357cd8e871741e,
ca4e33e3eb0d5f8314ff9a870fc43463, acfc6ae756e193b58d956cb71ccf0aa3,
187ea304069bc2a3c825bc10a59c7e84, 0ea411edc32d5c924d04bf126fa52d1e,
e2f9331fc7208b1b230a24045f3c869e, d9309ca864055eddf766a330352efc7a,
1a71bdf457288d449050141b5ff00c69, 0ba9089db28e977f86a27f90bbab9717,
fdbb3242d3b673bbe4790a47bc30576f, bbadaa1f0e62d8a8650080b824187850,
b1a5de30d8603bd5d9022e09c574501b, cc6a9fabe44347ed65e7c325faa72030,
313b17dbff2497f5041b57fe13fa651e, 6b788c498503ddd3e1433a4cd3fb4e39,
3d71274fe4f815882e9626e1cfa050d1, acc43e4b42c1a041078774f4f20a3ff5
wal.FSHLog: Too many hlogs: logs=53, maxlogs=32; forcing flush of 2
regions(s): fdbb3242d3b673bbe4790a47bc30576f,
6b788c498503ddd3e1433a4cd3fb4e39
regionserver.HRegionServer: regionserver60020.periodicFlusher requesting
flush for region
usertable,user3654,1392107806977.fdbb3242d3b673bbe4790a47bc30576f. after a
delay of 16689
regionserver.HRegionServer: regionserver60020.periodicFlusher requesting
flush for region
usertable,user6264,1392107806983.6b788c498503ddd3e1433a4cd3fb4e39. after a
delay of 15868
regionserver.HRegionServer: regionserver60020.periodicFlusher requesting
flush for region
usertable,user3654,1392107806977.fdbb3242d3b673bbe4790a47bc30576f. after a
delay of 20847
regionserver.HRegionServer: regionserver60020.periodicFlusher requesting
flush for region
usertable,user6264,1392107806983.6b788c498503ddd3e1433a4cd3fb4e39. after a
delay of 20099
regionserver.HRegionServer: regionserver60020.periodicFlusher requesting
flush for region
usertable,user3654,1392107806977.fdbb3242d3b673bbe4790a47bc30576f. after a
delay of 8677
wal.FSHLog: Too many hlogs: logs=54, maxlogs=32; forcing flush of 1
regions(s): fdbb3242d3b673bbe4790a47bc30576f
regions but this region stays with the RS that has this issue.  One
important observation is that in HRegion.internalflushCache() we need to
add a debug log here
does not happen and no logs related to flush are printed in the logs. so
due to some reason this memstore.size() has become 0( I assume this).  The
earlier bugs were also due to similar reason.


> In write heavy scenario one of the regions does not get flushed causing 
> RegionTooBusyException
> ----------------------------------------------------------------------------------------------
>
>                 Key: HBASE-10499
>                 URL: https://issues.apache.org/jira/browse/HBASE-10499
>             Project: HBase
>          Issue Type: Bug
>    Affects Versions: 0.98.0
>            Reporter: ramkrishna.s.vasudevan
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Critical
>             Fix For: 1.0.0, 2.0.0, 0.98.10, 1.1.0
>
>         Attachments: 10499-0.98.txt, 10499-1.0.txt, 10499-v2.txt, 
> 10499-v3.txt, 10499-v4.txt, 10499-v6.txt, 10499-v6.txt, 10499-v7.txt, 
> 10499-v8.txt, HBASE-10499.patch, HBASE-10499_v5.patch, compaction-queue.png, 
> hbase-root-master-ip-10-157-0-229.zip, 
> hbase-root-regionserver-ip-10-93-128-92.zip, master_4e39.log, 
> master_576f.log, rs_4e39.log, rs_576f.log, t1.dump, t2.dump, 
> workloada_0.98.dat
>
>
> I got this while testing 0.98RC.  But am not sure if it is specific to this 
> version.  Doesn't seem so to me.  
> Also it is something similar to HBASE-5312 and HBASE-5568.
> Using 10 threads i do writes to 4 RS using YCSB. The table created has 200 
> regions.  In one of the run with 0.98 server and 0.98 client I faced this 
> problem like the hlogs became more and the system requested flushes for those 
> many regions.
> One by one everything was flushed except one and that one thing remained 
> unflushed.  The ripple effect of this on the client side
> {code}
> com.yahoo.ycsb.DBException: 
> org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException: Failed 
> 54 actions: RegionTooBusyException: 54 times,
>         at com.yahoo.ycsb.db.HBaseClient.cleanup(HBaseClient.java:245)
>         at com.yahoo.ycsb.DBWrapper.cleanup(DBWrapper.java:73)
>         at com.yahoo.ycsb.ClientThread.run(Client.java:307)
> Caused by: 
> org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException: Failed 
> 54 actions: RegionTooBusyException: 54 times,
>         at 
> org.apache.hadoop.hbase.client.AsyncProcess$BatchErrors.makeException(AsyncProcess.java:187)
>         at 
> org.apache.hadoop.hbase.client.AsyncProcess$BatchErrors.access$500(AsyncProcess.java:171)
>         at 
> org.apache.hadoop.hbase.client.AsyncProcess.getErrors(AsyncProcess.java:897)
>         at 
> org.apache.hadoop.hbase.client.HTable.backgroundFlushCommits(HTable.java:961)
>         at 
> org.apache.hadoop.hbase.client.HTable.flushCommits(HTable.java:1225)
>         at com.yahoo.ycsb.db.HBaseClient.cleanup(HBaseClient.java:232)
>         ... 2 more
> {code}
> On one of the RS
> {code}
> 2014-02-11 08:45:58,714 INFO  [regionserver60020.logRoller] wal.FSHLog: Too 
> many hlogs: logs=38, maxlogs=32; forcing flush of 23 regions(s): 
> 97d8ae2f78910cc5ded5fbb1ddad8492, d396b8a1da05c871edcb68a15608fdf2, 
> 01a68742a1be3a9705d574ad68fec1d7, 1250381046301e7465b6cf398759378e, 
> 127c133f47d0419bd5ab66675aff76d4, 9f01c5d25ddc6675f750968873721253, 
> 29c055b5690839c2fa357cd8e871741e, ca4e33e3eb0d5f8314ff9a870fc43463, 
> acfc6ae756e193b58d956cb71ccf0aa3, 187ea304069bc2a3c825bc10a59c7e84, 
> 0ea411edc32d5c924d04bf126fa52d1e, e2f9331fc7208b1b230a24045f3c869e, 
> d9309ca864055eddf766a330352efc7a, 1a71bdf457288d449050141b5ff00c69, 
> 0ba9089db28e977f86a27f90bbab9717, fdbb3242d3b673bbe4790a47bc30576f, 
> bbadaa1f0e62d8a8650080b824187850, b1a5de30d8603bd5d9022e09c574501b, 
> cc6a9fabe44347ed65e7c325faa72030, 313b17dbff2497f5041b57fe13fa651e, 
> 6b788c498503ddd3e1433a4cd3fb4e39, 3d71274fe4f815882e9626e1cfa050d1, 
> acc43e4b42c1a041078774f4f20a3ff5
> ......................................................
> 2014-02-11 08:47:49,580 INFO  [regionserver60020.logRoller] wal.FSHLog: Too 
> many hlogs: logs=53, maxlogs=32; forcing flush of 2 regions(s): 
> fdbb3242d3b673bbe4790a47bc30576f, 6b788c498503ddd3e1433a4cd3fb4e39
> {code}
> {code}
> 2014-02-11 09:42:44,237 INFO  [regionserver60020.periodicFlusher] 
> regionserver.HRegionServer: regionserver60020.periodicFlusher requesting 
> flush for region 
> usertable,user3654,1392107806977.fdbb3242d3b673bbe4790a47bc30576f. after a 
> delay of 16689
> 2014-02-11 09:42:44,237 INFO  [regionserver60020.periodicFlusher] 
> regionserver.HRegionServer: regionserver60020.periodicFlusher requesting 
> flush for region 
> usertable,user6264,1392107806983.6b788c498503ddd3e1433a4cd3fb4e39. after a 
> delay of 15868
> 2014-02-11 09:42:54,238 INFO  [regionserver60020.periodicFlusher] 
> regionserver.HRegionServer: regionserver60020.periodicFlusher requesting 
> flush for region 
> usertable,user3654,1392107806977.fdbb3242d3b673bbe4790a47bc30576f. after a 
> delay of 20847
> 2014-02-11 09:42:54,238 INFO  [regionserver60020.periodicFlusher] 
> regionserver.HRegionServer: regionserver60020.periodicFlusher requesting 
> flush for region 
> usertable,user6264,1392107806983.6b788c498503ddd3e1433a4cd3fb4e39. after a 
> delay of 20099
> 2014-02-11 09:43:04,238 INFO  [regionserver60020.periodicFlusher] 
> regionserver.HRegionServer: regionserver60020.periodicFlusher requesting 
> flush for region 
> usertable,user3654,1392107806977.fdbb3242d3b673bbe4790a47bc30576f. after a 
> delay of 8677
> {code}
> {code}
> 2014-02-11 10:31:21,020 INFO  [regionserver60020.logRoller] wal.FSHLog: Too 
> many hlogs: logs=54, maxlogs=32; forcing flush of 1 regions(s): 
> fdbb3242d3b673bbe4790a47bc30576f
> {code}
> I restarted another RS and there were region movements with other regions but 
> this region stays with the RS that has this issue.  One important observation 
> is that in HRegion.internalflushCache() we need to add a debug log here
> {code}
> // If nothing to flush, return and avoid logging start/stop flush.
>     if (this.memstoreSize.get() <= 0) {
>       return false;
>     }
> {code}
> Because we can see that the region is requsted for a flush but it does not 
> happen and no logs related to flush are printed in the logs. so due to some 
> reason this memstore.size() has become 0( I assume this).  The earlier bugs 
> were also due to similar reason.



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

Reply via email to