You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "Eric Newton (JIRA)" <ji...@apache.org> on 2014/11/03 16:36:33 UTC

[jira] [Commented] (ACCUMULO-3273) majc holding up tablet unloads?

    [ https://issues.apache.org/jira/browse/ACCUMULO-3273?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14194628#comment-14194628 ] 

Eric Newton commented on ACCUMULO-3273:
---------------------------------------

Had this happen on a test system again today.  I was able to jstack the tserver.  The Unloader is waiting for MajC to stop.  The MajC is not stopping because it is not reading any data: it is hung waiting on HDFS.  System is using CDH-5.1.3.

Partial stack trace:

{noformat}
"MajC compacting f;00176b;00147f3 started 2014/11/03 08:44:37.184 file: hdfs://nn.default.domain:8020/accumulo/tables/f/t-00a4nz6/A00a7ajh.rf_tmp" daemon prio=10 tid=0x000000008a03c800 nid=0x5e10 runnable [0x00007fffd608a000]
   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:87)
        - locked <0x000000075e526f48> (a sun.nio.ch.Util$2)
        - locked <0x000000075e526f60> (a java.util.Collections$UnmodifiableSet)
        - locked <0x000000075f2a9708> (a sun.nio.ch.EPollSelectorImpl)
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
        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.hdfs.protocol.datatransfer.PacketReceiver.readChannelFully(PacketReceiver.java:258)
        at org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doReadFully(PacketReceiver.java:209)
        at org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doRead(PacketReceiver.java:171)
        at org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.receiveNextPacket(PacketReceiver.java:102)
        at org.apache.hadoop.hdfs.RemoteBlockReader2.readNextPacket(RemoteBlockReader2.java:173)
        at org.apache.hadoop.hdfs.RemoteBlockReader2.read(RemoteBlockReader2.java:138)
        - locked <0x000000075e553250> (a org.apache.hadoop.hdfs.RemoteBlockReader2)
        at org.apache.hadoop.hdfs.DFSInputStream$ByteArrayStrategy.doRead(DFSInputStream.java:682)
        at org.apache.hadoop.hdfs.DFSInputStream.readBuffer(DFSInputStream.java:738)
        - eliminated <0x000000075e553190> (a org.apache.hadoop.hdfs.DFSInputStream)
        at org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:795)
        at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:836)
        - locked <0x000000075e553190> (a org.apache.hadoop.hdfs.DFSInputStream)
        at java.io.DataInputStream.read(DataInputStream.java:149)
        at org.apache.accumulo.core.file.rfile.bcfile.BoundedRangeFileInputStream$1.run(BoundedRangeFileInputStream.java:104)
        at org.apache.accumulo.core.file.rfile.bcfile.BoundedRangeFileInputStream$1.run(BoundedRangeFileInputStream.java:100)
        at java.security.AccessController.doPrivileged(Native Method)
        at org.apache.accumulo.core.file.rfile.bcfile.BoundedRangeFileInputStream.read(BoundedRangeFileInputStream.java:100)
        - locked <0x000000075e598380> (a org.apache.hadoop.hdfs.client.HdfsDataInputStream)
        at org.apache.hadoop.io.compress.DecompressorStream.getCompressedData(DecompressorStream.java:159)
        at org.apache.hadoop.io.compress.DecompressorStream.decompress(DecompressorStream.java:143)
        at org.apache.hadoop.io.compress.DecompressorStream.read(DecompressorStream.java:85)
        at java.io.BufferedInputStream.fill(BufferedInputStream.java:235)
        at java.io.BufferedInputStream.read1(BufferedInputStream.java:275)
        at java.io.BufferedInputStream.read(BufferedInputStream.java:334)
        - locked <0x000000075f9bd1d8> (a java.io.BufferedInputStream)
        at java.io.DataInputStream.read(DataInputStream.java:149)
        at java.io.DataInputStream.readFully(DataInputStream.java:195)
        at org.apache.accumulo.core.data.Value.readFields(Value.java:157)
        at org.apache.accumulo.core.file.rfile.RFile$LocalityGroupReader._next(RFile.java:577)
        at org.apache.accumulo.core.file.rfile.RFile$LocalityGroupReader.next(RFile.java:543)
        at org.apache.accumulo.core.iterators.system.HeapIterator.next(HeapIterator.java:80)
        at org.apache.accumulo.server.problems.ProblemReportingIterator.next(ProblemReportingIterator.java:77)
        at org.apache.accumulo.core.iterators.system.HeapIterator.next(HeapIterator.java:80)
        at org.apache.accumulo.core.iterators.WrappingIterator.next(WrappingIterator.java:96)
        at org.apache.accumulo.tserver.Compactor$CountingIterator.next(Compactor.java:108)
        at org.apache.accumulo.core.iterators.system.DeletingIterator.next(DeletingIterator.java:58)
        at org.apache.accumulo.core.iterators.WrappingIterator.next(WrappingIterator.java:96)
        at org.apache.accumulo.core.iterators.SkippingIterator.next(SkippingIterator.java:34)
        at org.apache.accumulo.core.iterators.system.SynchronizedIterator.next(SynchronizedIterator.java:51)
        - locked <0x000000075e005ac8> (a org.apache.accumulo.core.iterators.system.SynchronizedIterator)
        at org.apache.accumulo.core.iterators.WrappingIterator.next(WrappingIterator.java:96)
        at org.apache.accumulo.core.iterators.user.VersioningIterator.skipRowColumn(VersioningIterator.java:97)
        at org.apache.accumulo.core.iterators.user.VersioningIterator.next(VersioningIterator.java:58)
        at org.apache.accumulo.tserver.Compactor.compactLocalityGroup(Compactor.java:505)
        at org.apache.accumulo.tserver.Compactor.call(Compactor.java:362)
{noformat}

> majc holding up tablet unloads?
> -------------------------------
>
>                 Key: ACCUMULO-3273
>                 URL: https://issues.apache.org/jira/browse/ACCUMULO-3273
>             Project: Accumulo
>          Issue Type: Bug
>          Components: tserver
>    Affects Versions: 1.6.0, 1.6.1
>            Reporter: Eric Newton
>            Assignee: Eric Newton
>            Priority: Minor
>             Fix For: 1.6.2, 1.7.0
>
>
> While testing ACCUMULO-3263 on a large cluster, the table being randomized would not go offline.  Each of these servers was performing a major compaction of the tablets to be offlined.  I thought that taking a tablet offline would abort the majc.  Need an IT to verify this.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)