[jira] [Commented] (CASSANDRA-11158) AssertionError: null in Slice$Bound.create

2016-02-22 Thread Samu Kallio (JIRA)

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

Samu Kallio commented on CASSANDRA-11158:
-

We applied this fix into our cluster (the earlier one, after it passed tests, 
not this latest one), and it fixes the original issue for us. However, we're 
now seeing a new AssertionError:

{noformat}
WARN  13:39:33 Uncaught exception on thread Thread[SharedPool-Worker-6,5,main]: 
{}
java.lang.AssertionError: null
at 
org.apache.cassandra.db.RangeTombstoneList.insertFrom(RangeTombstoneList.java:537)
 ~[apache-cassandra-3.0.3.jar:3.0.3-SNAPSHOT]
at org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:167) 
~[apache-cassandra-3.0.3.jar:3.0.3-SNAPSHOT]
at 
org.apache.cassandra.db.RangeTombstoneList.addAll(RangeTombstoneList.java:207) 
~[apache-cassandra-3.0.3.jar:3.0.3-SNAPSHOT]
at 
org.apache.cassandra.db.MutableDeletionInfo.add(MutableDeletionInfo.java:141) 
~[apache-cassandra-3.0.3.jar:3.0.3-SNAPSHOT]
at 
org.apache.cassandra.db.partitions.AtomicBTreePartition.addAllWithSizeDelta(AtomicBTreePartition.java:142)
 ~[apache-cassandra-3.0.3.jar:3.0.3-SNAPSHOT]
at org.apache.cassandra.db.Memtable.put(Memtable.java:243) 
~[apache-cassandra-3.0.3.jar:3.0.3-SNAPSHOT]
at org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:1145) 
~[apache-cassandra-3.0.3.jar:3.0.3-SNAPSHOT]
at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:502) 
~[apache-cassandra-3.0.3.jar:3.0.3-SNAPSHOT]
at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:389) 
~[apache-cassandra-3.0.3.jar:3.0.3-SNAPSHOT]
at 
org.apache.cassandra.db.SinglePartitionReadCommand$1.run(SinglePartitionReadCommand.java:750)
 ~[apache-cassandra-3.0.3.jar:3.0.3-SNAPSHOT]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[na:1.8.0_72]
at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164)
 ~[apache-cassandra-3.0.3.jar:3.0.3-SNAPSHOT]
at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) 
[apache-cassandra-3.0.3.jar:3.0.3-SNAPSHOT]
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_72]
{noformat}

This seems to be in the defragmentation code path, so it doesn't seem to cause 
any immediate issues, but I wonder if it has any implications about data 
integrity.

Should I open a new issue for this?

> AssertionError: null in Slice$Bound.create
> --
>
> Key: CASSANDRA-11158
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11158
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction, Local Write-Read Paths
>Reporter: Samu Kallio
>Assignee: Branimir Lambov
>Priority: Critical
> Fix For: 3.0.x
>
>
> We've been running Cassandra 3.0.2 for around a week now. Yesterday, we had a 
> network event that briefly isolated one node from others in a 3 node cluster. 
> Since then, we've been seeing a constant stream of "Finished hinted handoff" 
> messages, as well as:
> {noformat}
> WARN  16:34:39 Uncaught exception on thread 
> Thread[SharedPool-Worker-1,5,main]: {}
> java.lang.AssertionError: null
> at org.apache.cassandra.db.Slice$Bound.create(Slice.java:365) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.Slice$Bound$Serializer.deserializeValues(Slice.java:553)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.ClusteringPrefix$Serializer.deserialize(ClusteringPrefix.java:274)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:115) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:107) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.IndexHelper$IndexInfo$Serializer.deserialize(IndexHelper.java:149)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.RowIndexEntry$Serializer.deserialize(RowIndexEntry.java:218)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.getPosition(BigTableReader.java:216)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.getPosition(SSTableReader.java:1568)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.columniterator.SSTableIterator.(SSTableIterator.java:36)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:62)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndSSTablesInTimestampOrder(SinglePartitionReadCommand.java:715)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> 

[jira] [Commented] (CASSANDRA-11158) AssertionError: null in Slice$Bound.create

2016-02-19 Thread Samu Kallio (JIRA)

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

Samu Kallio commented on CASSANDRA-11158:
-

Does this mean that the fix is just a matter of checking for kind.isBoundary() 
== true in ClusteringPrefix.Serializer.deserialize, and handling those using 
RangeTombstone.Bound.serializer? If so, I can patch our cluster immediately to 
resolve this issue immediately for us until (perhaps better) upstream fix is 
available?

Thanks.

> AssertionError: null in Slice$Bound.create
> --
>
> Key: CASSANDRA-11158
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11158
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction, Local Write-Read Paths
>Reporter: Samu Kallio
>Assignee: Branimir Lambov
>Priority: Critical
> Fix For: 3.0.x
>
>
> We've been running Cassandra 3.0.2 for around a week now. Yesterday, we had a 
> network event that briefly isolated one node from others in a 3 node cluster. 
> Since then, we've been seeing a constant stream of "Finished hinted handoff" 
> messages, as well as:
> {noformat}
> WARN  16:34:39 Uncaught exception on thread 
> Thread[SharedPool-Worker-1,5,main]: {}
> java.lang.AssertionError: null
> at org.apache.cassandra.db.Slice$Bound.create(Slice.java:365) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.Slice$Bound$Serializer.deserializeValues(Slice.java:553)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.ClusteringPrefix$Serializer.deserialize(ClusteringPrefix.java:274)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:115) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:107) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.IndexHelper$IndexInfo$Serializer.deserialize(IndexHelper.java:149)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.RowIndexEntry$Serializer.deserialize(RowIndexEntry.java:218)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.getPosition(BigTableReader.java:216)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.getPosition(SSTableReader.java:1568)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.columniterator.SSTableIterator.(SSTableIterator.java:36)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:62)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndSSTablesInTimestampOrder(SinglePartitionReadCommand.java:715)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDiskInternal(SinglePartitionReadCommand.java:482)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDisk(SinglePartitionReadCommand.java:459)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryStorage(SinglePartitionReadCommand.java:325)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.ReadCommand.executeLocally(ReadCommand.java:350) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.ReadCommandVerbHandler.doVerb(ReadCommandVerbHandler.java:45)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:67) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> ~[na:1.8.0_72]
> at 
> org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) 
> [apache-cassandra-3.0.2.jar:3.0.2]
> at java.lang.Thread.run(Thread.java:745) [na:1.8.0_72]
> {noformat}
> and also
> {noformat}
> ERROR 06:10:11 Exception in thread Thread[CompactionExecutor:1,1,main]
> java.lang.AssertionError: null
> at org.apache.cassandra.db.Slice$Bound.create(Slice.java:365) 
> ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.db.Slice$Bound$Serializer.deserializeValues(Slice.java:553)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.db.ClusteringPrefix$Serializer.deserialize(ClusteringPrefix.java:274)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:115) 
> ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 

[jira] [Updated] (CASSANDRA-11158) AssertionError: null in Slice$Bound.create

2016-02-18 Thread Samu Kallio (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-11158?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Samu Kallio updated CASSANDRA-11158:

Priority: Critical  (was: Major)

I'm upgrading this ticket to critical as this really is causing us data loss. 
We're currently preparing to downgrade to 2.x, but would like to at least know 
if there is any way to save any of the data in the corrupt SSTables?

> AssertionError: null in Slice$Bound.create
> --
>
> Key: CASSANDRA-11158
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11158
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction, Local Write-Read Paths
>Reporter: Samu Kallio
>Priority: Critical
> Fix For: 3.0.x
>
>
> We've been running Cassandra 3.0.2 for around a week now. Yesterday, we had a 
> network event that briefly isolated one node from others in a 3 node cluster. 
> Since then, we've been seeing a constant stream of "Finished hinted handoff" 
> messages, as well as:
> {noformat}
> WARN  16:34:39 Uncaught exception on thread 
> Thread[SharedPool-Worker-1,5,main]: {}
> java.lang.AssertionError: null
> at org.apache.cassandra.db.Slice$Bound.create(Slice.java:365) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.Slice$Bound$Serializer.deserializeValues(Slice.java:553)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.ClusteringPrefix$Serializer.deserialize(ClusteringPrefix.java:274)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:115) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:107) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.IndexHelper$IndexInfo$Serializer.deserialize(IndexHelper.java:149)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.RowIndexEntry$Serializer.deserialize(RowIndexEntry.java:218)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.getPosition(BigTableReader.java:216)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.getPosition(SSTableReader.java:1568)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.columniterator.SSTableIterator.(SSTableIterator.java:36)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:62)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndSSTablesInTimestampOrder(SinglePartitionReadCommand.java:715)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDiskInternal(SinglePartitionReadCommand.java:482)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDisk(SinglePartitionReadCommand.java:459)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryStorage(SinglePartitionReadCommand.java:325)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.ReadCommand.executeLocally(ReadCommand.java:350) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.ReadCommandVerbHandler.doVerb(ReadCommandVerbHandler.java:45)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:67) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> ~[na:1.8.0_72]
> at 
> org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) 
> [apache-cassandra-3.0.2.jar:3.0.2]
> at java.lang.Thread.run(Thread.java:745) [na:1.8.0_72]
> {noformat}
> and also
> {noformat}
> ERROR 06:10:11 Exception in thread Thread[CompactionExecutor:1,1,main]
> java.lang.AssertionError: null
> at org.apache.cassandra.db.Slice$Bound.create(Slice.java:365) 
> ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.db.Slice$Bound$Serializer.deserializeValues(Slice.java:553)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.db.ClusteringPrefix$Serializer.deserialize(ClusteringPrefix.java:274)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:115) 
> ~[apache-cassandra-3.0.3.jar:3.0.3]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:107) 
> ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> 

[jira] [Updated] (CASSANDRA-11158) AssertionError: null in Slice$Bound.create

2016-02-15 Thread Samu Kallio (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-11158?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Samu Kallio updated CASSANDRA-11158:

Fix Version/s: 3.0.x

> AssertionError: null in Slice$Bound.create
> --
>
> Key: CASSANDRA-11158
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11158
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction, Local Write-Read Paths
>Reporter: Samu Kallio
> Fix For: 3.0.x
>
>
> We've been running Cassandra 3.0.2 for around a week now. Yesterday, we had a 
> network event that briefly isolated one node from others in a 3 node cluster. 
> Since then, we've been seeing a constant stream of "Finished hinted handoff" 
> messages, as well as:
> {noformat}
> WARN  16:34:39 Uncaught exception on thread 
> Thread[SharedPool-Worker-1,5,main]: {}
> java.lang.AssertionError: null
> at org.apache.cassandra.db.Slice$Bound.create(Slice.java:365) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.Slice$Bound$Serializer.deserializeValues(Slice.java:553)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.ClusteringPrefix$Serializer.deserialize(ClusteringPrefix.java:274)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:115) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:107) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.IndexHelper$IndexInfo$Serializer.deserialize(IndexHelper.java:149)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.RowIndexEntry$Serializer.deserialize(RowIndexEntry.java:218)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.getPosition(BigTableReader.java:216)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.getPosition(SSTableReader.java:1568)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.columniterator.SSTableIterator.(SSTableIterator.java:36)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:62)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndSSTablesInTimestampOrder(SinglePartitionReadCommand.java:715)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDiskInternal(SinglePartitionReadCommand.java:482)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDisk(SinglePartitionReadCommand.java:459)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryStorage(SinglePartitionReadCommand.java:325)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.ReadCommand.executeLocally(ReadCommand.java:350) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.ReadCommandVerbHandler.doVerb(ReadCommandVerbHandler.java:45)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:67) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> ~[na:1.8.0_72]
> at 
> org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) 
> [apache-cassandra-3.0.2.jar:3.0.2]
> at java.lang.Thread.run(Thread.java:745) [na:1.8.0_72]
> {noformat}
> and also
> {noformat}
> ERROR 06:10:11 Exception in thread Thread[CompactionExecutor:1,1,main]
> java.lang.AssertionError: null
> at org.apache.cassandra.db.Slice$Bound.create(Slice.java:365) 
> ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.db.Slice$Bound$Serializer.deserializeValues(Slice.java:553)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.db.ClusteringPrefix$Serializer.deserialize(ClusteringPrefix.java:274)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:115) 
> ~[apache-cassandra-3.0.3.jar:3.0.3]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:107) 
> ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.io.sstable.IndexHelper$IndexInfo$Serializer.deserialize(IndexHelper.java:149)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.db.RowIndexEntry$Serializer.deserialize(RowIndexEntry.java:218)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> 

[jira] [Comment Edited] (CASSANDRA-11158) AssertionError: null in Slice$Bound.create

2016-02-14 Thread Samu Kallio (JIRA)

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

Samu Kallio edited comment on CASSANDRA-11158 at 2/14/16 10:09 PM:
---

I ended up having to delete the corrupted SSTable files on all nodes as nothing 
could touch them without crashing. Everything seemed OK for a while, then one 
of the nodes wrote a corrupt SSTable again. Repairing the cluster is not 
possible right now because the merkle tree calculation crashes.

Also, 2 of the 3 nodes seem to be running hinted handoff every 10 seconds 
constantly, even though the whole cluster is up and has seen no network 
disruptions since the one that triggered this issue:

{noformat}
INFO  22:05:51 Deleted hint file 
97341307-d380-4808-89be-e04393bc0a5c-1455487534177-1.hints
INFO  22:05:51 Finished hinted handoff of file 
97341307-d380-4808-89be-e04393bc0a5c-1455487534177-1.hints to endpoint 
97341307-d380-4808-89be-e04393bc0a5c
INFO  22:06:01 Deleted hint file 
97341307-d380-4808-89be-e04393bc0a5c-1455487544177-1.hints
INFO  22:06:01 Finished hinted handoff of file 
97341307-d380-4808-89be-e04393bc0a5c-1455487544177-1.hints to endpoint 
97341307-d380-4808-89be-e04393bc0a5c
INFO  22:06:11 Deleted hint file 
97341307-d380-4808-89be-e04393bc0a5c-1455487554177-1.hints
INFO  22:06:11 Finished hinted handoff of file 
97341307-d380-4808-89be-e04393bc0a5c-1455487554177-1.hints to endpoint 
97341307-d380-4808-89be-e04393bc0a5c
{noformat}


was (Author: samukallio):
I ended up having to delete the corrupted SSTable files on all nodes as nothing 
could touch them without crashing. Everything seemed OK after a while, then one 
of the nodes wrote a corrupt SSTable again. Repairing the cluster is not 
possible right now because the merkle tree calculation crashes.

Also, 2 of the 3 nodes seem to be running hinted handoff every 10 seconds 
constantly, even though the whole cluster is up and has seen no network 
disruptions since the one that triggered this issue:

{noformat}
INFO  22:05:51 Deleted hint file 
97341307-d380-4808-89be-e04393bc0a5c-1455487534177-1.hints
INFO  22:05:51 Finished hinted handoff of file 
97341307-d380-4808-89be-e04393bc0a5c-1455487534177-1.hints to endpoint 
97341307-d380-4808-89be-e04393bc0a5c
INFO  22:06:01 Deleted hint file 
97341307-d380-4808-89be-e04393bc0a5c-1455487544177-1.hints
INFO  22:06:01 Finished hinted handoff of file 
97341307-d380-4808-89be-e04393bc0a5c-1455487544177-1.hints to endpoint 
97341307-d380-4808-89be-e04393bc0a5c
INFO  22:06:11 Deleted hint file 
97341307-d380-4808-89be-e04393bc0a5c-1455487554177-1.hints
INFO  22:06:11 Finished hinted handoff of file 
97341307-d380-4808-89be-e04393bc0a5c-1455487554177-1.hints to endpoint 
97341307-d380-4808-89be-e04393bc0a5c
{noformat}

> AssertionError: null in Slice$Bound.create
> --
>
> Key: CASSANDRA-11158
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11158
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction, Local Write-Read Paths
>Reporter: Samu Kallio
>
> We've been running Cassandra 3.0.2 for around a week now. Yesterday, we had a 
> network event that briefly isolated one node from others in a 3 node cluster. 
> Since then, we've been seeing a constant stream of "Finished hinted handoff" 
> messages, as well as:
> {noformat}
> WARN  16:34:39 Uncaught exception on thread 
> Thread[SharedPool-Worker-1,5,main]: {}
> java.lang.AssertionError: null
> at org.apache.cassandra.db.Slice$Bound.create(Slice.java:365) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.Slice$Bound$Serializer.deserializeValues(Slice.java:553)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.ClusteringPrefix$Serializer.deserialize(ClusteringPrefix.java:274)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:115) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:107) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.IndexHelper$IndexInfo$Serializer.deserialize(IndexHelper.java:149)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.RowIndexEntry$Serializer.deserialize(RowIndexEntry.java:218)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.getPosition(BigTableReader.java:216)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.getPosition(SSTableReader.java:1568)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.columniterator.SSTableIterator.(SSTableIterator.java:36)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> 

[jira] [Commented] (CASSANDRA-11158) AssertionError: null in Slice$Bound.create

2016-02-14 Thread Samu Kallio (JIRA)

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

Samu Kallio commented on CASSANDRA-11158:
-

I ended up having to delete the corrupted SSTable files on all nodes as nothing 
could touch them without crashing. Everything seemed OK after a while, then one 
of the nodes wrote a corrupt SSTable again. Repairing the cluster is not 
possible right now because the merkle tree calculation crashes.

Also, 2 of the 3 nodes seem to be running hinted handoff every 10 seconds 
constantly, even though the whole cluster is up and has seen no network 
disruptions since the one that triggered this issue:

{noformat}
INFO  22:05:51 Deleted hint file 
97341307-d380-4808-89be-e04393bc0a5c-1455487534177-1.hints
INFO  22:05:51 Finished hinted handoff of file 
97341307-d380-4808-89be-e04393bc0a5c-1455487534177-1.hints to endpoint 
97341307-d380-4808-89be-e04393bc0a5c
INFO  22:06:01 Deleted hint file 
97341307-d380-4808-89be-e04393bc0a5c-1455487544177-1.hints
INFO  22:06:01 Finished hinted handoff of file 
97341307-d380-4808-89be-e04393bc0a5c-1455487544177-1.hints to endpoint 
97341307-d380-4808-89be-e04393bc0a5c
INFO  22:06:11 Deleted hint file 
97341307-d380-4808-89be-e04393bc0a5c-1455487554177-1.hints
INFO  22:06:11 Finished hinted handoff of file 
97341307-d380-4808-89be-e04393bc0a5c-1455487554177-1.hints to endpoint 
97341307-d380-4808-89be-e04393bc0a5c
{noformat}

> AssertionError: null in Slice$Bound.create
> --
>
> Key: CASSANDRA-11158
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11158
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction, Local Write-Read Paths
>Reporter: Samu Kallio
>
> We've been running Cassandra 3.0.2 for around a week now. Yesterday, we had a 
> network event that briefly isolated one node from others in a 3 node cluster. 
> Since then, we've been seeing a constant stream of "Finished hinted handoff" 
> messages, as well as:
> {noformat}
> WARN  16:34:39 Uncaught exception on thread 
> Thread[SharedPool-Worker-1,5,main]: {}
> java.lang.AssertionError: null
> at org.apache.cassandra.db.Slice$Bound.create(Slice.java:365) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.Slice$Bound$Serializer.deserializeValues(Slice.java:553)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.ClusteringPrefix$Serializer.deserialize(ClusteringPrefix.java:274)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:115) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:107) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.IndexHelper$IndexInfo$Serializer.deserialize(IndexHelper.java:149)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.RowIndexEntry$Serializer.deserialize(RowIndexEntry.java:218)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.getPosition(BigTableReader.java:216)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.getPosition(SSTableReader.java:1568)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.columniterator.SSTableIterator.(SSTableIterator.java:36)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:62)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndSSTablesInTimestampOrder(SinglePartitionReadCommand.java:715)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDiskInternal(SinglePartitionReadCommand.java:482)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDisk(SinglePartitionReadCommand.java:459)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryStorage(SinglePartitionReadCommand.java:325)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.ReadCommand.executeLocally(ReadCommand.java:350) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.ReadCommandVerbHandler.doVerb(ReadCommandVerbHandler.java:45)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:67) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> ~[na:1.8.0_72]
> at 
> org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164)
>  

[jira] [Commented] (CASSANDRA-11158) AssertionError: null in Slice$Bound.create

2016-02-11 Thread Samu Kallio (JIRA)

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

Samu Kallio commented on CASSANDRA-11158:
-

Managed to trace this to a single corrupt SSTable file. Checking with 
sstableverify says the checksum is OK, but at least one row is still corrupted.

> AssertionError: null in Slice$Bound.create
> --
>
> Key: CASSANDRA-11158
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11158
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction, Local Write-Read Paths
>Reporter: Samu Kallio
>
> We've been running Cassandra 3.0.2 for around a week now. Yesterday, we had a 
> network event that briefly isolated one node from others in a 3 node cluster. 
> Since then, we've been seeing a constant stream of "Finished hinted handoff" 
> messages, as well as:
> {noformat}
> WARN  16:34:39 Uncaught exception on thread 
> Thread[SharedPool-Worker-1,5,main]: {}
> java.lang.AssertionError: null
> at org.apache.cassandra.db.Slice$Bound.create(Slice.java:365) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.Slice$Bound$Serializer.deserializeValues(Slice.java:553)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.ClusteringPrefix$Serializer.deserialize(ClusteringPrefix.java:274)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:115) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:107) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.IndexHelper$IndexInfo$Serializer.deserialize(IndexHelper.java:149)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.RowIndexEntry$Serializer.deserialize(RowIndexEntry.java:218)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.getPosition(BigTableReader.java:216)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.getPosition(SSTableReader.java:1568)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.columniterator.SSTableIterator.(SSTableIterator.java:36)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:62)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndSSTablesInTimestampOrder(SinglePartitionReadCommand.java:715)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDiskInternal(SinglePartitionReadCommand.java:482)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDisk(SinglePartitionReadCommand.java:459)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryStorage(SinglePartitionReadCommand.java:325)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.db.ReadCommand.executeLocally(ReadCommand.java:350) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.db.ReadCommandVerbHandler.doVerb(ReadCommandVerbHandler.java:45)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at 
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:67) 
> ~[apache-cassandra-3.0.2.jar:3.0.2]
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> ~[na:1.8.0_72]
> at 
> org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164)
>  ~[apache-cassandra-3.0.2.jar:3.0.2]
> at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) 
> [apache-cassandra-3.0.2.jar:3.0.2]
> at java.lang.Thread.run(Thread.java:745) [na:1.8.0_72]
> {noformat}
> and also
> {noformat}
> ERROR 06:10:11 Exception in thread Thread[CompactionExecutor:1,1,main]
> java.lang.AssertionError: null
> at org.apache.cassandra.db.Slice$Bound.create(Slice.java:365) 
> ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.db.Slice$Bound$Serializer.deserializeValues(Slice.java:553)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.db.ClusteringPrefix$Serializer.deserialize(ClusteringPrefix.java:274)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:115) 
> ~[apache-cassandra-3.0.3.jar:3.0.3]
> at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:107) 
> ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.io.sstable.IndexHelper$IndexInfo$Serializer.deserialize(IndexHelper.java:149)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> 

[jira] [Created] (CASSANDRA-11158) AssertionError: null in Slice$Bound.create

2016-02-10 Thread Samu Kallio (JIRA)
Samu Kallio created CASSANDRA-11158:
---

 Summary: AssertionError: null in Slice$Bound.create
 Key: CASSANDRA-11158
 URL: https://issues.apache.org/jira/browse/CASSANDRA-11158
 Project: Cassandra
  Issue Type: Bug
  Components: Compaction, Local Write-Read Paths
Reporter: Samu Kallio


We've been running Cassandra 3.0.2 for around a week now. Yesterday, we had a 
network event that briefly isolated one node from others in a 3 node cluster. 
Since then, we've been seeing a constant stream of "Finished hinted handoff" 
messages, as well as:

{noformat}
WARN  16:34:39 Uncaught exception on thread Thread[SharedPool-Worker-1,5,main]: 
{}
java.lang.AssertionError: null
at org.apache.cassandra.db.Slice$Bound.create(Slice.java:365) 
~[apache-cassandra-3.0.2.jar:3.0.2]
at 
org.apache.cassandra.db.Slice$Bound$Serializer.deserializeValues(Slice.java:553)
 ~[apache-cassandra-3.0.2.jar:3.0.2]
at 
org.apache.cassandra.db.ClusteringPrefix$Serializer.deserialize(ClusteringPrefix.java:274)
 ~[apache-cassandra-3.0.2.jar:3.0.2]
at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:115) 
~[apache-cassandra-3.0.2.jar:3.0.2]
at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:107) 
~[apache-cassandra-3.0.2.jar:3.0.2]
at 
org.apache.cassandra.io.sstable.IndexHelper$IndexInfo$Serializer.deserialize(IndexHelper.java:149)
 ~[apache-cassandra-3.0.2.jar:3.0.2]
at 
org.apache.cassandra.db.RowIndexEntry$Serializer.deserialize(RowIndexEntry.java:218)
 ~[apache-cassandra-3.0.2.jar:3.0.2]
at 
org.apache.cassandra.io.sstable.format.big.BigTableReader.getPosition(BigTableReader.java:216)
 ~[apache-cassandra-3.0.2.jar:3.0.2]
at 
org.apache.cassandra.io.sstable.format.SSTableReader.getPosition(SSTableReader.java:1568)
 ~[apache-cassandra-3.0.2.jar:3.0.2]
at 
org.apache.cassandra.db.columniterator.SSTableIterator.(SSTableIterator.java:36)
 ~[apache-cassandra-3.0.2.jar:3.0.2]
at 
org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:62)
 ~[apache-cassandra-3.0.2.jar:3.0.2]
at 
org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndSSTablesInTimestampOrder(SinglePartitionReadCommand.java:715)
 ~[apache-cassandra-3.0.2.jar:3.0.2]
at 
org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDiskInternal(SinglePartitionReadCommand.java:482)
 ~[apache-cassandra-3.0.2.jar:3.0.2]
at 
org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDisk(SinglePartitionReadCommand.java:459)
 ~[apache-cassandra-3.0.2.jar:3.0.2]
at 
org.apache.cassandra.db.SinglePartitionReadCommand.queryStorage(SinglePartitionReadCommand.java:325)
 ~[apache-cassandra-3.0.2.jar:3.0.2]
at org.apache.cassandra.db.ReadCommand.executeLocally(ReadCommand.java:350) 
~[apache-cassandra-3.0.2.jar:3.0.2]
at 
org.apache.cassandra.db.ReadCommandVerbHandler.doVerb(ReadCommandVerbHandler.java:45)
 ~[apache-cassandra-3.0.2.jar:3.0.2]
at 
org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:67) 
~[apache-cassandra-3.0.2.jar:3.0.2]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[na:1.8.0_72]
at 
org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164)
 ~[apache-cassandra-3.0.2.jar:3.0.2]
at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) 
[apache-cassandra-3.0.2.jar:3.0.2]
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_72]
{noformat}

and also

{noformat}
ERROR 06:10:11 Exception in thread Thread[CompactionExecutor:1,1,main]
java.lang.AssertionError: null
at org.apache.cassandra.db.Slice$Bound.create(Slice.java:365) 
~[apache-cassandra-3.0.3.jar:3.0.3]
at 
org.apache.cassandra.db.Slice$Bound$Serializer.deserializeValues(Slice.java:553)
 ~[apache-cassandra-3.0.3.jar:3.0.3]
at 
org.apache.cassandra.db.ClusteringPrefix$Serializer.deserialize(ClusteringPrefix.java:274)
 ~[apache-cassandra-3.0.3.jar:3.0.3]
at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:115) 
~[apache-cassandra-3.0.3.jar:3.0.3]
at org.apache.cassandra.db.Serializers$2.deserialize(Serializers.java:107) 
~[apache-cassandra-3.0.3.jar:3.0.3]
at 
org.apache.cassandra.io.sstable.IndexHelper$IndexInfo$Serializer.deserialize(IndexHelper.java:149)
 ~[apache-cassandra-3.0.3.jar:3.0.3]
at 
org.apache.cassandra.db.RowIndexEntry$Serializer.deserialize(RowIndexEntry.java:218)
 ~[apache-cassandra-3.0.3.jar:3.0.3]
at 
org.apache.cassandra.io.sstable.format.big.BigTableScanner$KeyScanningIterator.computeNext(BigTableScanner.java:305)
 ~[apache-cassandra-3.0.3.jar:3.0.3]
at 
org.apache.cassandra.io.sstable.format.big.BigTableScanner$KeyScanningIterator.computeNext(BigTableScanner.java:260)
 ~[apache-cassandra-3.0.3.jar:3.0.3]
at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
~[apache-cassandra-3.0.3.jar:3.0.3]
at