You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "T Jake Luciani (JIRA)" <ji...@apache.org> on 2012/10/05 20:12:03 UTC
[jira] [Commented] (CASSANDRA-4687) Exception: DecoratedKey(xxx,
yyy) != DecoratedKey(zzz, kkk)
[ https://issues.apache.org/jira/browse/CASSANDRA-4687?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13470512#comment-13470512 ]
T Jake Luciani commented on CASSANDRA-4687:
-------------------------------------------
I am seeing something similar here.
{code}
ERROR [ReadStage:70] 2012-10-05 13:26:40,215 AbstractCassandraDaemon.java (line 135) Exception in thread Thread[ReadStage:70,5,main]
java.io.IOError: java.io.EOFException: unable to seek to position 165713136 in /opt/data/cassandra/data/prod/market_data_state/prod-market_data_state-he-5-Data.db (74622511 bytes) in read-only mode
at org.apache.cassandra.io.util.CompressedSegmentedFile.getSegment(CompressedSegmentedFile.java:69)
at org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:898)
at org.apache.cassandra.db.columniterator.SSTableSliceIterator.<init>(SSTableSliceIterator.java:50)
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.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:51)
at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59)
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)
Caused by: java.io.EOFException: unable to seek to position 165713136 in /opt/data/cassandra/data/prod/market_data_state/prod-market_data_state-he-5-Data.db (74622511 bytes) in read-only mode
at org.apache.cassandra.io.util.RandomAccessReader.seek(RandomAccessReader.java:253)
at org.apache.cassandra.io.util.CompressedSegmentedFile.getSegment(CompressedSegmentedFile.java:64)
... 16 more
{code}
It seems to be hopping SSTables in compaction for the same row I'm trying to access so could be serialization issue.
> 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
> Affects Versions: 1.1.5
> Environment: CentOS 6.3 64-bit, Oracle JRE 1.6.0.33 64-bit, single node cluster
> Reporter: Leonid Shalupov
> Assignee: Pavel Yaskevich
> Priority: Critical
> Fix For: 1.1.6
>
> 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