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

Alex Petrov edited comment on CASSANDRA-15778 at 5/13/20, 7:28 AM:
-------------------------------------------------------------------

I've taken a short look and wanted to precaution against ignoring or even 
purging values from the hidden compact column, since there are scenarios when 
it can contain legitimate data. 

I've been able to reproduce an issue with a similar, albeit not exactly same 
stacktrace (leaving only the part from sstable iterator downward):

{code}
ERROR 18:08:11 Uncaught exception on thread Thread[SharedPool-Worker-2,10,main]
java.lang.RuntimeException: org.apache.cassandra.serializers.MarshalException: 
EmptyType only accept empty values
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2470)
 ~[main/:na]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[na:1.8.0_171]
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165)
 ~[main/:na]
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:137)
 [main/:na]
        at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) 
[main/:na]
        at java.lang.Thread.run(Thread.java:748) [na:1.8.0_171]
Caused by: org.apache.cassandra.serializers.MarshalException: EmptyType only 
accept empty values
        at 
org.apache.cassandra.serializers.EmptySerializer.validate(EmptySerializer.java:42)
 ~[main/:na]
        at 
org.apache.cassandra.db.marshal.AbstractType.validate(AbstractType.java:159) 
~[main/:na]
        at 
org.apache.cassandra.db.marshal.AbstractType.validateIfFixedSize(AbstractType.java:390)
 ~[main/:na]
        at 
org.apache.cassandra.db.LegacyLayout$CellGrouper.addCell(LegacyLayout.java:1457)
 ~[main/:na]
        at 
org.apache.cassandra.db.LegacyLayout$CellGrouper.addAtom(LegacyLayout.java:1377)
 ~[main/:na]
        at 
org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.readRow(UnfilteredDeserializer.java:542)
 ~[main/:na]
        at 
org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.hasNext(UnfilteredDeserializer.java:519)
 ~[main/:na]
        at 
org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer.hasNext(UnfilteredDeserializer.java:336)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.computeNext(SSTableIterator.java:140)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.hasNextInternal(SSTableIterator.java:172)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:336)
 ~
{code}

In short, this is a table created on 2.2 side with: {{CREATE TABLE table_0 (key 
text, value text, PRIMARY KEY (key, value)) WITH COMPACT STORAGE;}}

If you write data into this table with thrift ({{ThriftClient#batch_mutate}}), 
you will end up with data in the hidden column, which looks like this on the 
2.1 side:

{code}
select * from ks1.table_0;

 key   | value
-------+-------
 key22 |  key1
 key22 |  key4
 key11 |  key1
 key11 |  key4

(4 rows)
{code}

And like this in sstabledump:

{code}
[
{"key": "key22",
 "cells": [["key1","76616c756531",1589306163593],
           ["key4","76616c756534",1589306163594]]},
{"key": "key11",
 "cells": [["key1","76616c756531",1589306163593],
           ["key4","76616c756534",1589306163594]]}
]
{code}

Of course, on 3.x side you wouldn't be able to see the values.

UPDATE: Was able to reproduce it: 

{code}
java.lang.RuntimeException: 
org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
/Users/ifesdjeen/foss/java/apache-cassandra/data/data/ks1/table_0-05609b80948511eabfa891431c623cd5/md-2-big-Data.db
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2470)
 ~[main/:na]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[na:1.8.0_171]
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165)
 ~[main/:na]
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:137)
 [main/:na]
        at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) 
[main/:na]
        at java.lang.Thread.run(Thread.java:748) [na:1.8.0_171]
Caused by: org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
/Users/ifesdjeen/foss/java/apache-cassandra/data/data/ks1/table_0-05609b80948511eabfa891431c623cd5/md-2-big-Data.db
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:349)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator.hasNext(AbstractSSTableIterator.java:220)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator.hasNext(SSTableIterator.java:33)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:95)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
 ~[main/:na]
        at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
~[main/:na]
        at 
org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:129) 
~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:95)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
 ~[main/:na]
        at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
~[main/:na]
        at 
org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:129) 
~[main/:na]
        at 
org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:129) 
~[main/:na]
        at 
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:131)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:87)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:77)
 ~[main/:na]
        at 
org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:294)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:187)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:180)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:176)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:76) 
~[main/:na]
        at 
org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:341) 
~[main/:na]
        at 
org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1785)
 ~[main/:na]
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2466)
 ~[main/:na]
        ... 5 common frames omitted
Caused by: java.lang.ArrayIndexOutOfBoundsException: 97
        at 
org.apache.cassandra.db.ClusteringPrefix$Deserializer.prepare(ClusteringPrefix.java:431)
 ~[main/:na]
        at 
org.apache.cassandra.db.UnfilteredDeserializer$CurrentDeserializer.prepareNext(UnfilteredDeserializer.java:170)
 ~[main/:na]
        at 
org.apache.cassandra.db.UnfilteredDeserializer$CurrentDeserializer.hasNext(UnfilteredDeserializer.java:151)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.computeNext(SSTableIterator.java:140)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.hasNextInternal(SSTableIterator.java:172)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:336)
 ~[main/:na]
        ... 27 common frames omitted
{code}

Steps: 

1. Create a compact table in 2.1 (or 2.2), doesn't matter
2. Write data to it with thrift
3. Upgrade sstables
4. Try reading data from this table

On latest 3.0, you won't get past {{EmptySerializer#validate}}, because of 
[CASSANDRA-15373], so you'll have to skip validation manually or use a version 
that doesn't contain a fix.

But this only confirms that we should reconsider the way we treat EmptyType and 
thrift values.

UPDATE2: if we're making a thrift write against a fresh 3.0 node with compact 
storage, we'll end up with the same stack trace.


was (Author: ifesdjeen):
I've taken a short look and wanted to precaution against ignoring or even 
purging values from the hidden compact column, since there are scenarios when 
it can contain legitimate data. 

I've been able to reproduce an issue with a similar, albeit not exactly same 
stacktrace (leaving only the part from sstable iterator downward):

{code}
ERROR 18:08:11 Uncaught exception on thread Thread[SharedPool-Worker-2,10,main]
java.lang.RuntimeException: org.apache.cassandra.serializers.MarshalException: 
EmptyType only accept empty values
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2470)
 ~[main/:na]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[na:1.8.0_171]
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165)
 ~[main/:na]
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:137)
 [main/:na]
        at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) 
[main/:na]
        at java.lang.Thread.run(Thread.java:748) [na:1.8.0_171]
Caused by: org.apache.cassandra.serializers.MarshalException: EmptyType only 
accept empty values
        at 
org.apache.cassandra.serializers.EmptySerializer.validate(EmptySerializer.java:42)
 ~[main/:na]
        at 
org.apache.cassandra.db.marshal.AbstractType.validate(AbstractType.java:159) 
~[main/:na]
        at 
org.apache.cassandra.db.marshal.AbstractType.validateIfFixedSize(AbstractType.java:390)
 ~[main/:na]
        at 
org.apache.cassandra.db.LegacyLayout$CellGrouper.addCell(LegacyLayout.java:1457)
 ~[main/:na]
        at 
org.apache.cassandra.db.LegacyLayout$CellGrouper.addAtom(LegacyLayout.java:1377)
 ~[main/:na]
        at 
org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.readRow(UnfilteredDeserializer.java:542)
 ~[main/:na]
        at 
org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.hasNext(UnfilteredDeserializer.java:519)
 ~[main/:na]
        at 
org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer.hasNext(UnfilteredDeserializer.java:336)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.computeNext(SSTableIterator.java:140)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.hasNextInternal(SSTableIterator.java:172)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:336)
 ~
{code}

In short, this is a table created on 2.2 side with: {{CREATE TABLE table_0 (key 
text, value text, PRIMARY KEY (key, value)) WITH COMPACT STORAGE;}}

If you write data into this table with thrift ({{ThriftClient#batch_mutate}}), 
you will end up with data in the hidden column, which looks like this on the 
2.1 side:

{code}
select * from ks1.table_0;

 key   | value
-------+-------
 key22 |  key1
 key22 |  key4
 key11 |  key1
 key11 |  key4

(4 rows)
{code}

And like this in sstabledump:

{code}
[
{"key": "key22",
 "cells": [["key1","76616c756531",1589306163593],
           ["key4","76616c756534",1589306163594]]},
{"key": "key11",
 "cells": [["key1","76616c756531",1589306163593],
           ["key4","76616c756534",1589306163594]]}
]
{code}

Of course, on 3.x side you wouldn't be able to see the values.

UPDATE: Was able to reproduce it: 

{code}
java.lang.RuntimeException: 
org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
/Users/ifesdjeen/foss/java/apache-cassandra/data/data/ks1/table_0-05609b80948511eabfa891431c623cd5/md-2-big-Data.db
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2470)
 ~[main/:na]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[na:1.8.0_171]
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165)
 ~[main/:na]
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:137)
 [main/:na]
        at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) 
[main/:na]
        at java.lang.Thread.run(Thread.java:748) [na:1.8.0_171]
Caused by: org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
/Users/ifesdjeen/foss/java/apache-cassandra/data/data/ks1/table_0-05609b80948511eabfa891431c623cd5/md-2-big-Data.db
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:349)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator.hasNext(AbstractSSTableIterator.java:220)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator.hasNext(SSTableIterator.java:33)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:95)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
 ~[main/:na]
        at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
~[main/:na]
        at 
org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:129) 
~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:95)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
 ~[main/:na]
        at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
~[main/:na]
        at 
org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:129) 
~[main/:na]
        at 
org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:129) 
~[main/:na]
        at 
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:131)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:87)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:77)
 ~[main/:na]
        at 
org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:294)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:187)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:180)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:176)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:76) 
~[main/:na]
        at 
org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:341) 
~[main/:na]
        at 
org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1785)
 ~[main/:na]
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2466)
 ~[main/:na]
        ... 5 common frames omitted
Caused by: java.lang.ArrayIndexOutOfBoundsException: 97
        at 
org.apache.cassandra.db.ClusteringPrefix$Deserializer.prepare(ClusteringPrefix.java:431)
 ~[main/:na]
        at 
org.apache.cassandra.db.UnfilteredDeserializer$CurrentDeserializer.prepareNext(UnfilteredDeserializer.java:170)
 ~[main/:na]
        at 
org.apache.cassandra.db.UnfilteredDeserializer$CurrentDeserializer.hasNext(UnfilteredDeserializer.java:151)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.computeNext(SSTableIterator.java:140)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.hasNextInternal(SSTableIterator.java:172)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:336)
 ~[main/:na]
        ... 27 common frames omitted
{code}

Steps: 

1. Create a compact table in 2.1 (or 2.2), doesn't matter
2. Write data to it with thrift
3. Upgrade sstables
4. Try reading data from this table

On latest 3.0, you won't get past {{EmptySerializer#validate}}, because of 
[CASSANDRA-15373], so you'll have to skip validation manually or use a version 
that doesn't contain a fix.

But this only confirms that we should reconsider the way we treat EmptyType and 
thrift values.

> CorruptSSTableException after a 2.1 SSTable is upgraded to 3.0, failing reads
> -----------------------------------------------------------------------------
>
>                 Key: CASSANDRA-15778
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-15778
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Local/Compaction, Local/SSTable
>            Reporter: Sumanth Pasupuleti
>            Assignee: David Capwell
>            Priority: Normal
>             Fix For: 3.0.x
>
>
> Below is the exception with stack trace. This issue is consistently 
> reproduce-able.
> {code:java}
> ERROR [SharedPool-Worker-1] 2020-05-01 14:57:57,661 
> AbstractLocalAwareExecutorService.java:169 - Uncaught exception on thread 
> Thread[SharedPool-Worker-1,5,main]ERROR [SharedPool-Worker-1] 2020-05-01 
> 14:57:57,661 AbstractLocalAwareExecutorService.java:169 - Uncaught exception 
> on thread 
> Thread[SharedPool-Worker-1,5,main]org.apache.cassandra.io.sstable.CorruptSSTableException:
>  Corrupted: 
> /mnt/data/cassandra/data/<ks>/<cf-fda511301fb311e7bd79fd24f2fcfb0d/md-10151-big-Data.db
>  at 
> org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:349)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.columniterator.AbstractSSTableIterator.hasNext(AbstractSSTableIterator.java:220)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.columniterator.SSTableIterator.hasNext(SSTableIterator.java:33)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:95)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
> ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:129) 
> ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:95)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
> ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:129) 
> ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:129) 
> ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:131)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:87)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:77)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:294)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:187)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:180)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:176)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:76) 
> ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:341) 
> ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.ReadCommandVerbHandler.doVerb(ReadCommandVerbHandler.java:47)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:67) 
> ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> ~[na:1.8.0_231] at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:137)
>  [nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) 
> [nf-cassandra-3.0.19.8.jar:3.0.19.8] at java.lang.Thread.run(Thread.java:748) 
> [na:1.8.0_231]Caused by: java.lang.ArrayIndexOutOfBoundsException: 121 at 
> org.apache.cassandra.db.ClusteringPrefix$Deserializer.prepare(ClusteringPrefix.java:425)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.UnfilteredDeserializer$CurrentDeserializer.prepareNext(UnfilteredDeserializer.java:170)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.UnfilteredDeserializer$CurrentDeserializer.hasNext(UnfilteredDeserializer.java:151)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.computeNext(SSTableIterator.java:140)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.hasNextInternal(SSTableIterator.java:172)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] at 
> org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:336)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] ... 27 common frames omitted
> Caused by: java.lang.ArrayIndexOutOfBoundsException: 121
>     at 
> org.apache.cassandra.db.ClusteringPrefix$Deserializer.prepare(ClusteringPrefix.java:425)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8]
>     at 
> org.apache.cassandra.db.UnfilteredDeserializer$CurrentDeserializer.prepareNext(UnfilteredDeserializer.java:170)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8]
>     at 
> org.apache.cassandra.db.UnfilteredDeserializer$CurrentDeserializer.hasNext(UnfilteredDeserializer.java:151)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8]
>     at 
> org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.computeNext(SSTableIterator.java:140)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8]
>     at 
> org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.hasNextInternal(SSTableIterator.java:172)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8]
>     at 
> org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:336)
>  ~[nf-cassandra-3.0.19.8.jar:3.0.19.8] ... 27 common frames omitted
> {code}
> Column family definition
> {code:java}
> CREATE TABLE <keyspace>."<cf>" (
>  key text,
>  value text,
>  PRIMARY KEY (key, value)
>  ) WITH COMPACT STORAGE
>  AND CLUSTERING ORDER BY (value ASC)
>  AND bloom_filter_fp_chance = 0.01
>  AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}
>  AND comment = ''
>  AND compaction = {'class': 
> 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 
> 'max_threshold': '32', 'min_threshold': '4'}
>  AND compression = {'enabled': 'false'}
>  AND crc_check_chance = 1.0
>  AND dclocal_read_repair_chance = 0.1
>  AND default_time_to_live = 0
>  AND gc_grace_seconds = 864000
>  AND max_index_interval = 2048
>  AND memtable_flush_period_in_ms = 0
>  AND min_index_interval = 128
>  AND read_repair_chance = 0.0
>  AND speculative_retry = '99PERCENTILE';{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to