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

Jacek Lewandowski commented on CASSANDRA-18737:
-----------------------------------------------

So far, I don't understand that test fully, some statements and comments seem 
misleading to me, in particular:

{code:java}
        ColumnFamilyStore cfs = 
Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD2);
        Util.flush(cfs); // wait for sstables to be on disk else we won't be 
able to stream them
{code}

Since we use an external writer, it has nothing to do with flushing; I've 
checked - the number of live sstables before and after flushing is zero. It is 
expected and contradicts the purpose of flushing (as well as the comment). The 
writer saves sstables independently and asynchronously in a loop when the 
buffer gets exhausted. 

Next:


{code:java}
        //writer is still open so loader should not load anything
        SSTableLoader loader = new SSTableLoader(dataDir, new TestClient(), new 
OutputHandler.SystemOutput(false, false));
        loader.stream(Collections.emptySet(), 
completionStreamListener(latch)).get();

        List<FilteredPartition> partitions = Util.getAll(Util.cmd(cfs).build());

        assertTrue(partitions.size() > 0 && partitions.size() < NB_PARTITIONS);
{code}

The comment actually contradicts the assertion. We expect to load something, 
but not everything.

Now, what may be causing the problem? {{SSTableLoader}} does not require the 
{{STATS}} component to be present to consider a file to load. This works the 
same way in pre-17056 as in post-17056. This may be a problem in both versions 
because we can only load an sstable with {{STATS}} component. SSTables are 
stored asynchronously in a separate thread in 
{{SSTableSimpleUnsortedWriter.DiskWriter}}. {{DATA}} and {{PRIMARY_INDEX}} 
components are stored first, then {{STATS}} component is stored. The problem 
looks as is to be caused by trying to load an sstable for which {{DATA}} and 
{{PRIMARY_INDEX}} are saved, but {{STATS}} still needs to be saved. So, it is a 
race that explains this failure's flaky nature. 

I'm planning to investigate the transactions to check if SSTableLoader shall 
reject the sstables with unfinished components.

> Test failure: org.apache.cassandra.io.sstable.SSTableLoaderTest 
> (testLoadingIncompleteSSTable-.jdk17)
> -----------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-18737
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-18737
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Tool/bulk load
>            Reporter: Brandon Williams
>            Assignee: Jacek Lewandowski
>            Priority: Normal
>             Fix For: 5.0.x, 5.x
>
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> {noformat}
> java.lang.RuntimeException: Failed to list files in 
> /tmp/1409486429862512729/SSTableLoaderTest/Standard2-57877ac036d311eea01f83fcb8f6fee5
>       at 
> org.apache.cassandra.db.lifecycle.LogAwareFileLister.list(LogAwareFileLister.java:77)
>       at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.getFiles(LifecycleTransaction.java:626)
>       at 
> org.apache.cassandra.io.sstable.SSTableLoader.openSSTables(SSTableLoader.java:103)
>       at 
> org.apache.cassandra.io.sstable.SSTableLoader.stream(SSTableLoader.java:202)
>       at 
> org.apache.cassandra.io.sstable.SSTableLoaderTest.testLoadingIncompleteSSTable(SSTableLoaderTest.java:213)
>       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)
> Caused by: org.apache.cassandra.io.sstable.CorruptSSTableException: 
> Corrupted: 
> /tmp/1409486429862512729/SSTableLoaderTest/Standard2-57877ac036d311eea01f83fcb8f6fee5/nc-17-big
>       at 
> org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:111)
>       at 
> org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:397)
>       at 
> org.apache.cassandra.io.sstable.format.SSTableReader.openForBatch(SSTableReader.java:373)
>       at 
> org.apache.cassandra.io.sstable.SSTableLoader.lambda$openSSTables$0(SSTableLoader.java:152)
>       at 
> org.apache.cassandra.db.lifecycle.LogAwareFileLister.lambda$innerList$2(LogAwareFileLister.java:99)
>       at 
> java.base/java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:178)
>       at 
> java.base/java.util.TreeMap$EntrySpliterator.forEachRemaining(TreeMap.java:3287)
>       at 
> java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
>       at 
> java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
>       at 
> java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:921)
>       at 
> java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>       at 
> java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:682)
>       at 
> org.apache.cassandra.db.lifecycle.LogAwareFileLister.innerList(LogAwareFileLister.java:101)
>       at 
> org.apache.cassandra.db.lifecycle.LogAwareFileLister.list(LogAwareFileLister.java:73)
> Caused by: java.lang.NullPointerException
>       at 
> com.google.common.base.Preconditions.checkNotNull(Preconditions.java:903)
>       at 
> org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.buildSummaryAndBloomFilter(BigSSTableReaderLoadingBuilder.java:193)
>       at 
> org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:116)
>       at 
> org.apache.cassandra.io.sstable.format.big.BigSSTableReaderLoadingBuilder.openComponents(BigSSTableReaderLoadingBuilder.java:58)
>       at 
> org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder.build(SSTableReaderLoadingBuilder.java:92)
> {noformat}
> Seen here: 
> https://app.circleci.com/pipelines/github/driftx/cassandra/1174/workflows/263f1e22-e4d0-48b8-b3e2-496edb30a068/jobs/41924/tests



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

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to