Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-5079

DistributedLogSplitter interrupt can be hazardous to regionserver health

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Not A Problem
    • None
    • None
    • None

    Description

      The DLS interrupt can kill the regionserver if happens while conversation w/ namenode is going on.

      The interrupt is used to end a task on regionserver when done whether successful or to interrupt an ongoing split since assumed by another server.

      I saw this issue testing because I was killing servers. I also was suffering "HBASE-5078 DistributedLogSplitter failing to split file because it has edits for lots of regions" which made it more likely to happen.

      Here is what it looks like on the regionserver that died:

      2011-12-20 17:54:58,009 INFO org.apache.hadoop.hbase.regionserver.SplitLogWorker: task /hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A7000%2Fhbase%2F.logs%2Fsv4r31s44%2C7003%2C1324365396770-splitting%2Fsv4r31s44%252C7003%252C1324365396770.1324403495463 preempted from sv4r13s38,7003,1324365396583, current task state and owner=owned sv4r27s44,7003,1324365396664
      2011-12-20 17:54:58,009 INFO org.apache.hadoop.hbase.regionserver.SplitLogWorker: Sending interrupt to stop the worker thread
      2011-12-20 17:54:59,133 INFO org.apache.hadoop.hbase.regionserver.SplitLogWorker: task /hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A7000%2Fhbase%2F.logs%2Fsv4r31s44%2C7003%2C1324365396770-splitting%2Fsv4r31s44%252C7003%252C1324365396770.1324403495463 preempted from sv4r13s38,7003,1324365396583, current task state and owner=owned sv4r27s44,7003,1324365396664
      2011-12-20 17:54:59,134 INFO org.apache.hadoop.hbase.regionserver.SplitLogWorker: Sending interrupt to stop the worker thread
      ...
      2011-12-20 17:55:25,505 INFO org.apache.hadoop.hbase.regionserver.SplitLogWorker: task /hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A7000%2Fhbase%2F.logs%2Fsv4r31s44%2C7003%2C1324365396770-splitting%2Fsv4r31s44%252C7003%252C1324365396770.1324403495463 preempted from sv4r13s38,7003,1324365396583, current task state and owner=unassigned sv4r11s38,7001,1324365395047
      2011-12-20 17:55:25,505 INFO org.apache.hadoop.hbase.regionserver.SplitLogWorker: Sending interrupt to stop the worker thread
      

      Three interrupts are sent over period of 31 seconds or so.

      Eventually the interrupt has an effect and I get:

      2011-12-20 17:55:25,505 INFO org.apache.hadoop.hbase.regionserver.SplitLogWorker: Sending interrupt to stop the worker thread
      2011-12-20 17:55:48,022 DEBUG org.apache.hadoop.hbase.regionserver.LogRoller: HLog roll requested
      2011-12-20 17:55:58,070 WARN org.apache.hadoop.hdfs.DFSClient: DataStreamer Exception: java.io.IOException: Call to sv4r11s38/10.4.11.38:7000 failed on local exception: java.nio.channels.ClosedByInterruptException
              at org.apache.hadoop.ipc.Client.wrapException(Client.java:1103)
              at org.apache.hadoop.ipc.Client.call(Client.java:1071)
              at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:225)
              at $Proxy9.addBlock(Unknown Source)
              at sun.reflect.GeneratedMethodAccessor37.invoke(Unknown Source)
              at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
              at java.lang.reflect.Method.invoke(Method.java:597)
              at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82)
              at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59)
              at $Proxy9.addBlock(Unknown Source)
              at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.locateFollowingBlock(DFSClient.java:3507)
              at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(DFSClient.java:3370)
              at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2700(DFSClient.java:2586)
              at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2826)
      Caused by: java.nio.channels.ClosedByInterruptException
              at java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:184)
              at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:341)
              at org.apache.hadoop.net.SocketOutputStream$Writer.performIO(SocketOutputStream.java:55)
              at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:142)
              at org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:146)
              at org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:107)
              at java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:65)
              at java.io.BufferedOutputStream.flush(BufferedOutputStream.java:123)
              at java.io.DataOutputStream.flush(DataOutputStream.java:106)
              at org.apache.hadoop.ipc.Client$Connection.sendParam(Client.java:779)
              at org.apache.hadoop.ipc.Client.call(Client.java:1047)
              at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:225)
              at $Proxy9.getFileInfo(Unknown Source)
              at sun.reflect.GeneratedMethodAccessor28.invoke(Unknown Source)
              at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
              at java.lang.reflect.Method.invoke(Method.java:597)
              at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82)
              at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59)
              at $Proxy9.getFileInfo(Unknown Source)
              at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:875)
              at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:513)
              at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:768)
              at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.convertRegionEditsToTemp(HLogSplitter.java:1097)
              at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.createWAP(HLogSplitter.java:1066)
              at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.convertRegionEditsToTemp(HLogSplitter.java:1097)
              at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.createWAP(HLogSplitter.java:1066)
              at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HLogSplitter.java:410)
              at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HLogSplitter.java:351)
              at org.apache.hadoop.hbase.regionserver.SplitLogWorker$1.exec(SplitLogWorker.java: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)
      2011-12-20 17:55:58,070 WARN org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Could not prepare temp staging area
      java.io.IOException: Call to sv4r11s38/10.4.11.38:7000 failed on local exception: java.nio.channels.ClosedByInterruptException
              at org.apache.hadoop.ipc.Client.wrapException(Client.java:1103)
              at org.apache.hadoop.ipc.Client.call(Client.java:1071)
              at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:225)
              at $Proxy9.getFileInfo(Unknown Source)
              at sun.reflect.GeneratedMethodAccessor28.invoke(Unknown Source)
              at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
              at java.lang.reflect.Method.invoke(Method.java:597)
              at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82)
              at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59)
              at $Proxy9.getFileInfo(Unknown Source)
              at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:875)
              at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:513)
              at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:768)
              at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.convertRegionEditsToTemp(HLogSplitter.java:1097)
              at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.createWAP(HLogSplitter.java:1066)
              at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HLogSplitter.java:410)
              at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HLogSplitter.java:351)
              at org.apache.hadoop.hbase.regionserver.SplitLogWorker$1.exec(SplitLogWorker.java: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)
      Caused by: java.nio.channels.ClosedByInterruptException
              at java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:184)
              at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:341)
              at org.apache.hadoop.net.SocketOutputStream$Writer.performIO(SocketOutputStream.java:55)
              at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:142)
              at org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:146)
              at org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:107)
              at java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:65)
              at java.io.BufferedOutputStream.flush(BufferedOutputStream.java:123)
              at java.io.DataOutputStream.flush(DataOutputStream.java:106)
              at org.apache.hadoop.ipc.Client$Connection.sendParam(Client.java:779)
              at org.apache.hadoop.ipc.Client.call(Client.java:1047)
              ... 20 more
      

      Now here is the wacky part. Above we are trying to go to the namenode it looks like and the interrupt is closing socket. At same, time, I'm trying to flush and it fails with same stack trace ... and because we failed a flush, regionserver goes down:

      2011-12-20 17:55:58,071 WARN org.apache.hadoop.hdfs.DFSClient: Error Recovery for block blk_-6239131583587622790_187561 bad datanode[0] nodes == null
      2011-12-20 17:55:58,073 WARN org.apache.hadoop.hdfs.DFSClient: Could not get block locations. Source file "/hbase/TestTable/9f98e6764d322832c845b740336e5750/.tmp/0df1673c96274028bda24c9cb49e9c3e" - Aborting...
      2011-12-20 17:55:58,074 FATAL org.apache.hadoop.hbase.regionserver.HRegionServer: ABORTING region server sv4r13s38,7003,1324365396583: Replay of HLog required. Forcing server shutdown
      org.apache.hadoop.hbase.DroppedSnapshotException: region: TestTable,0134394898,1323822783216.9f98e6764d322832c845b740336e5750.
              at org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:1276)
              at org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:1160)
              at org.apache.hadoop.hbase.regionserver.HRegion.flushcache(HRegion.java:1102)
              at org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushRegion(MemStoreFlusher.java:400) 
              at org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushRegion(MemStoreFlusher.java:374)
              at org.apache.hadoop.hbase.regionserver.MemStoreFlusher.run(MemStoreFlusher.java:243)
              at java.lang.Thread.run(Thread.java:662)
      Caused by: java.io.IOException: Call to sv4r11s38/10.4.11.38:7000 failed on local exception: java.nio.channels.ClosedByInterruptException
              at org.apache.hadoop.ipc.Client.wrapException(Client.java:1103)
              at org.apache.hadoop.ipc.Client.call(Client.java:1071)
              at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:225)
              at $Proxy9.addBlock(Unknown Source)
              at sun.reflect.GeneratedMethodAccessor37.invoke(Unknown Source)
              at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
              at java.lang.reflect.Method.invoke(Method.java:597)
              at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82)
              at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59)
              at $Proxy9.addBlock(Unknown Source)
              at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.locateFollowingBlock(DFSClient.java:3507)
              at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(DFSClient.java:3370)
              at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2700(DFSClient.java:2586)
              at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2826)
      Caused by: java.nio.channels.ClosedByInterruptException
              at java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:184)
              at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:341)
              at org.apache.hadoop.net.SocketOutputStream$Writer.performIO(SocketOutputStream.java:55)
              at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:142)
              at org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:146)
              at org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:107)
              at java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:65)
              at java.io.BufferedOutputStream.flush(BufferedOutputStream.java:123)
              at java.io.DataOutputStream.flush(DataOutputStream.java:106)
              at org.apache.hadoop.ipc.Client$Connection.sendParam(Client.java:779)
              at org.apache.hadoop.ipc.Client.call(Client.java:1047)
              at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:225)
              at $Proxy9.getFileInfo(Unknown Source)
              at sun.reflect.GeneratedMethodAccessor28.invoke(Unknown Source)
              at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
              at java.lang.reflect.Method.invoke(Method.java:597)
              at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82)
              at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59)
              at $Proxy9.getFileInfo(Unknown Source)
      ...
      

      Attachments

        Activity

          People

            Unassigned Unassigned
            stack Michael Stack
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: