[ https://issues.apache.org/jira/browse/HBASE-5029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13172677#comment-13172677 ]
Prakash Khemani commented on HBASE-5029: ---------------------------------------- The cause for this error appears to be the following DFSClient exception 2011-12-17 01:14:48,369 ERROR [SplitLogWorker-janus.apache.org,53708,1324084461889] regionserver.SplitLogWorker(169): unexpected error java.lang.NullPointerException at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeThreads(DFSClient.jav a:3831) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.ja va:3874) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3809) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStr eam.java:61) at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:86) at org.apache.hadoop.io.SequenceFile$Writer.close(SequenceFile.java:1017) at org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter.close(Sequen ceFileLogWriter.java:214) at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HL ogSplitter.java:458) at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HL ogSplitter.java:351) at org.apache.hadoop.hbase.regionserver.SplitLogWorker$1.exec(SplitLogWorker.j ava:113) at org.apache.hadoop.hbase.regionserver.SplitLogWorker.grabTask(SplitLogWorker .java:266) at org.apache.hadoop.hbase.regionserver.SplitLogWorker.taskLoop(SplitLogWorker .java:197) at org.apache.hadoop.hbase.regionserver.SplitLogWorker.run(SplitLogWorker.java :165) at java.lang.Thread.run(Thread.java:662) (Much earlier in the logs, while the SplitLogWorker was trying to recover-lease on the log file it had received a Thread.interrupt() because the region server was exiting. That thread.interrupt() was unsuccessful in interrupting the recoverLease() call. It is also possible that the interrupt was eaten up during the recoverLease() call.) the split-log-worker thread continued to split the log file. It successfully split the file ... But in the end it hit this exception. It is possible that the file-system was closed by the time the above exception happened.) The fix probably requires some more checking in DFSClient$DFSOutputStream.closeInternal() for a closed file system. The more difficult task is to make sure that recoverLease() handles interrupts correctly. On 12/14/11 2:33 PM, "Zhihong Yu (Commented) (JIRA)" <j...@apache.org> > TestDistributedLogSplitting fails on occasion > --------------------------------------------- > > Key: HBASE-5029 > URL: https://issues.apache.org/jira/browse/HBASE-5029 > Project: HBase > Issue Type: Bug > Reporter: stack > Assignee: Prakash Khemani > Priority: Critical > Attachments: > 0001-HBASE-5029-jira-TestDistributedLogSplitting-fails-on.patch, > 5029-addingignore.txt, HBASE-5029.D891.1.patch, HBASE-5029.D891.2.patch > > > This is how it usually fails: > https://builds.apache.org/view/G-L/view/HBase/job/HBase-0.92/lastCompletedBuild/testReport/org.apache.hadoop.hbase.master/TestDistributedLogSplitting/testWorkerAbort/ > Assigning mighty Prakash since he offered to take a looksee. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira