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

Yang Yang commented on CASSANDRA-3179:
--------------------------------------

very interesting, I switched from mmap mode to "standard", and go the following 
errors on compaction,
I believe they follow the same path as the mmap route, but since it gave an 
exception here, instead of siliently SEGV, this could provide a useful hint to 
what caused the SEGV




Caused by: java.nio.channels.ClosedChannelException
        at 
org.apache.cassandra.io.util.RandomAccessReader.read(RandomAccessReader.java:268)
        at java.io.RandomAccessFile.readByte(RandomAccessFile.java:640)
        at 
org.apache.cassandra.utils.ByteBufferUtil.readShortLength(ByteBufferUtil.java:356)
        at 
org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:367)
        at 
org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:87)
        at 
org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:82)
        at 
org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:72)
        at 
org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:36)
        at 
org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:79)
        ... 21 more
ERROR 01:19:22,415 Fatal exception in thread Thread[ReadStage:246,5,main]
java.lang.RuntimeException: java.lang.RuntimeException: error reading 1 of 1
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1165)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
        at java.lang.Thread.run(Thread.java:679)
Caused by: java.lang.RuntimeException: error reading 1 of 1
        at 
org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:83)
        at 
org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:40)
        at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
        at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
        at 
org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SSTableSliceIterator.java:107)
        at 
org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:194)
        at 
org.apache.cassandra.utils.MergeIterator.<init>(MergeIterator.java:47)
        at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.<init>(MergeIterator.java:142)
        at org.apache.cassandra.utils.MergeIterator.get(MergeIterator.java:66)
        at 
org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.java:96)
        at 
org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:249)
        at 
org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:61)
        at 
org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1276)
        at 
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1171)

        at 
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1127)
        at org.apache.cassandra.db.Table.getRow(Table.java:388)
        at 
org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:61)
        at 
org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:694)
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1161)
        ... 3 more
Caused by: java.nio.channels.ClosedChannelException
        at 
org.apache.cassandra.io.util.RandomAccessReader.read(RandomAccessReader.java:268)
        at java.io.RandomAccessFile.readByte(RandomAccessFile.java:640)
        at 
org.apache.cassandra.utils.ByteBufferUtil.readShortLength(ByteBufferUtil.java:356)
        at 
org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:367)
        at 
org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:87)
        at 
org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:82)
        at 
org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:72)
        at 
org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:36)
        at 
org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:79)
        ... 21 more
 INFO 01:19:23,738 Compacted to 
[/mnt/cass/lib/cassandra/data/testBudget_items/measuredSession-h-10-Data.db,].  
16,840,264 to 16,840,036 (~99% of original) bytes for 22,531 keys at 
3.247707MBPS.  Time: 4,945ms.
 INFO 01:19:23,738 CF Total Bytes Compacted: 157,135,410


> JVM segfaults
> -------------
>
>                 Key: CASSANDRA-3179
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3179
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 1.0.0
>         Environment: java version "1.6.0_26"
> Java(TM) SE Runtime Environment (build 1.6.0_26-b03)
> Java HotSpot(TM) 64-Bit Server VM (build 20.1-b02, mixed mode)
>            Reporter: Brandon Williams
>            Assignee: Jonathan Ellis
>             Fix For: 1.0.0
>
>         Attachments: 3179-performance-test.txt, 3179-v2.txt, 3179.txt, 
> hs_err_pid12074.log, hs_err_pid28971.log, hs_err_pid6461.log, 
> hs_err_pid7031.log
>
>
> Both with and without compressed OOPs enabled.  Seems to mostly happen during 
> compaction+reads.  I'll attach some hs_err files shortly.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to