[ https://issues.apache.org/jira/browse/CASSANDRA-866?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12855172#action_12855172 ]
B. Todd Burruss commented on CASSANDRA-866: ------------------------------------------- awright, big daddy ... reproduced again, and data is saved ;) 2010-04-07 21:21:10,520 ERROR [ROW-READ-STAGE:4] [CassandraDaemon.java:78] Fatal exception in thread Thread[ROW-READ-STAGE:4,5,main] java.lang.AssertionError: DecoratedKey(147587030576932389559405437065042613628, vmguest85__-1131008275) != DecoratedKey(147587045543996727516366006491335105792, vmguest85__-888697030) in /data/cassandra-data/data/uds/bucket-1439-Data.db at org.apache.cassandra.db.filter.SSTableSliceIterator$ColumnGroupReader.<init>(SSTableSliceIterator.java:127) at org.apache.cassandra.db.filter.SSTableSliceIterator.<init>(SSTableSliceIterator.java:59) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:63) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:830) at org.apache.cassandra.db.ColumnFamilyStore.cacheRow(ColumnFamilyStore.java:727) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:752) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:719) at org.apache.cassandra.db.Table.getRow(Table.java:381) at org.apache.cassandra.db.SliceByNamesReadCommand.getRow(SliceByNamesReadCommand.java:56) at org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:70) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:40) 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:619) emailed jonathan an http site to download data files that reproduce this problem. AssertionError happened on node 105 (see ring below), and according to the keys shown in above exception, 105 is the primary for the key. ReplicationFactor = 3 so 102 and 103 are replicas, right? i can use the cassandra-cli to retrieve the value on the key's replica nodes, just not on the primary node. Address Status Load Range Ring 170141183460469231731687303715884105728 192.168.132.102Up 41.64 GB 42535295865117307932921825928971026431 |<--| 192.168.132.103Up 41.33 GB 85070591730234615865843651857942052863 | | 192.168.132.104Up 41.52 GB 127605887595351923798765477786913079295 | | 192.168.132.105Up 36.86 GB 170141183460469231731687303715884105728 |-->| > AssertionError SSTableSliceIterator.java:126 > -------------------------------------------- > > Key: CASSANDRA-866 > URL: https://issues.apache.org/jira/browse/CASSANDRA-866 > Project: Cassandra > Issue Type: Bug > Components: Core > Affects Versions: 0.6 > Reporter: B. Todd Burruss > Fix For: 0.6.1 > > > also seeing these, using cassandra-0.6.0-beta2/ > 2010-03-09 07:37:57,683 ERROR [ROW-READ-STAGE:77] [CassandraDaemon.java:78] > Fatal exception in thread Thread[ROW-READ-STAGE:77,5,main] > java.lang.AssertionError > at > org.apache.cassandra.db.filter.SSTableSliceIterator$ColumnGroupReader.<init>(SSTableSliceIterator.java:126) > at > org.apache.cassandra.db.filter.SSTableSliceIterator.<init>(SSTableSliceIterator.java:59) > at > org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:63) > at > org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:851) > at > org.apache.cassandra.db.ColumnFamilyStore.cacheRow(ColumnFamilyStore.java:748) > at > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:773) > at > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:740) > at org.apache.cassandra.db.Table.getRow(Table.java:381) > at > org.apache.cassandra.db.SliceByNamesReadCommand.getRow(SliceByNamesReadCommand.java:56) > at > org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:80) > at > org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:40) > 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:619) -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.