Klay created CASSANDRA-19623:
--------------------------------

             Summary: Read fail with "illegal RT bounds sequence" after 
migrating data from 2.2.19 to 3.0.30
                 Key: CASSANDRA-19623
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-19623
             Project: Cassandra
          Issue Type: Bug
          Components: Legacy/Local Write-Read Paths
            Reporter: Klay
         Attachments: cassandra.yaml, data.tar.gz, system.log

After migrating data from 2.2.19 to 3.0.30, reading the legacy data in 3.0.30 
would fail with the following exception.

 
{code:java}
INFO  [main] 2024-05-08 00:42:42,397 CassandraDaemon.java:653 - Startup complete
ERROR [SharedPool-Worker-2] 2024-05-08 00:42:49,975 
AbstractLocalAwareExecutorService.java:166 - Uncaught exception on thread 
Thread[SharedPool-Worker-2,10,main]
java.lang.RuntimeException: java.lang.IllegalStateException: SSTABLE 
UnfilteredRowIterator for ks.tb has an illegal RT bounds sequence: unexpected 
end bound or boundary Marker INCL_START_BOUND(1)@1715128933051779/1715128933
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2656)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162)
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:134)
        at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109)
        at java.lang.Thread.run(Thread.java:750)
Caused by: java.lang.IllegalStateException: SSTABLE UnfilteredRowIterator for 
ks.tb has an illegal RT bounds sequence: unexpected end bound or boundary 
Marker INCL_START_BOUND(1)@1715128933051779/1715128933
        at 
org.apache.cassandra.db.transform.RTBoundValidator$RowsTransformation.ise(RTBoundValidator.java:120)
        at 
org.apache.cassandra.db.transform.RTBoundValidator$RowsTransformation.applyToMarker(RTBoundValidator.java:87)
        at org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:144)
        at org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:129)
        at 
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:131)
        at 
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:87)
        at 
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:77)
        at 
org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:297)
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:187)
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:180)
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:176)
        at 
org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:76)
        at 
org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:347)
        at 
org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1914)
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2652)
        ... 5 common frames omitted {code}
 
h1. Reproduce

1. Start up cassandra-2.2.19, single node. Set column_index_size_in_kb in 
cassandra.yaml to 1.

 
{code:java}
// cassandra.yaml
column_index_size_in_kb: 1 {code}
2. Execute the following commands (I masked out all data in the INSERT command 
using long string of A to make it simple)

 

 
{code:java}
CREATE KEYSPACE  ks WITH REPLICATION = { 'class' : 'SimpleStrategy', 
'replication_factor' : 1 };
CREATE TABLE  ks.tb (c3 INT, c7 INT, c0 TEXT,c6 set<TEXT>, PRIMARY KEY (c3, c7, 
c0));DELETE FROM ks.tb WHERE c3 = 1 AND c7 = 1;
DELETE FROM ks.tb WHERE c3 = 1 AND c7 = 1 AND c0 = 'e';INSERT INTO ks.tb (c7, 
c3, c6, c0) VALUES 
(1,1,{'AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA','AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA','AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA'},'AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA');
 {code}
 

Perform a read in the old version, it works normally

 
{code:java}
cqlsh> SELECT c7, c0, c3 FROM ks.tb WHERE c3 = 1 AND c7 = 1 ORDER BY c7 DESC;

 c7 | c0                                               | c3
----+--------------------------------------------------+----
  1 | AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA |  1(1 rows) {code}
 

3. Drain and stop 2.2.19.

 
{code:java}
bin/nodetool -h ::FFFF:127.0.0.1 drain
bin/nodetool -h ::FFFF:127.0.0.1 stopdaemon; {code}
4. Copy data into 3.0.30 and start up (using default config is okay), read the 
legacy data with the following read command

 

 
{code:java}
SELECT c7, c0, c3 FROM ks.tb WHERE c3 = 1 AND c7 = 1 ORDER BY c7 DESC; {code}
 

It would fails and cassandra throws an error in system.log.

 
{code:java}
➜  apache-cassandra-3.0.30 bin/cqlsh
Connected to Test Cluster at 127.0.0.1:9042.
[cqlsh 5.0.1 | Cassandra 3.0.30-SNAPSHOT | CQL spec 3.4.0 | Native protocol v4]
Use HELP for help.
cqlsh> SELECT c7, c0, c3 FROM ks.tb WHERE c3 = 1 AND c7 = 1 ORDER BY c7 DESC;
ReadFailure: Error from server: code=1300 [Replica(s) failed to execute read] 
message="Operation failed - received 0 responses and 1 failures" 
info={'failures': 1, 'received_responses': 0, 'required_responses': 1, 
'consistency': 'ONE'} {code}
system.log
{code:java}
INFO  [main] 2024-05-08 00:52:11,649 CassandraDaemon.java:653 - Startup complete
ERROR [SharedPool-Worker-2] 2024-05-08 00:52:13,695 
AbstractLocalAwareExecutorService.java:166 - Uncaught exception on thread 
Thread[SharedPool-Worker-2,10,main]
java.lang.RuntimeException: java.lang.IllegalStateException: SSTABLE 
UnfilteredRowIterator for ks.tb has an illegal RT bounds sequence: unexpected 
end bound or boundary Marker INCL_START_BOUND(1)@1715129489466439/1715129489
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2656)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162)
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:134)
        at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109)
        at java.lang.Thread.run(Thread.java:750)
Caused by: java.lang.IllegalStateException: SSTABLE UnfilteredRowIterator for 
ks.tb has an illegal RT bounds sequence: unexpected end bound or boundary 
Marker INCL_START_BOUND(1)@1715129489466439/1715129489
        at 
org.apache.cassandra.db.transform.RTBoundValidator$RowsTransformation.ise(RTBoundValidator.java:120)
        at 
org.apache.cassandra.db.transform.RTBoundValidator$RowsTransformation.applyToMarker(RTBoundValidator.java:87)
        at org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:144)
        at org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:129)
        at 
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:131)
        at 
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:87)
        at 
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:77)
        at 
org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:297)
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:187)
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:180)
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:176)
        at 
org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:76)
        at 
org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:347)
        at 
org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1914)
        at org.apache.cassandra.service {code}
I attached the data.tar.gz file generated from the 2.2.19, using 3.0.30 to 
start with it and execute the above read command can directly reproduce it.

I also attached the cassandra.yaml file used in 2.2.19.

 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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

Reply via email to