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

Lars George commented on HBASE-3889:
------------------------------------

After adding the patch I got eventually this

{noformat}
...
2011-05-16 14:15:31,428 DEBUG org.apache.hadoop.hbase.monitoring.MonitoredTask: 
markComplete: processed 50012 edits across 47 regions threw away edits for 13 
regions log file = 
hdfs://localhost/hbase/.logs/10.0.0.65,60020,1305406356765/10.0.0.65%2C60020%2C1305406356765.1305409968389
 is corrupted = false
2011-05-16 14:15:31,428 DEBUG 
org.apache.hadoop.hbase.regionserver.SplitLogWorker: After Exec Status: DONE
2011-05-16 14:15:31,429 INFO 
org.apache.hadoop.hbase.regionserver.SplitLogWorker: successfully transitioned 
task 
/hbase/splitlog/hdfs%3A%2F%2Flocalhost%2Fhbase%2F.logs%2F10.0.0.65%2C60020%2C1305406356765%2F10.0.0.65%252C60020%252C1305406356765.1305409968389
 to final state done
2011-05-16 14:15:31,429 INFO 
org.apache.hadoop.hbase.regionserver.SplitLogWorker: worker 
10.0.0.64,60020,1305546944326 done with task 
/hbase/splitlog/hdfs%3A%2F%2Flocalhost%2Fhbase%2F.logs%2F10.0.0.65%2C60020%2C1305406356765%2F10.0.0.65%252C60020%252C1305406356765.1305409968389
 in 879381ms
...
{noformat}

This was a 15GB edits that lingered since I had a crashed YSCB bulk load. The 
extra messages above are from log statements I added everywhere to see what is 
going on (might make sense to add a debug level output of 
MonitoredTaskImpl.setStatus(), was helpful here to see where it got lost).


> NPE in Distributed Log Splitting
> --------------------------------
>
>                 Key: HBASE-3889
>                 URL: https://issues.apache.org/jira/browse/HBASE-3889
>             Project: HBase
>          Issue Type: Bug
>          Components: regionserver
>    Affects Versions: 0.92.0
>         Environment: Pseudo-distributed on MacOS
>            Reporter: Lars George
>            Assignee: Lars George
>             Fix For: 0.92.0
>
>         Attachments: HBASE-3889.patch
>
>
> There is an issue with the log splitting under the specific condition of 
> edits belonging to a non existing region (which went away after a split for 
> example). The HLogSplitter fails to check the condition, which is handled on 
> a lower level, logging manifests it as 
> {noformat}
> 2011-05-16 13:56:10,300 INFO 
> org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: This region's 
> directory doesn't exist: 
> hdfs://localhost:8020/hbase/usertable/30c4d0a47703214845d0676d0c7b36f0. It is 
> very likely that it was already split so it's safe to discard those edits.
> {noformat}
> The code returns a null reference which is not check in 
> HLogSplitter.splitLogFileToTemp():
> {code}
> ...
>         WriterAndPath wap = (WriterAndPath)o;
>         if (wap == null) {
>           wap = createWAP(region, entry, rootDir, tmpname, fs, conf);
>           if (wap == null) {
>             logWriters.put(region, BAD_WRITER);
>           } else {
>             logWriters.put(region, wap);
>           }
>         }
>         wap.w.append(entry);
> ...
> {code}
> The createWAP does return "null" when the above message is logged based on 
> the obsolete region reference in the edit.
> What made this difficult to detect is that the error (and others) are 
> silently ignored in SplitLogWorker.grabTask(). I added a catch and error 
> logging to see the NPE that was caused by the above.
> {code}
> ...
>           break;
>       }
>     } catch (Exception e) {
>       LOG.error("An error occurred.", e);
>     } finally {
>       if (t > 0) {
> ...
> {code}
> As a side note, there are other errors/asserts triggered that this 
> try/finally not handles. For example
> {noformat}
> 2011-05-16 13:58:30,647 WARN 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker: BADVERSION failed to 
> assert ownership for 
> /hbase/splitlog/hdfs%3A%2F%2Flocalhost%2Fhbase%2F.logs%2F10.0.0.65%2C60020%2C1305406356765%2F10.0.0.65%252C60020%252C1305406356765.1305409968389
> org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = 
> BadVersion for 
> /hbase/splitlog/hdfs%3A%2F%2Flocalhost%2Fhbase%2F.logs%2F10.0.0.65%2C60020%2C1305406356765%2F10.0.0.65%252C60020%252C1305406356765.1305409968389
>         at 
> org.apache.zookeeper.KeeperException.create(KeeperException.java:106)
>         at 
> org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>         at org.apache.zookeeper.ZooKeeper.setData(ZooKeeper.java:1038)
>         at 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker.ownTask(SplitLogWorker.java:329)
>         at 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker.access$100(SplitLogWorker.java:68)
>         at 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker$2.progress(SplitLogWorker.java:265)
>         at 
> org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HLogSplitter.java:432)
>         at 
> org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HLogSplitter.java:354)
>         at 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker$1.exec(SplitLogWorker.java:113)
>         at 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker.grabTask(SplitLogWorker.java:260)
>         at 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker.taskLoop(SplitLogWorker.java:191)
>         at 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker.run(SplitLogWorker.java:164)
>         at java.lang.Thread.run(Thread.java:680)
> {noformat}
> This should probably be handled - or at least documented - in another issue?
> The NPE made the log split end and the SplitLogManager add an endless amount 
> of RESCAN entries as this never came to an end.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to