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

Jakub Zytka commented on CASSANDRA-18589:
-----------------------------------------

Reproduction test is on branch:

[https://github.com/jakubzytka/cassandra/pull/new/CASSANDRA-15589-repro]

 

Reproduction:

{{ant test-jvm-dtest-some 
-Dtest.name=org.apache.cassandra.distributed.test.DropColumnAndReadsTest 
-Dno-checkstyle=1}}

 

Example failure:
{code:java}
 [junit-timeout] Testcase: 
testReadsAfterComplexColumnDrop[0](org.apache.cassandra.distributed.test.DropColumnAndReadsTest)-.jdk11:
    Caused an ERROR
[junit-timeout] Cassandra failure during read query at consistency LOCAL_ONE (1 
responses were required but only 0 replica responded, 1 failed)
[junit-timeout] com.datastax.driver.core.exceptions.ReadFailureException: 
Cassandra failure during read query at consistency LOCAL_ONE (1 responses were 
required but only 0 replica responded, 1 failed)
[junit-timeout]     at 
com.datastax.driver.core.exceptions.ReadFailureException.copy(ReadFailureException.java:180)
[junit-timeout]     at 
com.datastax.driver.core.exceptions.ReadFailureException.copy(ReadFailureException.java:30)
[junit-timeout]     at 
com.datastax.driver.core.DriverThrowables.propagateCause(DriverThrowables.java:35)
[junit-timeout]     at 
com.datastax.driver.core.DefaultResultSetFuture.getUninterruptibly(DefaultResultSetFuture.java:293)
[junit-timeout]     at 
com.datastax.driver.core.AbstractSession.execute(AbstractSession.java:58)
[junit-timeout]     at 
org.apache.cassandra.distributed.test.DropColumnAndReadsTest.testReadsAfterComplexColumnDrop(DropColumnAndReadsTest.java:159)
[junit-timeout]     at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
[junit-timeout]     at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
[junit-timeout]     at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
[junit-timeout]     Suppressed: 
org.apache.cassandra.distributed.shared.ShutdownException: Uncaught exceptions 
were thrown during test
[junit-timeout]         at 
org.apache.cassandra.distributed.impl.AbstractCluster.checkAndResetUncaughtExceptions(AbstractCluster.java:1078)
[junit-timeout]         at 
org.apache.cassandra.distributed.impl.AbstractCluster.close(AbstractCluster.java:1064)
[junit-timeout]         at 
org.apache.cassandra.distributed.test.DropColumnAndReadsTest.testReadsAfterComplexColumnDrop(DropColumnAndReadsTest.java:75)
[junit-timeout]         Suppressed: java.lang.RuntimeException: 
java.lang.NullPointerException
[junit-timeout]             at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2585)
[junit-timeout]             at 
org.apache.cassandra.concurrent.ExecutionFailure$2.run(ExecutionFailure.java:163)
[junit-timeout]             at 
org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:143)
[junit-timeout]             at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
[junit-timeout]             at java.base/java.lang.Thread.run(Thread.java:829)
[junit-timeout]         Caused by: java.lang.NullPointerException
[junit-timeout]             at 
org.apache.cassandra.db.rows.Row$Merger$ColumnDataReducer.getReduced(Row.java:854)
[junit-timeout]             at 
org.apache.cassandra.db.rows.Row$Merger$ColumnDataReducer.getReduced(Row.java:793)
[junit-timeout]             at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:219)
[junit-timeout]             at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:158)
[junit-timeout]             at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
[junit-timeout]             at 
org.apache.cassandra.db.rows.Row$Merger.merge(Row.java:770)
[junit-timeout]             at 
org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator$MergeReducer.getReduced(UnfilteredRowIterators.java:588)
[junit-timeout]             at 
org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator$MergeReducer.getReduced(UnfilteredRowIterators.java:552)
[junit-timeout]             at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:219)
[junit-timeout]             at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:158)
[junit-timeout]             at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
[junit-timeout]             at 
org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:534)
[junit-timeout]             at 
org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:402)
[junit-timeout]             at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
[junit-timeout]             at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:95)
[junit-timeout]             at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
[junit-timeout]             at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
[junit-timeout]             at 
org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:133)
[junit-timeout]             at 
org.apache.cassandra.db.transform.UnfilteredRows.isEmpty(UnfilteredRows.java:74)
[junit-timeout]             at 
org.apache.cassandra.db.partitions.PurgeFunction.applyToPartition(PurgeFunction.java:86)
[junit-timeout]             at 
org.apache.cassandra.db.partitions.PurgeFunction.applyToPartition(PurgeFunction.java:26)
[junit-timeout]             at 
org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:95)
[junit-timeout]             at 
org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:303)
[junit-timeout]             at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:201)
[junit-timeout]             at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:186)
[junit-timeout]             at 
org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:48)
[junit-timeout]             at 
org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:341)
[junit-timeout]             at 
org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:2186)
[junit-timeout]             at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2581)
{code}

> NPE during reads after complex column drop
> ------------------------------------------
>
>                 Key: CASSANDRA-18589
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-18589
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Local/SSTable
>            Reporter: Jakub Zytka
>            Assignee: Jakub Zytka
>            Priority: Normal
>
> When writing data in parallel with dropping a complex column, the subsequent 
> reads may fail with NPE until the affected sstable is compacted. 
>  
> The scenario leading to NPE is as follows: there exists a row which contains 
> data for a complex column that is now dropped. There are no other complex 
> columns. The removed column is not skipped during deserialization of the row 
> (ColumnFilter is not aware of dropped columns).
> At the same time, {{Row$Merger$ColumnDataReducer}} is not aware of existence 
> of a complex column ({{{}hasComplex==false{}}}) and thus doesn't have a 
> builder for complex data, eventually yielding NPE when processing said 
> complex column (backtrace from 3.11):
> {{ERROR [ReadStage-2] node2 2023-06-13 11:00:46,756 Uncaught exception on 
> thread Thread[ReadStage-2,5,node2]}}
> {{java.lang.RuntimeException: java.lang.NullPointerException}}
> {{        at 
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2777)}}
> {{        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:113)}}
> {{        at java.lang.Thread.run(Thread.java:748)}}
> {{Caused by: java.lang.NullPointerException: null}}
> {{        at 
> org.apache.cassandra.db.rows.Row$Merger$ColumnDataReducer.getReduced(Row.java:789)}}
> {{        at 
> org.apache.cassandra.db.rows.Row$Merger$ColumnDataReducer.getReduced(Row.java:726)}}
> {{        at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:217)}}
> {{        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.rows.Row$Merger.merge(Row.java:703)}}
> {{        at 
> org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator$MergeReducer.getReduced(UnfilteredRowIterators.}}
> {{java:587)}}
> {{        at 
> org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator$MergeReducer.getReduced(UnfilteredRowIterators.java:551)}}
> {{        at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:217)}}
> {{        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.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:533)}}
> {{        at 
> org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:390)}}
> {{        at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)}}
> {{        at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:100)}}
> {{        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.UnfilteredRows.isEmpty(UnfilteredRows.java:74)}}
> {{        at 
> org.apache.cassandra.db.partitions.PurgeFunction.applyToPartition(PurgeFunction.java:75)}}
> {{        at 
> org.apache.cassandra.db.partitions.PurgeFunction.applyToPartition(PurgeFunction.java:26)}}
> {{        at 
> org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:96)}}
> {{        at 
> org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:305)}}
> {{        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:360)}}
> {{        at 
> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:2007)}}
> {{        at 
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2773)}}
> The NPE problem races with another problem in that scenario 
> (CASSANDRA-18591), so running the reproduction test YMMV which one you hit.
>  
> While it may be tempting to fix the NPE by lazy initialization of the needed 
> builder structure et al., it seems that there is an implicit assumption that 
> columns like the dropped one should not get into read path machinery at all 
> at this point. 
> Thus, instead of just fixing the NPE and hoping no other class makes such an 
> assumption I intend to instead make the assumption valid by cutting out the 
> dropped column as soon as possible (i.e. during deserialization)
> I don't know if I need to care about memtable (instead of sstable contents 
> only).
> I don't think schema agreement etc. is relevant - currently the ColumnFilter 
> uses some specific TableMetadata, so if I use the very same TableMetadata as 
> the source of dropped column info there should be internal consistency 
> between ColumnFilter and the ColumnDataReducer (or potentially, other classes)
> Thoughts? [~blerer] [~blambov] 



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