accumulo-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Keith Turner <ke...@deenlo.com>
Subject Re: cached connector
Date Fri, 26 Feb 2016 23:55:08 GMT
So for T=0,2,3 all stacks are calling createScanner.  Do you know if T=1
was calling createScanner?   I think nid=0x1f46 so we are seeing the same
thread continually in createScanner.  Is your code creating a single
scanner per thread?

At T=4 some thread was actually scanning, do you know if thats thread
0x1f46?

At T=5, do you know what high level Accumulo op thread 0x1f46 was
performing?

On Fri, Feb 26, 2016 at 6:26 PM, z11373 <z11373@outlook.com> wrote:

> Hi Keith,
> I wrote a script to capture the jstack of that process, sleep 1s, and then
> capture again.
> Below is the sequence of stack trace (without my service code), hope it'll
> give you some hints.
> I see the part which looks suspicious is that ZK retry code.
>
>
> *** T+0 ***:
> "qtp6320204-7632" #7632 prio=5 os_prio=0 tid=0x00007ff07c008800 nid=0x1f46
> runnable [0x00007feea30ee000]
>    java.lang.Thread.State: RUNNABLE
>         at
> org.apache.accumulo.fate.zookeeper.ZooCache.retry(ZooCache.java:154)
>         - locked <0x00000006800daa48> (a
> org.apache.accumulo.fate.zookeeper.ZooCache)
>         at
> org.apache.accumulo.fate.zookeeper.ZooCache.getChildren(ZooCache.java:220)
>         - locked <0x00000006800daa48> (a
> org.apache.accumulo.fate.zookeeper.ZooCache)
>         at
> org.apache.accumulo.core.client.impl.Namespaces.getMap(Namespaces.java:94)
>         at
>
> org.apache.accumulo.core.client.impl.Namespaces.getIdToNameMap(Namespaces.java:135)
>         at
>
> org.apache.accumulo.core.client.impl.Namespaces.getNamespaceName(Namespaces.java:124)
>         at
> org.apache.accumulo.core.client.impl.Tables.getMap(Tables.java:146)
>         at
> org.apache.accumulo.core.client.impl.Tables.getNameToIdMap(Tables.java:200)
>         at
> org.apache.accumulo.core.client.impl.Tables._getTableId(Tables.java:176)
>         at
> org.apache.accumulo.core.client.impl.Tables.getTableId(Tables.java:169)
>         at
>
> org.apache.accumulo.core.client.impl.ConnectorImpl.getTableId(ConnectorImpl.java:80)
>         at
>
> org.apache.accumulo.core.client.impl.ConnectorImpl.createScanner(ConnectorImpl.java:147)
>         ...
>
> *** T+1 ***:
> "qtp6320204-7632" #7632 prio=5 os_prio=0 tid=0x00007ff07c008800 nid=0x1f46
> runnable [0x00007feea30ee000]
>    java.lang.Thread.State: RUNNABLE
>         at java.util.zip.Deflater.init(Native Method)
>         at java.util.zip.Deflater.<init>(Deflater.java:171)
>         at java.util.zip.GZIPOutputStream.<init>(GZIPOutputStream.java:90)
>         at java.util.zip.GZIPOutputStream.<init>(GZIPOutputStream.java:109)
>         at
>
> org.apache.hadoop.io.WritableUtils.writeCompressedByteArray(WritableUtils.java:64)
>         at
>
> org.apache.accumulo.core.client.security.tokens.PasswordToken.write(PasswordToken.java:87)
>         at
>
> org.apache.accumulo.core.client.security.tokens.AuthenticationToken$AuthenticationTokenSerializer.serialize(AuthenticationToken.java:115)
>         at
> org.apache.accumulo.core.security.Credentials.toThrift(Credentials.java:87)
>         at
>
> org.apache.accumulo.core.client.impl.SecurityOperationsImpl$6.execute(SecurityOperationsImpl.java:185)
>         at
>
> org.apache.accumulo.core.client.impl.SecurityOperationsImpl$6.execute(SecurityOperationsImpl.java:182)
>         at
>
> org.apache.accumulo.core.client.impl.ServerClient.executeRaw(ServerClient.java:81)
>         at
>
> org.apache.accumulo.core.client.impl.SecurityOperationsImpl.execute(SecurityOperationsImpl.java:69)
>         at
>
> org.apache.accumulo.core.client.impl.SecurityOperationsImpl.getUserAuthorizations(SecurityOperationsImpl.java:182)
>         ...
>
> *** T+2 ***:
> "qtp6320204-7632" #7632 prio=5 os_prio=0 tid=0x00007ff07c008800 nid=0x1f46
> runnable [0x00007feea30ee000]
>    java.lang.Thread.State: RUNNABLE
>         at java.util.HashMap.getNode(HashMap.java:578)
>         at java.util.HashMap.containsKey(HashMap.java:595)
>         at
> org.apache.accumulo.fate.zookeeper.ZooCache$2.run(ZooCache.java:255)
>         at
> org.apache.accumulo.fate.zookeeper.ZooCache.retry(ZooCache.java:161)
>         - eliminated <0x00000006800daa48> (a
> org.apache.accumulo.fate.zookeeper.ZooCache)
>         at
> org.apache.accumulo.fate.zookeeper.ZooCache.get(ZooCache.java:291)
>         - eliminated <0x00000006800daa48> (a
> org.apache.accumulo.fate.zookeeper.ZooCache)
>         at
> org.apache.accumulo.fate.zookeeper.ZooCache.get(ZooCache.java:237)
>         - locked <0x00000006800daa48> (a
> org.apache.accumulo.fate.zookeeper.ZooCache)
>         at
> org.apache.accumulo.core.client.impl.Tables.getMap(Tables.java:135)
>         at
> org.apache.accumulo.core.client.impl.Tables.getNameToIdMap(Tables.java:200)
>         at
> org.apache.accumulo.core.client.impl.Tables._getTableId(Tables.java:176)
>         at
> org.apache.accumulo.core.client.impl.Tables.getTableId(Tables.java:169)
>         at
>
> org.apache.accumulo.core.client.impl.ConnectorImpl.getTableId(ConnectorImpl.java:80)
>         at
>
> org.apache.accumulo.core.client.impl.ConnectorImpl.createScanner(ConnectorImpl.java:147)
>         ...
>
> *** T+3 ***:
> "qtp6320204-7632" #7632 prio=5 os_prio=0 tid=0x00007ff07c008800 nid=0x1f46
> runnable [0x00007feea30ee000]
>    java.lang.Thread.State: RUNNABLE
>         at
> org.apache.accumulo.fate.zookeeper.ZooReader.getSession(ZooReader.java:39)
>         at
>
> org.apache.accumulo.fate.zookeeper.ZooReader.getZooKeeper(ZooReader.java:43)
>         at
> org.apache.accumulo.fate.zookeeper.ZooCache.getZooKeeper(ZooCache.java:58)
>         at
> org.apache.accumulo.fate.zookeeper.ZooCache.retry(ZooCache.java:158)
>         - eliminated <0x00000006800daa48> (a
> org.apache.accumulo.fate.zookeeper.ZooCache)
>         at
> org.apache.accumulo.fate.zookeeper.ZooCache.get(ZooCache.java:291)
>         - eliminated <0x00000006800daa48> (a
> org.apache.accumulo.fate.zookeeper.ZooCache)
>         at
> org.apache.accumulo.fate.zookeeper.ZooCache.get(ZooCache.java:237)
>         - locked <0x00000006800daa48> (a
> org.apache.accumulo.fate.zookeeper.ZooCache)
>         at
>
> org.apache.accumulo.core.client.ZooKeeperInstance.getInstanceID(ZooKeeperInstance.java:176)
>         at
> org.apache.accumulo.core.zookeeper.ZooUtil.getRoot(ZooUtil.java:39)
>         at
> org.apache.accumulo.core.client.impl.Tables.getMap(Tables.java:134)
>         at
> org.apache.accumulo.core.client.impl.Tables.getNameToIdMap(Tables.java:200)
>         at
> org.apache.accumulo.core.client.impl.Tables._getTableId(Tables.java:176)
>         at
> org.apache.accumulo.core.client.impl.Tables.getTableId(Tables.java:169)
>         at
>
> org.apache.accumulo.core.client.impl.ConnectorImpl.getTableId(ConnectorImpl.java:80)
>         at
>
> org.apache.accumulo.core.client.impl.ConnectorImpl.createScanner(ConnectorImpl.java:147)
>         ...
>
> *** T+4 ***:
> My service code actually spawned few threads, each create Scanner, and read
> the data from Accumulo, so at this point I see a few stacks with something
> like:
>    java.lang.Thread.State: RUNNABLE
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked <0x00000006a6f02480> (a sun.nio.ch.Util$2)
>         - locked <0x00000006a6f02470> (a
> java.util.Collections$UnmodifiableSet)
>         - locked <0x00000006a6f02248> (a sun.nio.ch.EPollSelectorImpl)
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at
>
> org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:335)
>         at
>
> org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
>         at
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161)
>         at
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:131)
>         at java.io.FilterInputStream.read(FilterInputStream.java:133)
>         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 <0x00000006a70f7a60> (a java.io.BufferedInputStream)
>         at
>
> org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:127)
>         at
> org.apache.thrift.transport.TTransport.readAll(TTransport.java:84)
>         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:84)
>         at
>
> org.apache.accumulo.core.client.impl.ThriftTransportPool$CachedTTransport.readAll(ThriftTransportPool.java:274)
>         at
>
> org.apache.thrift.protocol.TCompactProtocol.readByte(TCompactProtocol.java:601)
>         at
>
> org.apache.thrift.protocol.TCompactProtocol.readMessageBegin(TCompactProtocol.java:470)
>         at
> org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:69)
>         at
>
> org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.recv_startScan(TabletClientService.java:228)
>         at
>
> org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.startScan(TabletClientService.java:204)
>         at
>
> org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:403)
>         at
>
> org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:279)
>         at
>
> org.apache.accumulo.core.client.impl.ScannerIterator$Reader.run(ScannerIterator.java:84)
>         at
>
> org.apache.accumulo.core.client.impl.ScannerIterator.hasNext(ScannerIterator.java:177)
>         ...
>
> *** T+5 ***:
> "qtp6320204-7632" #7632 prio=5 os_prio=0 tid=0x00007ff07c008800 nid=0x1f46
> runnable [0x00007feea30ee000]
>    java.lang.Thread.State: RUNNABLE
>         at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>         at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>         at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79)
>         at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>         - locked <0x00000006a6fded70> (a sun.nio.ch.Util$2)
>         - locked <0x00000006a6fded60> (a
> java.util.Collections$UnmodifiableSet)
>         - locked <0x00000006a6fdeb38> (a sun.nio.ch.EPollSelectorImpl)
>         at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>         at
>
> org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:335)
>         at
>
> org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
>         at
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161)
>         at
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:131)
>         at java.io.FilterInputStream.read(FilterInputStream.java:133)
>         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 <0x00000006a6fc3e90> (a java.io.BufferedInputStream)
>         at
>
> org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:127)
>         at
> org.apache.thrift.transport.TTransport.readAll(TTransport.java:84)
>         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:84)
>         at
>
> org.apache.accumulo.core.client.impl.ThriftTransportPool$CachedTTransport.readAll(ThriftTransportPool.java:274)
>         at
>
> org.apache.thrift.protocol.TCompactProtocol.readByte(TCompactProtocol.java:601)
>         at
>
> org.apache.thrift.protocol.TCompactProtocol.readMessageBegin(TCompactProtocol.java:470)
>         at
> org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:69)
>         at
>
> org.apache.accumulo.core.client.impl.thrift.ClientService$Client.recv_getUserAuthorizations(ClientService.java:582)
>         at
>
> org.apache.accumulo.core.client.impl.thrift.ClientService$Client.getUserAuthorizations(ClientService.java:567)
>         at
>
> org.apache.accumulo.core.client.impl.SecurityOperationsImpl$6.execute(SecurityOperationsImpl.java:185)
>         at
>
> org.apache.accumulo.core.client.impl.SecurityOperationsImpl$6.execute(SecurityOperationsImpl.java:182)
>         at
>
> org.apache.accumulo.core.client.impl.ServerClient.executeRaw(ServerClient.java:81)
>         at
>
> org.apache.accumulo.core.client.impl.SecurityOperationsImpl.execute(SecurityOperationsImpl.java:69)
>         at
>
> org.apache.accumulo.core.client.impl.SecurityOperationsImpl.getUserAuthorizations(SecurityOperationsImpl.java:182)
>         ...
>
>
>
>
>
> --
> View this message in context:
> http://apache-accumulo.1065345.n5.nabble.com/cached-connector-tp16202p16240.html
> Sent from the Developers mailing list archive at Nabble.com.
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message