[ 
https://issues.apache.org/jira/browse/HBASE-7836?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jeffrey Zhong updated HBASE-7836:
---------------------------------

    Attachment: hbase-7836_v2.patch

{quote}
testWorkerAbort(org.apache.hadoop.hbase.master.TestDistributedLogSplitting): 
none of the following counters went up in 80000 milliseconds - 
tot_wkr_task_resigned, tot_wkr_task_err, tot_wkr_final_transition_failed, 
tot_wkr_task_done, tot_wkr_preempt_task
{quote}

This is due to we don't handle FSHDFSUtils.recoverFileLease 
java.nio.channels.ClosedByInterruptException. When we get this exception, we 
still call {code}FSDataOutputStream out = fs.append(p);{code}, that causes one 
extra min wait and then fails the test case due to timeout. Below are related 
log traces:

{code}
2013-04-01 14:45:04,735 DEBUG [SplitLogWorker-10.11.2.103,58161,1364852631051] 
util.FSHDFSUtils(95): Failed fs.recoverLease invocation, java.io.IOException: 
Call to localhost/127.0.0.1:58147 failed on local exception: 
java.nio.channels.ClosedByInterruptException, trying fs.append instead
2013-04-01 14:45:04,735 DEBUG [SplitLogWorker-10.11.2.103,58161,1364852631051] 
util.FSHDFSUtils(100): trying fs.append for 
hdfs://localhost:58147/user/jzhong/hbase/.logs/10.11.2.103,58161,1364852631051/10.11.2.103%2C58161%2C1364852631051.1364852632043
 with java.io.IOException: Call to localhost/127.0.0.1:58147 failed on local 
exception: java.nio.channels.ClosedByInterruptException
...
2013-04-01 14:46:04,737 WARN  [SplitLogWorker-10.11.2.103,58161,1364852631051] 
regionserver.SplitLogWorker$1(124): log splitting of 
hdfs://localhost:58147/user/jzhong/hbase/.logs/10.11.2.103,58161,1364852631051/10.11.2.103%2C58161%2C1364852631051.1364852632043
 failed, returning error
java.io.IOException: Failed to open 
hdfs://localhost:58147/user/jzhong/hbase/.logs/10.11.2.103,58161,1364852631051/10.11.2.103%2C58161%2C1364852631051.1364852632043
 for append
        at 
org.apache.hadoop.hbase.util.FSHDFSUtils.recoverFileLease(FSHDFSUtils.java:126)
        at 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.getReader(HLogSplitter.java:743)
        at 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFile(HLogSplitter.java:436)
        at 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFile(HLogSplitter.java:397)
        at 
org.apache.hadoop.hbase.regionserver.SplitLogWorker$1.exec(SplitLogWorker.java:111)
        at 
org.apache.hadoop.hbase.regionserver.SplitLogWorker.grabTask(SplitLogWorker.java:274)
        at 
org.apache.hadoop.hbase.regionserver.SplitLogWorker.taskLoop(SplitLogWorker.java:195)
        at 
org.apache.hadoop.hbase.regionserver.SplitLogWorker.run(SplitLogWorker.java:162)
        at java.lang.Thread.run(Thread.java:680)
Caused by: java.io.IOException: Call to localhost/127.0.0.1:58147 failed on 
local exception: java.nio.channels.ClosedByInterruptException
        at org.apache.hadoop.ipc.Client.wrapException(Client.java:1144)
        at org.apache.hadoop.ipc.Client.call(Client.java:1112)
        at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:229)
….
{code}

I fixed other test failures in the new patch.

Thanks,
-Jeffrey
                
> Create a new "replay" command so that recovered edits won't mess up normal 
> coprocessing & metrics
> -------------------------------------------------------------------------------------------------
>
>                 Key: HBASE-7836
>                 URL: https://issues.apache.org/jira/browse/HBASE-7836
>             Project: HBase
>          Issue Type: Sub-task
>            Reporter: Jeffrey Zhong
>            Assignee: Jeffrey Zhong
>             Fix For: 0.95.0
>
>         Attachments: hbase-7836_v1.patch, hbase-7836_v2.patch
>
>


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to