HBase
  1. HBase
  2. HBASE-10922

Log splitting status should always be closed

    Details

    • Type: Bug Bug
    • Status: Closed
    • Priority: Minor Minor
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 0.99.0, 0.98.4
    • Component/s: wal
    • Labels:
      None
    • Hadoop Flags:
      Reviewed

      Description

      With distributed log replay enabled by default, I ran into an issue that log splitting hasn't completed after 13 hours. It seems to hang somewhere.

      1. hbase-10922.patch
        2 kB
        Jeffrey Zhong
      2. hbase-10922-v2.patch
        3 kB
        Jeffrey Zhong
      3. log-splitting_hang.png
        69 kB
        Jimmy Xiang
      4. master.log.gz
        335 kB
        stack
      5. master-log-grep.txt
        16 kB
        Jimmy Xiang

        Activity

        Hide
        Enis Soztutar added a comment -

        Closing this issue after 0.99.0 release.

        Show
        Enis Soztutar added a comment - Closing this issue after 0.99.0 release.
        Hide
        Hudson added a comment -

        SUCCESS: Integrated in HBase-TRUNK #5169 (See https://builds.apache.org/job/HBase-TRUNK/5169/)
        HBASE-10922: Log splitting status should always be closed (jzhong: rev 7465c319472dfb3c98474369cdc15514eeea6293)

        • hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java
        Show
        Hudson added a comment - SUCCESS: Integrated in HBase-TRUNK #5169 (See https://builds.apache.org/job/HBase-TRUNK/5169/ ) HBASE-10922 : Log splitting status should always be closed (jzhong: rev 7465c319472dfb3c98474369cdc15514eeea6293) hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java
        Hide
        Hudson added a comment -

        SUCCESS: Integrated in HBase-0.98-on-Hadoop-1.1 #308 (See https://builds.apache.org/job/HBase-0.98-on-Hadoop-1.1/308/)
        HBASE-10922: Log splitting status should always be closed (jzhong: rev 0b2a5f5880ba8bb102efbb72e54970e88726d8f9)

        • hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java
        Show
        Hudson added a comment - SUCCESS: Integrated in HBase-0.98-on-Hadoop-1.1 #308 (See https://builds.apache.org/job/HBase-0.98-on-Hadoop-1.1/308/ ) HBASE-10922 : Log splitting status should always be closed (jzhong: rev 0b2a5f5880ba8bb102efbb72e54970e88726d8f9) hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java
        Hide
        Hudson added a comment -

        SUCCESS: Integrated in HBase-0.98 #326 (See https://builds.apache.org/job/HBase-0.98/326/)
        HBASE-10922: Log splitting status should always be closed (jzhong: rev 0b2a5f5880ba8bb102efbb72e54970e88726d8f9)

        • hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java
        Show
        Hudson added a comment - SUCCESS: Integrated in HBase-0.98 #326 (See https://builds.apache.org/job/HBase-0.98/326/ ) HBASE-10922 : Log splitting status should always be closed (jzhong: rev 0b2a5f5880ba8bb102efbb72e54970e88726d8f9) hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java
        Hide
        Jeffrey Zhong added a comment -

        I've integrated the fix into master and 0.98 branches. Thanks Jimmy Xiang & Stack for the comments!

        Show
        Jeffrey Zhong added a comment - I've integrated the fix into master and 0.98 branches. Thanks Jimmy Xiang & Stack for the comments!
        Hide
        Jeffrey Zhong added a comment -

        Sure. I'll commit it today. Almost forgot this JIRA.

        Show
        Jeffrey Zhong added a comment - Sure. I'll commit it today. Almost forgot this JIRA.
        Hide
        Jimmy Xiang added a comment -

        Jeffrey Zhong, could you commit your patch? Thanks.

        Show
        Jimmy Xiang added a comment - Jeffrey Zhong , could you commit your patch? Thanks.
        Hide
        stack added a comment -

        Jimmy Xiang and Jeffrey Zhong It does not look like you fellas committed this though you agreed on the patch?

        Show
        stack added a comment - Jimmy Xiang and Jeffrey Zhong It does not look like you fellas committed this though you agreed on the patch?
        Hide
        Jimmy Xiang added a comment -

        +1

        Show
        Jimmy Xiang added a comment - +1
        Hide
        Jeffrey Zhong added a comment -

        Thanks Jimmy Xiang for the comments. I incorporated your feedbacks into the v2 patch.

        Show
        Jeffrey Zhong added a comment - Thanks Jimmy Xiang for the comments. I incorporated your feedbacks into the v2 patch.
        Hide
        Jimmy Xiang added a comment -

        Cool. Looks good. One question, progress_failed is set to false initially. If finishWritingAndClose throws any exception, we will log saying it not failed, right? Can we set progress_failed to true initially instead?

        I was also wondering if it is good to override progress_failed in the finally block. Should we use a different variable instead? I know this is some existing code already there for a while.

        Show
        Jimmy Xiang added a comment - Cool. Looks good. One question, progress_failed is set to false initially. If finishWritingAndClose throws any exception, we will log saying it not failed, right? Can we set progress_failed to true initially instead? I was also wondering if it is good to override progress_failed in the finally block. Should we use a different variable instead? I know this is some existing code already there for a while.
        Hide
        Jeffrey Zhong added a comment -

        Jimmy Xiang Here is that patch I'm thinking about. Thanks.

        Show
        Jeffrey Zhong added a comment - Jimmy Xiang Here is that patch I'm thinking about. Thanks.
        Hide
        Jimmy Xiang added a comment -

        Jeffrey Zhong, yes, I should have uploaded the whole log file. Thanks Stack put it up here. Would you like to take this issue since you'd like to enhance the status/markComplete too?

        stack, I meant this IOE:

        Caused by: org.apache.hadoop.hbase.ipc.RemoteWithExtrasException(org.apache.hadoop.hbase.NotServingRegionException): org.apache.hadoop.hbase.NotServingRegionException: Region d900a9b5ab41c96cf9ad0fb5f039fa8b is not online on e1418.halxg.cloudera.com,36020,1396841264248
                at org.apache.hadoop.hbase.regionserver.HRegionServer.getRegionByEncodedName(HRegionServer.java:2454)
                at org.apache.hadoop.hbase.regionserver.HRegionServer.getRegionByEncodedName(HRegionServer.java:2436)
                at org.apache.hadoop.hbase.regionserver.RSRpcServices.replay(RSRpcServices.java:1295)
        

        My understanding is that e1120 was splitting the log. However, it took some time and the task was taken by another worker. So the HLogSplitter wanted to error-out. In closing the output sink, it got this IOE so it didn't have chance to close the status. The IOE was logged in the caller: SplitLogWorker.

        Show
        Jimmy Xiang added a comment - Jeffrey Zhong , yes, I should have uploaded the whole log file. Thanks Stack put it up here. Would you like to take this issue since you'd like to enhance the status/markComplete too? stack , I meant this IOE: Caused by: org.apache.hadoop.hbase.ipc.RemoteWithExtrasException(org.apache.hadoop.hbase.NotServingRegionException): org.apache.hadoop.hbase.NotServingRegionException: Region d900a9b5ab41c96cf9ad0fb5f039fa8b is not online on e1418.halxg.cloudera.com,36020,1396841264248 at org.apache.hadoop.hbase.regionserver.HRegionServer.getRegionByEncodedName(HRegionServer.java:2454) at org.apache.hadoop.hbase.regionserver.HRegionServer.getRegionByEncodedName(HRegionServer.java:2436) at org.apache.hadoop.hbase.regionserver.RSRpcServices.replay(RSRpcServices.java:1295) My understanding is that e1120 was splitting the log. However, it took some time and the task was taken by another worker. So the HLogSplitter wanted to error-out. In closing the output sink, it got this IOE so it didn't have chance to close the status. The IOE was logged in the caller: SplitLogWorker.
        Hide
        stack added a comment -

        The first mention of the stuck log is because of:

        122816 2014-04-06 19:50:22,096 INFO [M_LOG_REPLAY_OPS-e1120:36020-9] master.SplitLogManager: started splitting 17 logs in [hdfs://e1120:35802/hbase/WALs/e1320.halxg.cloudera.com,36020,1396838053131-split ting]

        The task is acquired by e1121.halxg.cloudera.com,36020,1396838094934

        Over on e1121, I see the task acquired, the lease successfully obtained and then nothing else. We are silent hereafter on events related to the splitting of 1396838053131.1396838130980

        Over on master we wait and then resubmit:

        124174 2014-04-06 19:52:23,291 INFO [e1120.halxg.cloudera.com,36020,1396838229159.splitLogManagerTimeoutMonitor] master.SplitLogManager: resubmitting task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com% 2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980

        The log gets assigned to the master (combined master + regionserver).

        We then crash at Sun Apr 6 19:52:52 PDT 2014 Starting master on e1120.halxg.cloudera.com

        When master comes back up, it finds the orphan'd log splitting. Requeues the task. Complains about znode already existing. Then spends about tens of seconds note being acquired. Then gets resubmitted. Its not being split fast enough? Splitting is going really slow? Resubmitted a few times.

        Eventually we get this:

        2014-04-06 20:25:54,708 INFO [e1120.halxg.cloudera.com,36020,1396839174186.splitLogManagerTimeoutMonitor] master.SplitLogManager: Skipping resubmissions of task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 because threshold 3 reached

        Should this be WARN? Or ERROR?

        Seems like we keep going though, trying to split. Maybe the above means we won't retry against another server anymore? We seem to get stuck waiting on the one server to return.

        Is the IOE we're talking of above, this one:

        Caused by: org.apache.hadoop.hbase.ipc.RemoteWithExtrasException(org.apache.hadoop.hbase.NotServingRegionException): org.apache.hadoop.hbase.NotServingRegionException: Region d900a9b5ab41c96cf9ad0fb5f039fa8b is not online on e1418.halxg.cloudera.com,36020,1396841264248
                at org.apache.hadoop.hbase.regionserver.HRegionServer.getRegionByEncodedName(HRegionServer.java:2454)
                at org.apache.hadoop.hbase.regionserver.HRegionServer.getRegionByEncodedName(HRegionServer.java:2436)
                at org.apache.hadoop.hbase.regionserver.RSRpcServices.replay(RSRpcServices.java:1295)
                at org.apache.hadoop.hbase.protobuf.generated.AdminProtos$AdminService$2.callBlockingMethod(AdminProtos.java:20056)
                at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2002)
                at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:98)
                at org.apache.hadoop.hbase.ipc.SimpleRpcScheduler.consumerLoop(SimpleRpcScheduler.java:162)
                at org.apache.hadoop.hbase.ipc.SimpleRpcScheduler.access$000(SimpleRpcScheduler.java:38)
                at org.apache.hadoop.hbase.ipc.SimpleRpcScheduler$1.run(SimpleRpcScheduler.java:110)
                at java.lang.Thread.run(Thread.java:724)
        
                at org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:1499)
                at org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(RpcClient.java:1661)
                at org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImplementation.callBlockingMethod(RpcClient.java:1726)
                at org.apache.hadoop.hbase.protobuf.generated.AdminProtos$AdminService$BlockingStub.replay(AdminProtos.java:20926)
                at org.apache.hadoop.hbase.regionserver.wal.WALEditsReplaySink$ReplayServerCallable.replayToServer(WALEditsReplaySink.java:215)
                ... 10 more
        

        ... where the replay gets a NSRE?

        We keep retrying... I see one of these:

        2014-04-06 20:39:30,582 WARN  [RS_LOG_REPLAY_OPS-e1120:36020-1] regionserver.SplitLogWorker: BADVERSION failed to assert ownership for /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980
        org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = BadVersion for /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980
                at org.apache.zookeeper.KeeperException.create(KeeperException.java:115)
                at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
                at org.apache.zookeeper.ZooKeeper.setData(ZooKeeper.java:1266)
                at org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.setData(RecoverableZooKeeper.java:407)
                at org.apache.hadoop.hbase.regionserver.SplitLogWorker.attemptToOwnTask(SplitLogWorker.java:393)
                at org.apache.hadoop.hbase.regionserver.SplitLogWorker$2.progress(SplitLogWorker.java:460)
                at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter$OutputSink.finishWriting(HLogSplitter.java:954)
                at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter$LogReplayOutputSink.finishWritingAndClose(HLogSplitter.java:1683)
                at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFile(HLogSplitter.java:360)
                at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFile(HLogSplitter.java:224)
                at org.apache.hadoop.hbase.regionserver.SplitLogWorker$1.exec(SplitLogWorker.java:143)
                at org.apache.hadoop.hbase.regionserver.handler.HLogSplitterHandler.process(HLogSplitterHandler.java:79)
                at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:128)
                at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
                at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
                at java.lang.Thread.run(Thread.java:724)
        2014-04-06 20:39:30,583 WARN  [RS_LOG_REPLAY_OPS-e1120:36020-1] regionserver.SplitLogWorker: Failed to heartbeat the task/hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980
        2014-04-06 20:39:30,583 WARN  [RS_LOG_REPLAY_OPS-e1120:36020-1] regionserver.SplitLogWorker: log splitting of WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838130980 failed, returning error
        
        
        and this.....
        
        
        2014-04-06 20:39:30,590 WARN  [RS_LOG_REPLAY_OPS-e1120:36020-1] handler.HLogSplitterHandler: transisition task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 to ERR e1120.halxg.cloudera.com,36020,1396839174186 failed because of version mismatch
        org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = BadVersion for /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980
                at org.apache.zookeeper.KeeperException.create(KeeperException.java:115)
                at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
                at org.apache.zookeeper.ZooKeeper.setData(ZooKeeper.java:1266)
                at org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.setData(RecoverableZooKeeper.java:407)
                at org.apache.hadoop.hbase.zookeeper.ZKUtil.setData(ZKUtil.java:878)
                at org.apache.hadoop.hbase.regionserver.handler.HLogSplitterHandler.endTask(HLogSplitterHandler.java:122)
                at org.apache.hadoop.hbase.regionserver.handler.HLogSplitterHandler.process(HLogSplitterHandler.java:91)
                at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:128)
                at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
                at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
                at java.lang.Thread.run(Thread.java:724)
        
        ...
        
        2014-04-06 20:39:30,590 INFO  [RS_LOG_REPLAY_OPS-e1120:36020-1] handler.HLogSplitterHandler: worker e1120.halxg.cloudera.com,36020,1396839174186 done with task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 in 2788095ms
        

        This seems like an old running task because we are still trying against e1321.

        Server is stuck waiting on 4 tasks to complete including this particular log file.

        It eventually fails after 40 odd minutes on e1321

        2014-04-06 21:08:53,346 INFO [main-EventThread] master.SplitLogManager: task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 entered state: ERR e1321.halxg.cloudera.com,36020,1396841215304

        Then we are able to split it on the master.

        On e1321 I see this:

        Caused by: org.apache.hadoop.hbase.ipc.RemoteWithExtrasException(org.apache.hadoop.hbase.NotServingRegionException): org.apache.hadoop.hbase.NotServingRegionException: Region d900a9b5ab41c96cf9ad0fb5f039fa8b is not online on e1418.halxg.cloudera.com,36020,1396841264248

        .... just before the error out on log splitting. Maybe this is the cause of our erroring out? We are retrying this region over and over?

        d900a9b5ab41c96cf9ad0fb5f039fa8b is online but can see folks are confused about where it is.

        Does DLR update its location if it fails to find region in presumed location?

        Show
        stack added a comment - The first mention of the stuck log is because of: 122816 2014-04-06 19:50:22,096 INFO [M_LOG_REPLAY_OPS-e1120:36020-9] master.SplitLogManager: started splitting 17 logs in [hdfs://e1120:35802/hbase/WALs/e1320.halxg.cloudera.com,36020,1396838053131-split ting] The task is acquired by e1121.halxg.cloudera.com,36020,1396838094934 Over on e1121, I see the task acquired, the lease successfully obtained and then nothing else. We are silent hereafter on events related to the splitting of 1396838053131.1396838130980 Over on master we wait and then resubmit: 124174 2014-04-06 19:52:23,291 INFO [e1120.halxg.cloudera.com,36020,1396838229159.splitLogManagerTimeoutMonitor] master.SplitLogManager: resubmitting task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com% 2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 The log gets assigned to the master (combined master + regionserver). We then crash at Sun Apr 6 19:52:52 PDT 2014 Starting master on e1120.halxg.cloudera.com When master comes back up, it finds the orphan'd log splitting. Requeues the task. Complains about znode already existing. Then spends about tens of seconds note being acquired. Then gets resubmitted. Its not being split fast enough? Splitting is going really slow? Resubmitted a few times. Eventually we get this: 2014-04-06 20:25:54,708 INFO [e1120.halxg.cloudera.com,36020,1396839174186.splitLogManagerTimeoutMonitor] master.SplitLogManager: Skipping resubmissions of task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 because threshold 3 reached Should this be WARN? Or ERROR? Seems like we keep going though, trying to split. Maybe the above means we won't retry against another server anymore? We seem to get stuck waiting on the one server to return. Is the IOE we're talking of above, this one: Caused by: org.apache.hadoop.hbase.ipc.RemoteWithExtrasException(org.apache.hadoop.hbase.NotServingRegionException): org.apache.hadoop.hbase.NotServingRegionException: Region d900a9b5ab41c96cf9ad0fb5f039fa8b is not online on e1418.halxg.cloudera.com,36020,1396841264248 at org.apache.hadoop.hbase.regionserver.HRegionServer.getRegionByEncodedName(HRegionServer.java:2454) at org.apache.hadoop.hbase.regionserver.HRegionServer.getRegionByEncodedName(HRegionServer.java:2436) at org.apache.hadoop.hbase.regionserver.RSRpcServices.replay(RSRpcServices.java:1295) at org.apache.hadoop.hbase.protobuf.generated.AdminProtos$AdminService$2.callBlockingMethod(AdminProtos.java:20056) at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2002) at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:98) at org.apache.hadoop.hbase.ipc.SimpleRpcScheduler.consumerLoop(SimpleRpcScheduler.java:162) at org.apache.hadoop.hbase.ipc.SimpleRpcScheduler.access$000(SimpleRpcScheduler.java:38) at org.apache.hadoop.hbase.ipc.SimpleRpcScheduler$1.run(SimpleRpcScheduler.java:110) at java.lang. Thread .run( Thread .java:724) at org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:1499) at org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(RpcClient.java:1661) at org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImplementation.callBlockingMethod(RpcClient.java:1726) at org.apache.hadoop.hbase.protobuf.generated.AdminProtos$AdminService$BlockingStub.replay(AdminProtos.java:20926) at org.apache.hadoop.hbase.regionserver.wal.WALEditsReplaySink$ReplayServerCallable.replayToServer(WALEditsReplaySink.java:215) ... 10 more ... where the replay gets a NSRE? We keep retrying... I see one of these: 2014-04-06 20:39:30,582 WARN [RS_LOG_REPLAY_OPS-e1120:36020-1] regionserver.SplitLogWorker: BADVERSION failed to assert ownership for /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = BadVersion for /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 at org.apache.zookeeper.KeeperException.create(KeeperException.java:115) at org.apache.zookeeper.KeeperException.create(KeeperException.java:51) at org.apache.zookeeper.ZooKeeper.setData(ZooKeeper.java:1266) at org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.setData(RecoverableZooKeeper.java:407) at org.apache.hadoop.hbase.regionserver.SplitLogWorker.attemptToOwnTask(SplitLogWorker.java:393) at org.apache.hadoop.hbase.regionserver.SplitLogWorker$2.progress(SplitLogWorker.java:460) at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter$OutputSink.finishWriting(HLogSplitter.java:954) at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter$LogReplayOutputSink.finishWritingAndClose(HLogSplitter.java:1683) at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFile(HLogSplitter.java:360) at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFile(HLogSplitter.java:224) at org.apache.hadoop.hbase.regionserver.SplitLogWorker$1.exec(SplitLogWorker.java:143) at org.apache.hadoop.hbase.regionserver.handler.HLogSplitterHandler.process(HLogSplitterHandler.java:79) at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:128) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang. Thread .run( Thread .java:724) 2014-04-06 20:39:30,583 WARN [RS_LOG_REPLAY_OPS-e1120:36020-1] regionserver.SplitLogWorker: Failed to heartbeat the task/hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 2014-04-06 20:39:30,583 WARN [RS_LOG_REPLAY_OPS-e1120:36020-1] regionserver.SplitLogWorker: log splitting of WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838130980 failed, returning error and this ..... 2014-04-06 20:39:30,590 WARN [RS_LOG_REPLAY_OPS-e1120:36020-1] handler.HLogSplitterHandler: transisition task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 to ERR e1120.halxg.cloudera.com,36020,1396839174186 failed because of version mismatch org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = BadVersion for /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 at org.apache.zookeeper.KeeperException.create(KeeperException.java:115) at org.apache.zookeeper.KeeperException.create(KeeperException.java:51) at org.apache.zookeeper.ZooKeeper.setData(ZooKeeper.java:1266) at org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.setData(RecoverableZooKeeper.java:407) at org.apache.hadoop.hbase.zookeeper.ZKUtil.setData(ZKUtil.java:878) at org.apache.hadoop.hbase.regionserver.handler.HLogSplitterHandler.endTask(HLogSplitterHandler.java:122) at org.apache.hadoop.hbase.regionserver.handler.HLogSplitterHandler.process(HLogSplitterHandler.java:91) at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:128) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang. Thread .run( Thread .java:724) ... 2014-04-06 20:39:30,590 INFO [RS_LOG_REPLAY_OPS-e1120:36020-1] handler.HLogSplitterHandler: worker e1120.halxg.cloudera.com,36020,1396839174186 done with task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 in 2788095ms This seems like an old running task because we are still trying against e1321. Server is stuck waiting on 4 tasks to complete including this particular log file. It eventually fails after 40 odd minutes on e1321 2014-04-06 21:08:53,346 INFO [main-EventThread] master.SplitLogManager: task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 entered state: ERR e1321.halxg.cloudera.com,36020,1396841215304 Then we are able to split it on the master. On e1321 I see this: Caused by: org.apache.hadoop.hbase.ipc.RemoteWithExtrasException(org.apache.hadoop.hbase.NotServingRegionException): org.apache.hadoop.hbase.NotServingRegionException: Region d900a9b5ab41c96cf9ad0fb5f039fa8b is not online on e1418.halxg.cloudera.com,36020,1396841264248 .... just before the error out on log splitting. Maybe this is the cause of our erroring out? We are retrying this region over and over? d900a9b5ab41c96cf9ad0fb5f039fa8b is online but can see folks are confused about where it is. Does DLR update its location if it fails to find region in presumed location?
        Hide
        stack added a comment -

        Here is master log. Has the first time *C1396838053131.1396838130980 is mentioned up to the last time when it is cleared because successfully replayed.

        Show
        stack added a comment - Here is master log. Has the first time *C1396838053131.1396838130980 is mentioned up to the last time when it is cleared because successfully replayed.
        Hide
        stack added a comment -

        Jimmy Xiang The IOE is the one thrown because there are still files under the splitting dir? Or is it another IOE you are referencing?

        This one?

            for (Path logDir: logDirs){
              status.setStatus("Cleaning up log directory...");
              try {
                if (fs.exists(logDir) && !fs.delete(logDir, false)) {
                  LOG.warn("Unable to delete log src dir. Ignoring. " + logDir);
                }
              } catch (IOException ioe) {
                FileStatus[] files = fs.listStatus(logDir);
                if (files != null && files.length > 0) {
                  LOG.warn("Returning success without actually splitting and " + 
                      "deleting all the log files in path " + logDir);
                } else {
                  LOG.warn("Unable to delete log src dir. Ignoring. " + logDir, ioe);
                }
              }
        

        Or another one?

        The log was failing to split before the master restarted at around 19:52. Was an IOE thrown then?

        Good stuff.

        Show
        stack added a comment - Jimmy Xiang The IOE is the one thrown because there are still files under the splitting dir? Or is it another IOE you are referencing? This one? for (Path logDir: logDirs){ status.setStatus( "Cleaning up log directory..." ); try { if (fs.exists(logDir) && !fs.delete(logDir, false )) { LOG.warn( "Unable to delete log src dir. Ignoring. " + logDir); } } catch (IOException ioe) { FileStatus[] files = fs.listStatus(logDir); if (files != null && files.length > 0) { LOG.warn( "Returning success without actually splitting and " + "deleting all the log files in path " + logDir); } else { LOG.warn( "Unable to delete log src dir. Ignoring. " + logDir, ioe); } } Or another one? The log was failing to split before the master restarted at around 19:52. Was an IOE thrown then? Good stuff.
        Hide
        Jeffrey Zhong added a comment -

        Good findings! You should post the full log so that I could have spotted it earlier. Since you'll touch this code, I saw below code in the same function. In which case, we will call markComplete twice(plus the one in finally block). I think we should remove this markComplete call and add more status info into the markComplete in the finally block. Thanks.

              if (in == null) {
                status.markComplete("Was nothing to split in log file");
                LOG.warn("Nothing to split in log file " + logPath);
                return true;
              }
        
        Show
        Jeffrey Zhong added a comment - Good findings! You should post the full log so that I could have spotted it earlier . Since you'll touch this code, I saw below code in the same function. In which case, we will call markComplete twice(plus the one in finally block). I think we should remove this markComplete call and add more status info into the markComplete in the finally block. Thanks. if (in == null ) { status.markComplete( "Was nothing to split in log file" ); LOG.warn( "Nothing to split in log file " + logPath); return true ; }
        Hide
        Jimmy Xiang added a comment -

        I think I found the cause:

              if (outputSinkStarted) {
                progress_failed = outputSink.finishWritingAndClose() == null;
              }
        

        It throws an IOException and the status is not closed. Let me post a patch.

        Show
        Jimmy Xiang added a comment - I think I found the cause: if (outputSinkStarted) { progress_failed = outputSink.finishWritingAndClose() == null ; } It throws an IOException and the status is not closed. Let me post a patch.
        Hide
        Jimmy Xiang added a comment -

        There is no time zone issue I think. The time looks fine to me. I started the test yesterday. So the time is about right.
        The UI page is still up and refreshing. Now it shows the job stuck there for 19+ hours.

        Show
        Jimmy Xiang added a comment - There is no time zone issue I think. The time looks fine to me. I started the test yesterday. So the time is about right. The UI page is still up and refreshing. Now it shows the job stuck there for 19+ hours.
        Hide
        Jeffrey Zhong added a comment -

        Is this UI refresh problem or time zone issue? I saw the following log lines:

        hbase-jxiang-master-e1120.halxg.cloudera.com.log:2014-04-06 19:52:23,358 INFO  [RS_LOG_REPLAY_OPS-e1120:36020-0] wal.HLogSplitter: Splitting hlog: hdfs://e1120:35802/hbase/WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838130980, length=133872798
        ....
        hbase-jxiang-master-e1120.halxg.cloudera.com.log:2014-04-06 21:28:12,172 INFO  [RS_LOG_REPLAY_OPS-e1120:36020-1] wal.HLogSplitter: Processed 16 edits across 1 regions; log file=hdfs://e1120:35802/hbase/WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838130980 is corrupted = false progress failed = false
        

        The log splitting happened from 19:52:23 and complete on 21:28:12 due to there are some errors between.

        hbase-jxiang-master-e1120.halxg.cloudera.com.log:2014-04-06 20:24:28,634 INFO  [RS_LOG_REPLAY_OPS-e1120:36020-0] wal.HLogSplitter: Splitting hlog: hdfs://e1120:35802/hbase/WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838124343, length=133855324
        ...
        hbase-jxiang-master-e1120.halxg.cloudera.com.log:2014-04-06 20:39:30,583 WARN  [RS_LOG_REPLAY_OPS-e1120:36020-1] regionserver.SplitLogWorker: log splitting of WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838130980 failed, returning error
        hbase-jxiang-master-e1120.halxg.cloudera.com.log:2014-04-06 21:05:30,113 WARN  [RS_LOG_REPLAY_OPS-e1120:36020-0] regionserver.SplitLogWorker: log splitting of WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838124343 failed, returning error
        hbase-jxiang-master-e1120.halxg.cloudera.com.log:2014-04-06 21:28:09,619 INFO  [main-EventThread] wal.HLogSplitter: Archived processed log hdfs://e1120:35802/hbase/WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838124343 to hdfs://e1120:35802/hbase/oldWALs/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838124343
        

        The above log splitting is similar started from 20:24:28 and completed until 21:05:30. There are several errors. From the code in HLogSplitter#splitLogFile, the status.markComplete is in finally block. So we should see the status updates at least several times.

         status.markComplete(msg);
        
        Show
        Jeffrey Zhong added a comment - Is this UI refresh problem or time zone issue? I saw the following log lines: hbase-jxiang-master-e1120.halxg.cloudera.com.log:2014-04-06 19:52:23,358 INFO [RS_LOG_REPLAY_OPS-e1120:36020-0] wal.HLogSplitter: Splitting hlog: hdfs://e1120:35802/hbase/WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838130980, length=133872798 .... hbase-jxiang-master-e1120.halxg.cloudera.com.log:2014-04-06 21:28:12,172 INFO [RS_LOG_REPLAY_OPS-e1120:36020-1] wal.HLogSplitter: Processed 16 edits across 1 regions; log file=hdfs://e1120:35802/hbase/WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838130980 is corrupted = false progress failed = false The log splitting happened from 19:52:23 and complete on 21:28:12 due to there are some errors between. hbase-jxiang-master-e1120.halxg.cloudera.com.log:2014-04-06 20:24:28,634 INFO [RS_LOG_REPLAY_OPS-e1120:36020-0] wal.HLogSplitter: Splitting hlog: hdfs://e1120:35802/hbase/WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838124343, length=133855324 ... hbase-jxiang-master-e1120.halxg.cloudera.com.log:2014-04-06 20:39:30,583 WARN [RS_LOG_REPLAY_OPS-e1120:36020-1] regionserver.SplitLogWorker: log splitting of WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838130980 failed, returning error hbase-jxiang-master-e1120.halxg.cloudera.com.log:2014-04-06 21:05:30,113 WARN [RS_LOG_REPLAY_OPS-e1120:36020-0] regionserver.SplitLogWorker: log splitting of WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838124343 failed, returning error hbase-jxiang-master-e1120.halxg.cloudera.com.log:2014-04-06 21:28:09,619 INFO [main-EventThread] wal.HLogSplitter: Archived processed log hdfs://e1120:35802/hbase/WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838124343 to hdfs://e1120:35802/hbase/oldWALs/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838124343 The above log splitting is similar started from 20:24:28 and completed until 21:05:30. There are several errors. From the code in HLogSplitter#splitLogFile, the status.markComplete is in finally block. So we should see the status updates at least several times. status.markComplete(msg);
        Hide
        stack added a comment -

        The log is eventually split about 95 minutes later:

        2014-04-06 21:28:12,180 INFO  [RS_LOG_REPLAY_OPS-e1120:36020-1] handler.HLogSplitterHandler: successfully transitioned task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 to final state DONE e1120.halxg.cloudera.com,36020,1396839174186
        2014-04-06 21:28:12,180 INFO  [RS_LOG_REPLAY_OPS-e1120:36020-1] handler.HLogSplitterHandler: worker e1120.halxg.cloudera.com,36020,1396839174186 done with task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 in 1891ms
        2014-04-06 21:28:12,181 INFO  [main-EventThread] master.SplitLogManager: task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 entered state: DONE e1120.halxg.cloudera.com,36020,1396839174186
        2014-04-06 21:28:12,193 INFO  [main-EventThread] wal.HLogSplitter: Archived processed log hdfs://e1120:35802/hbase/WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838130980 to hdfs://e1120:35802/hbase/oldWALs/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838130980
        2014-04-06 21:28:12,194 INFO  [main-EventThread] master.SplitLogManager: Done splitting /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980
        

        TODO: Why is task not removed from the UI list-of-tasks?

        The master is restarted around 19:52. We have been having trouble w/ this log before the restart.

        Here is some history on this log:

        2014-04-06 19:50:22,222 DEBUG [main-EventThread] master.SplitLogManager: put up splitlog task at znode /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980
        ...
        2014-04-06 19:50:22,280 INFO  [main-EventThread] master.SplitLogManager: task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 acquired by e1121.halxg.cloudera.com,36020,1396838094934
        ...
        
        

        It is acquired by a machine and it starts to work but then no more...

        2014-04-06 19:50:22,305 INFO [RS_LOG_REPLAY_OPS-e1121:36020-0] util.FSHDFSUtils: recoverLease=true, attempt=0 on file=hdfs://e1120:35802/hbase/WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838130980 after 1ms

        ... I don't see the "Processed N edits ..." for this file. I see it for others.

        We then 'resubmit' the task on the master:

        2014-04-06 19:52:23,291 INFO  [e1120.halxg.cloudera.com,36020,1396838229159.splitLogManagerTimeoutMonitor] master.SplitLogManager: resubmitting task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980
        

        Master dies. Comes back. We find the outstanding 'orphan' task. Requeue a splitting task. We are waiting on it to complete:

        2014-04-06 19:53:03,579 DEBUG [M_LOG_REPLAY_OPS-e1120:36020-2] master.SplitLogManager: Previously orphan task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 is now being waited upon
        

        Then weirdly, we 'find' preexisting node and put up new task:

        2014-04-06 19:53:03,592 DEBUG [main-EventThread] master.SplitLogManager$CreateAsyncCallback: found pre-existing znode /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980
        2014-04-06 19:53:03,593 DEBUG [main-EventThread] master.SplitLogManager: put up splitlog task at znode /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980
        

        Then resubmit tasks like we are not making progress.

        After the last restart, the SplitLogManager is stuck outputting every 5 seconds its status.... that there are 4 tasks being split.

        Finally, I see this:

        2014-04-06 21:08:53,346 INFO [main-EventThread] master.SplitLogManager: task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 entered state: ERR e1321.halxg.cloudera.com,36020,1396841215304

        And then the log is successfully split. This is a few hours later.

        Here is what it has over on the slave worker node:

        2014-04-06 20:26:57,685 INFO  [SplitLogWorker-e1321.halxg.cloudera.com,36020,1396841215304] regionserver.SplitLogWorker: worker e1321.halxg.cloudera.com,36020,1396841215304 acquired task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980
        ....
        2014-04-06 21:08:53,370 INFO  [RS_LOG_REPLAY_OPS-e1321:36020-0] handler.HLogSplitterHandler: successfully transitioned task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 to final state ERR e1321.halxg.cloudera.com,36020,1396841215304
        2014-04-06 21:08:53,370 INFO  [RS_LOG_REPLAY_OPS-e1321:36020-0] handler.HLogSplitterHandler: worker e1321.halxg.cloudera.com,36020,1396841215304 done with task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 in 2515676ms
        

        It took 42 minutes to fail the split.

        On this retry, the replay actually worked.

        I see a bunch of ERR fails over in a slave log. I wonder if it is an unassigned region? I don't see errors. I see unassigned region errors but they look legit.

        Any input Jeffrey Zhong? Thanks boss.

        Show
        stack added a comment - The log is eventually split about 95 minutes later: 2014-04-06 21:28:12,180 INFO [RS_LOG_REPLAY_OPS-e1120:36020-1] handler.HLogSplitterHandler: successfully transitioned task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 to final state DONE e1120.halxg.cloudera.com,36020,1396839174186 2014-04-06 21:28:12,180 INFO [RS_LOG_REPLAY_OPS-e1120:36020-1] handler.HLogSplitterHandler: worker e1120.halxg.cloudera.com,36020,1396839174186 done with task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 in 1891ms 2014-04-06 21:28:12,181 INFO [main-EventThread] master.SplitLogManager: task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 entered state: DONE e1120.halxg.cloudera.com,36020,1396839174186 2014-04-06 21:28:12,193 INFO [main-EventThread] wal.HLogSplitter: Archived processed log hdfs: //e1120:35802/hbase/WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838130980 to hdfs://e1120:35802/hbase/oldWALs/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838130980 2014-04-06 21:28:12,194 INFO [main-EventThread] master.SplitLogManager: Done splitting /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 TODO: Why is task not removed from the UI list-of-tasks? The master is restarted around 19:52. We have been having trouble w/ this log before the restart. Here is some history on this log: 2014-04-06 19:50:22,222 DEBUG [main-EventThread] master.SplitLogManager: put up splitlog task at znode /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 ... 2014-04-06 19:50:22,280 INFO [main-EventThread] master.SplitLogManager: task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 acquired by e1121.halxg.cloudera.com,36020,1396838094934 ... It is acquired by a machine and it starts to work but then no more... 2014-04-06 19:50:22,305 INFO [RS_LOG_REPLAY_OPS-e1121:36020-0] util.FSHDFSUtils: recoverLease=true, attempt=0 on file=hdfs://e1120:35802/hbase/WALs/e1320.halxg.cloudera.com,36020,1396838053131-splitting/e1320.halxg.cloudera.com%2C36020%2C1396838053131.1396838130980 after 1ms ... I don't see the "Processed N edits ..." for this file. I see it for others. We then 'resubmit' the task on the master: 2014-04-06 19:52:23,291 INFO [e1120.halxg.cloudera.com,36020,1396838229159.splitLogManagerTimeoutMonitor] master.SplitLogManager: resubmitting task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 Master dies. Comes back. We find the outstanding 'orphan' task. Requeue a splitting task. We are waiting on it to complete: 2014-04-06 19:53:03,579 DEBUG [M_LOG_REPLAY_OPS-e1120:36020-2] master.SplitLogManager: Previously orphan task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 is now being waited upon Then weirdly, we 'find' preexisting node and put up new task: 2014-04-06 19:53:03,592 DEBUG [main-EventThread] master.SplitLogManager$CreateAsyncCallback: found pre-existing znode /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 2014-04-06 19:53:03,593 DEBUG [main-EventThread] master.SplitLogManager: put up splitlog task at znode /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 Then resubmit tasks like we are not making progress. After the last restart, the SplitLogManager is stuck outputting every 5 seconds its status.... that there are 4 tasks being split. Finally, I see this: 2014-04-06 21:08:53,346 INFO [main-EventThread] master.SplitLogManager: task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 entered state: ERR e1321.halxg.cloudera.com,36020,1396841215304 And then the log is successfully split. This is a few hours later. Here is what it has over on the slave worker node: 2014-04-06 20:26:57,685 INFO [SplitLogWorker-e1321.halxg.cloudera.com,36020,1396841215304] regionserver.SplitLogWorker: worker e1321.halxg.cloudera.com,36020,1396841215304 acquired task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 .... 2014-04-06 21:08:53,370 INFO [RS_LOG_REPLAY_OPS-e1321:36020-0] handler.HLogSplitterHandler: successfully transitioned task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 to final state ERR e1321.halxg.cloudera.com,36020,1396841215304 2014-04-06 21:08:53,370 INFO [RS_LOG_REPLAY_OPS-e1321:36020-0] handler.HLogSplitterHandler: worker e1321.halxg.cloudera.com,36020,1396841215304 done with task /hbase/splitWAL/WALs%2Fe1320.halxg.cloudera.com%2C36020%2C1396838053131-splitting%2Fe1320.halxg.cloudera.com%252C36020%252C1396838053131.1396838130980 in 2515676ms It took 42 minutes to fail the split. On this retry, the replay actually worked. I see a bunch of ERR fails over in a slave log. I wonder if it is an unassigned region? I don't see errors. I see unassigned region errors but they look legit. Any input Jeffrey Zhong ? Thanks boss.
        Hide
        Jimmy Xiang added a comment -

        In ZK, there is no splitting task pending. Could it be the splitting is really done, but the status of some failed try is never cleaned up?

        Show
        Jimmy Xiang added a comment - In ZK, there is no splitting task pending. Could it be the splitting is really done, but the status of some failed try is never cleaned up?
        Hide
        stack added a comment -

        I'm taking a look. Interesting is we think the splitting is 'done' but we still have outstanding tasks. The tasks are running on the master. Must be some issue w/ accounting of tasks messed up by the fact that a split worker is on the master itself... looking.

        Show
        stack added a comment - I'm taking a look. Interesting is we think the splitting is 'done' but we still have outstanding tasks. The tasks are running on the master. Must be some issue w/ accounting of tasks messed up by the fact that a split worker is on the master itself... looking.
        Hide
        Jimmy Xiang added a comment -

        Attached the screen shot of the master web UI.

        Show
        Jimmy Xiang added a comment - Attached the screen shot of the master web UI.

          People

          • Assignee:
            Jeffrey Zhong
            Reporter:
            Jimmy Xiang
          • Votes:
            0 Vote for this issue
            Watchers:
            7 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development