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

Balázs Póka commented on CASSANDRA-4687:
----------------------------------------

I've just reproduced it with Cassandra 1.2.6 (on Ubuntu 12.04 OpenVZ box). 1 
node cluster.

A have multiple big column families, and have been doing lots of reads/writes 
while running a data migration program. Those CFs seem to be affected more 
which get most of the reads and writes.

Log of the first exception and lines immediately before it:

INFO [OptionalTasks:1] 2013-07-23 16:36:08,651 ColumnFamilyStore.java (line 
631) Enqueuing flush of Memtable-measuredata_201305@1749058828(11711/26488 
serialized/live bytes, 276 ops)
 INFO [FlushWriter:755] 2013-07-23 16:36:08,681 Memtable.java (line 461) 
Writing Memtable-measuredata_201305@1749058828(11711/26488 serialized/live 
bytes, 276 ops)
 INFO [FlushWriter:755] 2013-07-23 16:36:08,789 Memtable.java (line 495) 
Completed flushing 
/mnt/db/cassandra/gps/measuredata_201305/gps-measuredata_201305-ic-1-Data.db 
(7808 bytes) for commitlog position ReplayPosition(segmentId=136978869451
5, position=7660)
ERROR [ReadStage:55564] 2013-07-23 17:10:00,268 CassandraDaemon.java (line 175) 
Exception in thread Thread[ReadStage:55564,5,main]
java.lang.RuntimeException: java.lang.IllegalArgumentException: unable to seek 
to position 60965 in 
/mnt/db/cassandra/gps/measuredata_201305/gps-measuredata_201305-ic-1-Data.db 
(12801 bytes) in read-only mode
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1582)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:722)
Caused by: java.lang.IllegalArgumentException: unable to seek to position 60965 
in /mnt/db/cassandra/gps/measuredata_201305/gps-measuredata_201305-ic-1-Data.db 
(12801 bytes) in read-only mode
        at 
org.apache.cassandra.io.util.RandomAccessReader.seek(RandomAccessReader.java:306)
        at 
org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(PoolingSegmentedFile.java:42)
        at 
org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:976)
        at 
org.apache.cassandra.db.columniterator.SSTableNamesIterator.createFileDataInput(SSTableNamesIterator.java:94)
        at 
org.apache.cassandra.db.columniterator.SSTableNamesIterator.read(SSTableNamesIterator.java:112)
        at 
org.apache.cassandra.db.columniterator.SSTableNamesIterator.<init>(SSTableNamesIterator.java:60)
        at 
org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:81)
        at 
org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:68)
        at 
org.apache.cassandra.db.CollationController.collectTimeOrderedData(CollationController.java:133)
        at 
org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:65)
        at 
org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1357)
        at 
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1214)
        at 
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1126)
        at org.apache.cassandra.db.Table.getRow(Table.java:347)
        at 
org.apache.cassandra.db.SliceByNamesReadCommand.getRow(SliceByNamesReadCommand.java:64)
        at 
org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1052)
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1578)
        ... 3 more

Clearing the key cache with "nodetool invalidatekeycache" effectively fixed the 
problem. I have not disabled it. Row cache is disabled.
                
> Exception: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk)
> -----------------------------------------------------------
>
>                 Key: CASSANDRA-4687
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-4687
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: CentOS 6.3 64-bit, Oracle JRE 1.6.0.33 64-bit, single 
> node cluster
>            Reporter: Leonid Shalupov
>            Priority: Minor
>         Attachments: 4687-debugging.txt
>
>
> Under heavy write load sometimes cassandra fails with assertion error.
> git bisect leads to commit 295aedb278e7a495213241b66bc46d763fd4ce66.
> works fine if global key/row caches disabled in code.
> {quote}
> java.lang.AssertionError: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk) in 
> /var/lib/cassandra/data/...-he-1-Data.db
>       at 
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.<init>(SSTableSliceIterator.java:60)
>       at 
> org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:67)
>       at 
> org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79)
>       at 
> org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:256)
>       at 
> org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64)
>       at 
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1345)
>       at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1207)
>       at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1142)
>       at org.apache.cassandra.db.Table.getRow(Table.java:378)
>       at 
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:69)
>       at 
> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:819)
>       at 
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1253)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>       at java.lang.Thread.run(Thread.java:662)
> {quote}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to