Uploaded image for project: 'Accumulo'
  1. Accumulo
  2. ACCUMULO-4405

GC collection cycle stuck on waitForFlush RPC to Master

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.7.1, 1.7.2
    • 1.7.3, 1.8.0
    • gc, master
    • None

    Description

      While testing out 1.8.0rc1, all of the TabletServers had crashed due to an OOME, I believe, because I temporarily ran out of HDFS space because HDFS trash was enabled (trash could not be cleaned up fast enough for Accumulo generating more trash). I came back to the system after restarting the TabletServers and found that the GC had not run a new cycle after restarting the TabletServers. In a jstack of the GC, I saw:

      "gc" #13 prio=5 os_prio=0 tid=0x00000000021f3800 nid=0x4dd5 runnable [0x00007f6f1ebc0000]
         java.lang.Thread.State: RUNNABLE
              at java.net.SocketInputStream.socketRead0(Native Method)
              at java.net.SocketInputStream.socketRead(SocketInputStream.java:116)
              at java.net.SocketInputStream.read(SocketInputStream.java:170)
              at java.net.SocketInputStream.read(SocketInputStream.java:141)
              at java.io.BufferedInputStream.fill(BufferedInputStream.java:246)
              at java.io.BufferedInputStream.read1(BufferedInputStream.java:286)
              at java.io.BufferedInputStream.read(BufferedInputStream.java:345)
              - locked <0x00000000f5b4b750> (a java.io.BufferedInputStream)
              at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:127)
              at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
              at org.apache.thrift.transport.TFramedTransport.readFrame(TFramedTransport.java:129)
              at org.apache.thrift.transport.TFramedTransport.read(TFramedTransport.java:101)
              at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
              at org.apache.accumulo.core.client.impl.ThriftTransportPool$CachedTTransport.readAll(ThriftTransportPool.java:270)
              at org.apache.thrift.protocol.TCompactProtocol.readByte(TCompactProtocol.java:634)
              at org.apache.thrift.protocol.TCompactProtocol.readMessageBegin(TCompactProtocol.java:501)
              at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:77)
              at org.apache.accumulo.core.master.thrift.MasterClientService$Client.recv_waitForFlush(MasterClientService.java:209)
              at org.apache.accumulo.core.master.thrift.MasterClientService$Client.waitForFlush(MasterClientService.java:190)
              at org.apache.accumulo.core.client.impl.TableOperationsImpl._flush(TableOperationsImpl.java:820)
              at org.apache.accumulo.core.client.impl.TableOperationsImpl.compact(TableOperationsImpl.java:758)
              at org.apache.accumulo.core.client.impl.TableOperationsImpl.compact(TableOperationsImpl.java:727)
              at org.apache.accumulo.core.client.impl.TableOperationsImpl.compact(TableOperationsImpl.java:721)
              at org.apache.accumulo.gc.SimpleGarbageCollector.run(SimpleGarbageCollector.java:592)
              at org.apache.accumulo.gc.SimpleGarbageCollector.main(SimpleGarbageCollector.java:160)
              at org.apache.accumulo.gc.GCExecutable.execute(GCExecutable.java:34)
              at org.apache.accumulo.start.Main$1.run(Main.java:120)
              at java.lang.Thread.run(Thread.java:745)
      
         Locked ownable synchronizers:
              - None
      

      The Master was also stuck with an active thread/RPC:

      "ClientPool 23257" #45412 daemon prio=5 os_prio=0 tid=0x00000000049cc000 nid=0x6401 waiting on condition [0x00007f8462d94000]
         java.lang.Thread.State: TIMED_WAITING (sleeping)
              at java.lang.Thread.sleep(Native Method)
              at org.apache.accumulo.core.client.impl.ThriftScanner.pause(ThriftScanner.java:211)
              at org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:259)
              at org.apache.accumulo.core.client.impl.ScannerIterator$Reader.run(ScannerIterator.java:79)
              at org.apache.accumulo.core.client.impl.ScannerIterator.hasNext(ScannerIterator.java:150)
              at org.apache.accumulo.core.client.IsolatedScanner$RowBufferingIterator.readRow(IsolatedScanner.java:70)
              at org.apache.accumulo.core.client.IsolatedScanner$RowBufferingIterator.<init>(IsolatedScanner.java:149)
              at org.apache.accumulo.core.client.IsolatedScanner.iterator(IsolatedScanner.java:238)
              at org.apache.accumulo.core.client.RowIterator.<init>(RowIterator.java:117)
              at org.apache.accumulo.master.MasterClientServiceHandler.waitForFlush(MasterClientServiceHandler.java:188)
              at sun.reflect.GeneratedMethodAccessor8.invoke(Unknown Source)
              at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
              at java.lang.reflect.Method.invoke(Method.java:497)
              at org.apache.accumulo.core.trace.wrappers.RpcServerInvocationHandler.invoke(RpcServerInvocationHandler.java:46)
              at org.apache.accumulo.server.rpc.RpcWrapper$1.invoke(RpcWrapper.java:74)
              at com.sun.proxy.$Proxy18.waitForFlush(Unknown Source)
              at org.apache.accumulo.core.master.thrift.MasterClientService$Processor$waitForFlush.getResult(MasterClientService.java:1436)
              at org.apache.accumulo.core.master.thrift.MasterClientService$Processor$waitForFlush.getResult(MasterClientService.java:1420)
              at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)
              at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
              at org.apache.accumulo.server.rpc.TimedProcessor.process(TimedProcessor.java:63)
              at org.apache.thrift.server.AbstractNonblockingServer$FrameBuffer.invoke(AbstractNonblockingServer.java:518)
              at org.apache.accumulo.server.rpc.CustomNonBlockingServer$CustomFrameBuffer.invoke(CustomNonBlockingServer.java:106)
              at org.apache.thrift.server.Invocation.run(Invocation.java:18)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
              at org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
              at java.lang.Thread.run(Thread.java:745)
      
         Locked ownable synchronizers:
              - <0x00000006caeb94d8> (a java.util.concurrent.ThreadPoolExecutor$Worker)
      

      It would appear that in the waitForFlush implementation in MasterClientServiceHandler, we construct a scanner (with the default timeout) and wrap an IsolatedScanner around that. We do not set the timeout on the scanner which ends up using a default value of Long.MAX_VALUE (essentially, never time out).

      The problem is that in ThriftScanner, the backoff policy for retrying the failed RPC's starts at 100ms and then, for each subsequent retry loop, doubles the previous value (+/- 10%). In the face of a prolonged inability to scan a Tablet, the value we sleep grows: 100, ~200, ~400, ~800, ~1600ms, etc.

      Because the scan will not time out until Long.MAX_VALUE, we've essentially create an RPC which never times out. We should apply an upper-limit, on the order of seconds, to the maximum amount of time that ThriftScanner will sleep before retrying (without needing to change the overall timeout property)

      Attachments

        1. ACCUMULO-4405.001-1.7.patch
          7 kB
          Josh Elser

        Issue Links

          Activity

            People

              elserj Josh Elser
              elserj Josh Elser
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 0.5h
                  0.5h