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

Branimir Lambov reassigned CASSANDRA-11158:
-------------------------------------------

    Assignee: Branimir Lambov

> 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.<init>(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 
> org.apache.cassandra.io.sstable.format.big.BigTableScanner.hasNext(BigTableScanner.java:240)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:369)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:189)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:158)
>  ~[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 
> org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$2.hasNext(UnfilteredPartitionIterators.java:150)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:72)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.db.compaction.CompactionIterator.hasNext(CompactionIterator.java:226)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:177)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
> ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:78)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:60)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at 
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:263)
>  ~[apache-cassandra-3.0.3.jar:3.0.3]
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> ~[na:1.8.0_72]
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_72]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  ~[na:1.8.0_72]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_72]
> at java.lang.Thread.run(Thread.java:745) [na:1.8.0_72]
> {noformat}
> on all 3 nodes. I'm now upgrading the nodes to 3.0.3, but the issue seems to 
> persist.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to