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

Dmitry Konstantinov updated CASSANDRA-20396:
--------------------------------------------
     Bug Category: Parent values: Correctness(12982)Level 1 values: Test 
Failure(12990)
       Complexity: Normal
      Component/s:     (was: Test/unit)
    Discovered By: Unit Test
         Severity: Low
           Status: Open  (was: Triage Needed)

cc [~blambov] (I think I will need your review for a fix here)

I have analyzed the issue - it looks like a real regression:
 * the test creates a set of SSTables and corrupt some of them in a random 
place by writing a slice of random bytes. Then major compaction is invoked 
multiple times: it is expected that corrupted SSTables are marked as suspected 
by compaction and excluded from the next invocations of compactions, so finally 
we exclude all corrupted SSTables and compact the remaining normal ones.
 * In our case the same corrupted SSTable is present again and again in each 
compaction causing it to fail which means the corrupted SSTable is not marked 
as suspected now.
 * the issue is observing only for "test-compression" Ant target execution. I 
have reproduced it in IDE as well by taking a single parameter from the Ant 
target: {_}-Dcassandra.test.compression=true{_}. The parameter defines enables 
compression for SSTables (by default the test create non-compressed tables).
 * 5.0 does not suffer from this issue. I have bisected recent commits to trunk 
and found then it was started: CASSANDRA-20092. So the new scanner logic does 
not mark an SSTable as corrupted when we read a bad compressed block.
Stacktrace of the invocation:
{code:java}
ERROR [CompactionExecutor:4] 2025-03-04 12:47:00,885 Exception in thread 
Thread[CompactionExecutor:4,5,CompactionExecutor]
org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: 
/home/cassandra/cassandra/build/test/cassandra/data/CorruptedSSTablesCompactionsTest/Standard_LCS-c39a304af8f611ef894ef9e5dfadae6d/oa-18-big-Data.db
    at 
org.apache.cassandra.io.util.CompressedChunkReader$Standard.readChunk(CompressedChunkReader.java:171)
    at org.apache.cassandra.cache.ChunkCache.load(ChunkCache.java:164)
    at org.apache.cassandra.cache.ChunkCache.load(ChunkCache.java:46)
    at 
com.github.benmanes.caffeine.cache.LocalLoadingCache.lambda$newMappingFunction$3(LocalLoadingCache.java:183)
    at 
com.github.benmanes.caffeine.cache.LocalCache.lambda$statsAware$2(LocalCache.java:167)
    at 
com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$doComputeIfAbsent$14(BoundedLocalCache.java:2688)
    at 
java.base/java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1908)
    at 
com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:2686)
    at 
com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:2669)
    at 
com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:112)
    at 
com.github.benmanes.caffeine.cache.LocalLoadingCache.get(LocalLoadingCache.java:58)
    at 
org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:232)
    at 
org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:211)
    at 
org.apache.cassandra.io.util.RandomAccessReader.reBufferAt(RandomAccessReader.java:67)
    at 
org.apache.cassandra.io.util.RandomAccessReader.seek(RandomAccessReader.java:209)
    at 
org.apache.cassandra.io.sstable.format.SSTableSimpleScanner.advanceRange(SSTableSimpleScanner.java:162)
    at 
org.apache.cassandra.io.sstable.format.SSTableSimpleScanner.hasNext(SSTableSimpleScanner.java:146)

    at 
org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:375)
    at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:187)
    at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:156)
    at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
    at 
org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$2.hasNext(UnfilteredPartitionIterators.java:201)
    at 
org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:90)
    at 
org.apache.cassandra.db.compaction.CompactionIterator.hasNext(CompactionIterator.java:304)
    at 
org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:256)
    at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:26)
    at 
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:90)
    at 
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:110)
    at 
org.apache.cassandra.db.compaction.CompactionManager$9.runMayThrow(CompactionManager.java:1166)
    at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:26)
    at org.apache.cassandra.concurrent.FutureTask$3.call(FutureTask.java:141)
    at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
    at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
    at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
    at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
    at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
    at java.base/java.lang.Thread.run(Thread.java:829)
Caused by: org.apache.cassandra.io.compress.CorruptBlockException: 
(/home/cassandra/cassandra/build/test/cassandra/data/CorruptedSSTablesCompactionsTest/Standard_LCS-c39a304af8f611ef894ef9e5dfadae6d/oa-18-big-Data.db):
 corruption detected, chunk at 0 of length 167.
    at 
org.apache.cassandra.io.util.CompressedChunkReader$Standard.readChunk(CompressedChunkReader.java:133)
    ... 36 common frames omitted
{code}
vs the old scanner stacktrace for the same case where we did the marking:
{code:java}
"CompactionExecutor:1@7879" daemon prio=5 tid=0x29 nid=NA runnable
  java.lang.Thread.State: RUNNABLE
      at 
org.apache.cassandra.io.sstable.format.SSTableReader.markSuspect(SSTableReader.java:915)
      at 
org.apache.cassandra.io.sstable.format.SSTableScanner$BaseKeyScanningIterator.computeNext(SSTableScanner.java:294)
      at 
org.apache.cassandra.io.sstable.format.SSTableScanner$BaseKeyScanningIterator.computeNext(SSTableScanner.java:244)
      at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
      at 
org.apache.cassandra.io.sstable.format.SSTableScanner.hasNext(SSTableScanner.java:206)

      at 
org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:375)
      at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:187)
      at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:156)
      at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
      at 
org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$2.hasNext(UnfilteredPartitionIterators.java:201)
      at 
org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:90)
      at 
org.apache.cassandra.db.compaction.CompactionIterator.hasNext(CompactionIterator.java:304)
      at 
org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:206)
      at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:26)
      at 
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:86)
      at 
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:100)
      at 
org.apache.cassandra.db.compaction.CompactionManager$9.runMayThrow(CompactionManager.java:1039)
      at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:26)
      at org.apache.cassandra.concurrent.FutureTask$3.call(FutureTask.java:141)
      at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
      at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
      at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
      at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
      at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
      at java.lang.Thread.run(Thread.java:840)
{code}
* More specifically we do not process CorruptSSTableException thrown by 
dfile.seek() in 
org.apache.cassandra.io.sstable.format.SSTableSimpleScanner#advanceRange

I am preparing a patch for the issue and going to share it soon.

> Test failure: CorruptedSSTablesCompactionsTest
> ----------------------------------------------
>
>                 Key: CASSANDRA-20396
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-20396
>             Project: Apache Cassandra
>          Issue Type: Bug
>          Components: Local/SSTable
>            Reporter: Dmitry Konstantinov
>            Assignee: Dmitry Konstantinov
>            Priority: Normal
>             Fix For: 5.x
>
>         Attachments: 
> TEST-org.apache.cassandra.db.compaction.CorruptedSSTablesCompactionsTest.log.xz,
>  
> TEST-org.apache.cassandra.db.compaction.CorruptedSSTablesCompactionsTest.xml.xz,
>  image-2025-03-04-21-46-34-829.png
>
>
> CorruptedSSTablesCompactionsTest is failing for trunk on CI:
>  * [https://ci-cassandra.apache.org/job/Cassandra-trunk/2049/]
>  * [https://ci-cassandra.apache.org/job/Cassandra-trunk/2044/] 
>  * 
> [https://ci-cassandra.apache.org/job/Cassandra-trunk/2043/|https://ci-cassandra.apache.org/job/Cassandra-trunk/2043/#showFailuresLink]
> with errors like:
> {code:java}
> junit.framework.AssertionFailedError: expected:<8> but was:<25>
>       at 
> org.apache.cassandra.db.compaction.CorruptedSSTablesCompactionsTest.testCorruptedSSTables(CorruptedSSTablesCompactionsTest.java:267)
>       at 
> org.apache.cassandra.db.compaction.CorruptedSSTablesCompactionsTest.testCorruptedSSTablesWithSizeTieredCompactionStrategy(CorruptedSSTablesCompactionsTest.java:146)
>       at 
> java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
>       at 
> java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>  {code}
> note: the failure is observing for test-compression suite



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