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

Alex Petrov commented on CASSANDRA-18932:
-----------------------------------------

+1 on the patch. 

I feel like we need to also add a symptomatic test, ie one that would attempt 
to cross index boundary, like this one:
{code:java}
    @Test
    public void silentDataLossTest() throws Throwable
    {
        try (Cluster cluster = builder().withNodes(1)
                                        .withConfig((cfg) -> 
cfg.set("column_index_size", "1KiB"))
                                        .start())
        {
            cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS 
distributed_test_keyspace WITH replication = {'class': 'SimpleStrategy', 
'replication_factor': 1};");
            cluster.schemaChange("CREATE TABLE IF NOT EXISTS 
distributed_test_keyspace.sut (pk1 bigint,ck1 bigint,v1 ascii, PRIMARY KEY 
(pk1, ck1));");
            cluster.schemaChange("CREATE TABLE IF NOT EXISTS 
distributed_test_keyspace.model (pk1 bigint,ck1 bigint,v1 ascii, PRIMARY KEY 
(pk1, ck1));");

            for (int size = 0; size < 1000; size += 20)
            {
                long pk = size;
                String longString = "";
                for (int i = 0; i < size; i++)
                    longString += "a";

                for (String tbl : new String[]{ "model", "sut" })
                {
                    cluster.coordinator(1).execute("INSERT INTO 
distributed_test_keyspace." + tbl + " (pk1,ck1,v1) VALUES (?, ?, ?);", QUORUM,
                                                   pk, 0L, longString);
                    cluster.coordinator(1).execute("INSERT INTO 
distributed_test_keyspace." + tbl + " (pk1,ck1) VALUES (?, ?);", QUORUM,
                                                   pk, 1L);
                    cluster.coordinator(1).execute("DELETE FROM 
distributed_test_keyspace." + tbl + " WHERE pk1 = ? AND ck1 > ? AND ck1 < ?;", 
QUORUM,
                                                   pk, 1L, 5L);
                    cluster.coordinator(1).execute("INSERT INTO 
distributed_test_keyspace." + tbl + " (pk1,ck1,v1) VALUES (?, ?, ?);", QUORUM,
                                                   pk, 2L, longString);
                    cluster.coordinator(1).execute("DELETE FROM 
distributed_test_keyspace." + tbl + " WHERE pk1 = ? AND ck1 > ? AND ck1 < ?;", 
QUORUM,
                                                   pk, 2L, 5L);
                }
                cluster.get(1).nodetool("flush", "distributed_test_keyspace", 
"sut");
                Iterator<Object[]> iterSut = 
cluster.coordinator(1).executeWithPaging("SELECT * FROM 
distributed_test_keyspace.sut WHERE pk1 = ?;",
                                                                                
      QUORUM, 1, pk);
                Iterator<Object[]> iterModel = 
cluster.coordinator(1).executeWithPaging("SELECT * FROM 
distributed_test_keyspace.model WHERE pk1 = ?;",
                                                                                
        QUORUM, 1, pk);
                while (iterSut.hasNext() && iterModel.hasNext())
                    Assert.assertArrayEquals(iterModel.next(), iterSut.next());

                if (iterModel.hasNext())
                    throw new AssertionError(String.format("Model iterator has 
a row %s", Arrays.asList(iterModel.next())));
                if (iterSut.hasNext())
                    throw new AssertionError(String.format("SUT iterator has a 
row %s", Arrays.asList(iterSut.next())));
            }
        }
    }
{code}
The setup is a bit different here, but it also catches a silent data loss issue 
(CASSANDRA-18993), and iterates through different sizes that allows to check 
more underlying conditions.

> Harry-found CorruptSSTableException / RT Closer issue when reading entire 
> partition
> -----------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-18932
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-18932
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Local/SSTable
>            Reporter: Alex Petrov
>            Assignee: Jacek Lewandowski
>            Priority: Normal
>             Fix For: 5.0-beta, 5.0.x, 5.x
>
>         Attachments: node1_.zip, operation.log.zip, screenshot-1.png
>
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> While testing some new machinery for Harry, I have encountered a new RT 
> closer / SSTable Corruption issue. I have grounds to believe this was 
> introduced during the last year.
> Issue seems to happen because of intricate interleaving of flushes with 
> writes and deletes.
> {code:java}
> ERROR [ReadStage-2] 2023-10-16 18:47:06,696 JVMStabilityInspector.java:76 - 
> Exception in thread Thread[ReadStage-2,5,SharedPool]
> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
> RandomAccessReader:BufferManagingRebufferer.Aligned:CompressedChunkReader.Mmap(/Users/ifesdjeen/foss/java/apache-cassandra-4.0/data/data1/harry/table_1-07c35a606c0a11eeae7a4f6ca489eb0c/nc-5-big-Data.db
>  - LZ4Compressor, chunk length 16384, data length 232569)
>         at 
> org.apache.cassandra.io.sstable.AbstractSSTableIterator$AbstractReader.hasNext(AbstractSSTableIterator.java:381)
>         at 
> org.apache.cassandra.io.sstable.AbstractSSTableIterator.hasNext(AbstractSSTableIterator.java:242)
>         at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:95)
>         at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
>         at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
>         at 
> org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:133)
>         at 
> org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:376)
>         at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:188)
>         at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:157)
>         at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
>         at 
> org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:534)
>         at 
> org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:402)
>         at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
>         at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:95)
>         at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
>         at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
>         at 
> org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:133)
>         at 
> org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:133)
>         at 
> org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:151)
>         at 
> org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:101)
>         at 
> org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:86)
>         at 
> org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:343)
>         at 
> org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:201)
>         at 
> org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:186)
>         at 
> org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:48)
>         at 
> org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:346)
>         at 
> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:2186)
>         at 
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2581)
>         at 
> org.apache.cassandra.concurrent.ExecutionFailure$2.run(ExecutionFailure.java:163)
>         at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:143)
>         at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>         at java.base/java.lang.Thread.run(Thread.java:829)
>         Suppressed: java.lang.IllegalStateException: PROCESSED 
> UnfilteredRowIterator for harry.table_1 (key: 
> ZinzDdUuABgDknItABgDknItABgDknItXEFrgBnOmPmPylWrwXHqjBHgeQrGfnZd1124124583:ZinzDdUuABgDknItABgDknItABgDknItABgDknItABgDknItzHqchghqCXLhVYKM22215251:3.2758E-41
>  omdt: [deletedAt=564416, localDeletion=1697450085]) has an illegal RT bounds 
> sequence: expected all RTs to be closed, but the last one is open
>                 at 
> org.apache.cassandra.db.transform.RTBoundValidator$RowsTransformation.ise(RTBoundValidator.java:117)
>                 at 
> org.apache.cassandra.db.transform.RTBoundValidator$RowsTransformation.onPartitionClose(RTBoundValidator.java:112)
>                 at 
> org.apache.cassandra.db.transform.BaseRows.runOnClose(BaseRows.java:91)
>                 at 
> org.apache.cassandra.db.transform.BaseIterator.close(BaseIterator.java:95)
>                 at 
> org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:341)
>                 ... 10 common frames omitted
> Caused by: java.io.IOException: Invalid Columns subset bytes; too many bits 
> set:10
>         at 
> org.apache.cassandra.db.Columns$Serializer.deserializeSubset(Columns.java:578)
>         at 
> org.apache.cassandra.db.rows.UnfilteredSerializer.deserializeRowBody(UnfilteredSerializer.java:604)
>         at 
> org.apache.cassandra.db.UnfilteredDeserializer.readNext(UnfilteredDeserializer.java:143)
>         at 
> org.apache.cassandra.io.sstable.format.big.SSTableIterator$ForwardIndexedReader.computeNext(SSTableIterator.java:175)
>         at 
> org.apache.cassandra.io.sstable.AbstractSSTableIterator$ForwardReader.hasNextInternal(AbstractSSTableIterator.java:533)
>         at 
> org.apache.cassandra.io.sstable.AbstractSSTableIterator$AbstractReader.hasNext(AbstractSSTableIterator.java:368)
>         ... 31 common frames omitted {code}
>  
> Unfortunately, harry branch is not ready for release yet. That said, I have a 
> snapshot pinned for quick repro and can share SSTables that will easily repro 
> the issue if there are any takers.
> To reproduce:
> {code:java}
> paging 1; # make sure to set page size to 1 (it breaks with other page sizes, 
> too)
> select * from harry.table_1;
> {code}
> Please also make sure to modify snitch to set rack/dc:
> {code:java}
> +    public static final String DATA_CENTER_NAME = "datacenter0";
> +    public static final String RACK_NAME = "rack0";
> {code}
> And set directories in cassandra.yaml: 
> {code:java}
> +data_file_directories:
> + - /your/path/data/data0
> + - /your/path/data/data1
> + -/your/path/data/data2{code}
> SHA for repro: 865d7c30e4755e74c4e4d26205a7aed4cfb55710



--
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