Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 908BC200B63 for ; Mon, 15 Aug 2016 18:55:22 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 8F422160ABE; Mon, 15 Aug 2016 16:55:22 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id B77A9160A8A for ; Mon, 15 Aug 2016 18:55:21 +0200 (CEST) Received: (qmail 92077 invoked by uid 500); 15 Aug 2016 16:55:21 -0000 Mailing-List: contact notifications-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: jira@apache.org Delivered-To: mailing list notifications@accumulo.apache.org Received: (qmail 92029 invoked by uid 99); 15 Aug 2016 16:55:20 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 15 Aug 2016 16:55:20 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id BB5EF2C02A4 for ; Mon, 15 Aug 2016 16:55:20 +0000 (UTC) Date: Mon, 15 Aug 2016 16:55:20 +0000 (UTC) From: "Josh Elser (JIRA)" To: notifications@accumulo.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (ACCUMULO-4405) GC collection cycle stuck on waitForFlush RPC to Master MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Mon, 15 Aug 2016 16:55:22 -0000 [ https://issues.apache.org/jira/browse/ACCUMULO-4405?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Josh Elser updated ACCUMULO-4405: --------------------------------- Affects Version/s: 1.7.1 1.7.2 > GC collection cycle stuck on waitForFlush RPC to Master > ------------------------------------------------------- > > Key: ACCUMULO-4405 > URL: https://issues.apache.org/jira/browse/ACCUMULO-4405 > Project: Accumulo > Issue Type: Bug > Components: gc, master > Affects Versions: 1.7.1, 1.7.2 > Reporter: Josh Elser > Assignee: Josh Elser > Fix For: 1.7.3, 1.8.1 > > > 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: > {noformat} > "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 > {noformat} > The Master was also stuck with an active thread/RPC: > {noformat} > "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.(IsolatedScanner.java:149) > at org.apache.accumulo.core.client.IsolatedScanner.iterator(IsolatedScanner.java:238) > at org.apache.accumulo.core.client.RowIterator.(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) > {noformat} > 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) -- This message was sent by Atlassian JIRA (v6.3.4#6332)